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/05/02 03:56:45 UTC

[01/20] curator git commit: temp Curator/Snapshot dep - added test for TTL nodes

Repository: curator
Updated Branches:
  refs/heads/CURATOR-397 3d593105d -> 2cbbf9992


temp Curator/Snapshot dep - added test for TTL nodes


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

Branch: refs/heads/CURATOR-397
Commit: 521652dcaadfe04bca98bb3249507ca187f32e16
Parents: 027dee3
Author: randgalt <ra...@apache.org>
Authored: Fri Sep 30 14:43:05 2016 +0200
Committer: randgalt <ra...@apache.org>
Committed: Fri Sep 30 14:43:05 2016 +0200

----------------------------------------------------------------------
 .../curator/framework/api/CreateBuilder.java    |   5 +-
 .../framework/imps/CreateBuilderImpl.java       | 128 ++++++++-----------
 .../curator/framework/imps/TestTtlNodes.java    |  88 +++++++++++++
 pom.xml                                         |   2 +-
 4 files changed, 146 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/521652dc/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 564d11b..8c98e0d 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
@@ -18,8 +18,9 @@
  */
 package org.apache.curator.framework.api;
 
-public interface CreateBuilder extends
-    CreateBuilderMain
+public interface CreateBuilder extends CreateBuilderMain
 {
+    CreateBuilderMain withTtl(long ttl);
+
     CreateBuilderMain orSetData();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/521652dc/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 d7126dc..6ec2943 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
@@ -54,6 +54,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     private String protectedId;
     private ACLing acling;
     private Stat storingStat;
+    private long ttl;
 
     @VisibleForTesting
     boolean failNextCreateForTesting = false;
@@ -74,6 +75,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         setDataIfExists = false;
         protectedId = null;
         storingStat = null;
+        ttl = -1;
     }
 
     @Override
@@ -83,6 +85,13 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         return this;
     }
 
+    @Override
+    public CreateBuilderMain withTtl(long ttl)
+    {
+        this.ttl = ttl;
+        return this;
+    }
+
     <T> TransactionCreateBuilder<T> asTransactionCreateBuilder(final T context, final CuratorMultiTransactionRecord transaction)
     {
         return new TransactionCreateBuilder<T>()
@@ -123,7 +132,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                 }
 
                 String fixedPath = client.fixForNamespace(path);
-                transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode), OperationType.CREATE, path);
+                transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode, ttl), OperationType.CREATE, path);
                 return context;
             }
         };
@@ -136,7 +145,8 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         return new CreateBackgroundModeStatACLable()
         {
             @Override
-            public CreateBackgroundModeACLable storingStatIn(Stat stat) {
+            public CreateBackgroundModeACLable storingStatIn(Stat stat)
+            {
                 storingStat = stat;
                 return asCreateBackgroundModeACLable();
             }
@@ -537,80 +547,50 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         {
             final TimeTrace trace = client.getZookeeperClient().startTracer("CreateBuilderImpl-Background");
 
-            if(storingStat == null)
-            {
-                client.getZooKeeper().create
-                (
-                    operationAndData.getData().getPath(),
-                    operationAndData.getData().getData(),
-                    acling.getAclList(operationAndData.getData().getPath()),
-                    createMode,
-                    new AsyncCallback.StringCallback()
-                    {
-                        @Override
-                        public void processResult(int rc, String path, Object ctx, String name)
-                        {
-                            trace.commit();
+            client.getZooKeeper().create
+            (
+                operationAndData.getData().getPath(),
+                operationAndData.getData().getData(),
+                acling.getAclList(operationAndData.getData().getPath()),
+                createMode,
+                new AsyncCallback.Create2Callback() {
 
-                            if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
-                            {
-                                backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
-                            }
-                            else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
-                            {
-                                backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
-                            }
-                            else
-                            {
-                                sendBackgroundResponse(rc, path, ctx, name, null, operationAndData);
-                            }
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
+                        trace.commit();
+
+                        if ( (stat != null) && (storingStat != null) )
+                        {
+                            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());
                         }
-                    },
-                    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();
-
-                            if ( stat != null )
-                            {
-                                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);
-                            }
+                        if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
+                        {
+                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
                         }
-                    },
-                    backgrounding.getContext()
-                );
-            }
+                        else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
+                        {
+                            backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
+                        }
+                        else
+                        {
+                            sendBackgroundResponse(rc, path, ctx, name, stat, operationAndData);
+                        }
+                    }
+                },
+                backgrounding.getContext(),
+                ttl
+            );
         }
         catch ( Throwable e )
         {
@@ -1052,14 +1032,14 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                         {
                             try
                             {
-                                createdPath = client.getZooKeeper().create(path, data, aclList, createMode, storingStat);
+                                createdPath = client.getZooKeeper().create(path, data, aclList, createMode, storingStat, ttl);
                             }
                             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, storingStat);
+                                    createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat, ttl);
                                 }
                                 else
                                 {

http://git-wip-us.apache.org/repos/asf/curator/blob/521652dc/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
new file mode 100644
index 0000000..c544474
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTtlNodes.java
@@ -0,0 +1,88 @@
+/**
+ * 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.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.Timing;
+import org.apache.zookeeper.CreateMode;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.util.concurrent.CountDownLatch;
+
+public class TestTtlNodes extends BaseClassForTests
+{
+    @BeforeMethod
+    @Override
+    public void setup() throws Exception
+    {
+        System.setProperty("znode.container.checkIntervalMs", "1");
+        super.setup();
+    }
+
+    @AfterMethod
+    @Override
+    public void teardown() throws Exception
+    {
+        super.teardown();
+        System.clearProperty("znode.container.checkIntervalMs");
+    }
+
+    @Test
+    public void testBasic() throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
+        {
+            client.start();
+
+            client.create().withTtl(10).creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT_WITH_TTL).forPath("/a/b/c");
+            Thread.sleep(20);
+            Assert.assertNull(client.checkExists().forPath("/a/b/c"));
+        }
+    }
+
+    @Test
+    public void testBasicInBackground() throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
+        {
+            client.start();
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    latch.countDown();
+                }
+            };
+            client.create().withTtl(10).creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT_WITH_TTL).inBackground(callback).forPath("/a/b/c");
+            Assert.assertTrue(new Timing().awaitLatch(latch));
+            Thread.sleep(20);
+            Assert.assertNull(client.checkExists().forPath("/a/b/c"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/521652dc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 905342c..b086df0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -59,7 +59,7 @@
         <jdk-version>1.7</jdk-version>
 
         <!-- versions -->
-        <zookeeper-version>3.5.1-alpha</zookeeper-version>
+        <zookeeper-version>3.6.0-SNAPSHOT</zookeeper-version>
         <maven-project-info-reports-plugin-version>2.7</maven-project-info-reports-plugin-version>
         <maven-bundle-plugin-version>2.3.7</maven-bundle-plugin-version>
         <maven-javadoc-plugin-version>2.10.3</maven-javadoc-plugin-version>


[10/20] curator git commit: correct version of ZK that has TTL nodes

Posted by ra...@apache.org.
correct version of ZK that has TTL nodes


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

Branch: refs/heads/CURATOR-397
Commit: e7d57eccfc756efb2cac83edf25da9d57f740553
Parents: 9654778
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 3 10:33:56 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 3 10:33:56 2017 -0500

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/e7d57ecc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b086df0..4027ae7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -59,7 +59,7 @@
         <jdk-version>1.7</jdk-version>
 
         <!-- versions -->
-        <zookeeper-version>3.6.0-SNAPSHOT</zookeeper-version>
+        <zookeeper-version>3.5.3-beta</zookeeper-version>
         <maven-project-info-reports-plugin-version>2.7</maven-project-info-reports-plugin-version>
         <maven-bundle-plugin-version>2.3.7</maven-bundle-plugin-version>
         <maven-javadoc-plugin-version>2.10.3</maven-javadoc-plugin-version>


[07/20] curator git commit: check for all events on test node

Posted by ra...@apache.org.
check for all events on test node


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

Branch: refs/heads/CURATOR-397
Commit: be4960a522ca91519671708a3a59530fd1a16850
Parents: 7aa5617
Author: randgalt <ra...@apache.org>
Authored: Fri Dec 9 15:37:47 2016 +0100
Committer: randgalt <ra...@apache.org>
Committed: Fri Dec 9 15:37:47 2016 +0100

----------------------------------------------------------------------
 .../framework/recipes/nodes/TestPersistentTtlNode.java        | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/be4960a5/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
index 848c034..be38f6e 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -79,12 +79,9 @@ public class TestPersistentTtlNode extends BaseClassForTests
                         @Override
                         public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                         {
-                            if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_UPDATED )
+                            if ( (event.getData() != null) && "/test".equals(event.getData().getPath()) )
                             {
-                                if ( event.getData().getPath().equals("/test") )
-                                {
-                                    changes.release();
-                                }
+                                changes.release();
                             }
                         }
                     };


[05/20] curator git commit: initial test and doc updates

Posted by ra...@apache.org.
initial test and doc updates


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

Branch: refs/heads/CURATOR-397
Commit: 994778c31242942c43da741c5124d8c0f2320ea0
Parents: d029966
Author: randgalt <ra...@apache.org>
Authored: Fri Dec 9 14:39:35 2016 +0100
Committer: randgalt <ra...@apache.org>
Committed: Fri Dec 9 14:39:35 2016 +0100

----------------------------------------------------------------------
 .../org/apache/curator/utils/ThreadUtils.java   |   4 +-
 .../framework/recipes/nodes/PersistentNode.java |   2 +-
 .../recipes/nodes/PersistentTtlNode.java        | 190 +++++++++++++++++++
 .../src/site/confluence/index.confluence        |   3 +-
 .../persistent-ephemeral-node.confluence        |   2 +-
 .../confluence/persistent-ttl-node.confluence   |  39 ++++
 .../recipes/nodes/TestPersistentTtlNode.java    |  58 ++++++
 7 files changed, 294 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-client/src/main/java/org/apache/curator/utils/ThreadUtils.java
----------------------------------------------------------------------
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ThreadUtils.java b/curator-client/src/main/java/org/apache/curator/utils/ThreadUtils.java
index 74b4e40..bc93604 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ThreadUtils.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ThreadUtils.java
@@ -31,12 +31,14 @@ public class ThreadUtils
 {
     private static final Logger log = LoggerFactory.getLogger(ThreadUtils.class);
 
-    public static void checkInterrupted(Throwable e)
+    public static boolean checkInterrupted(Throwable e)
     {
         if ( e instanceof InterruptedException )
         {
             Thread.currentThread().interrupt();
+            return true;
         }
+        return false;
     }
 
     public static ExecutorService newSingleThreadExecutor(String processName)

http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
index eaa91b7..e956266 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
@@ -386,7 +386,7 @@ public class PersistentNode implements Closeable
         return this.data.get();
     }
 
-    private void deleteNode() throws Exception
+    protected void deleteNode() throws Exception
     {
         String localNodePath = nodePath.getAndSet(null);
         if ( localNodePath != null )

http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
new file mode 100644
index 0000000..3eda9e2
--- /dev/null
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
@@ -0,0 +1,190 @@
+package org.apache.curator.framework.recipes.nodes;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.utils.ThreadUtils;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Objects;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * <p>
+ *     Manages a {@link PersistentNode} that uses {@link CreateMode#CONTAINER}. Asynchronously
+ *     it creates or updates a child on the persistent node that is marked with a provided TTL.
+ * </p>
+ *
+ * <p>
+ *     The effect of this is to have a node that can be watched, etc. The child node serves as
+ *     a method of having the parent node deleted if the TTL expires. i.e. if the process
+ *     that is running the PersistentTtlNode crashes and the TTL elapses, first the child node
+ *     will be deleted due to the TTL expiration and then the parent node will be deleted as it's
+ *     a container node with no children.
+ * </p>
+ *
+ * <p>
+ *     PersistentTtlNode is useful when you need to create a TTL node but don't want to keep
+ *     it alive manually by periodically setting data - PersistentTtlNode does that for you. Further
+ *     the keep-alive is done in a way that does not generate watch triggers on the parent node.
+ * </p>
+ */
+public class PersistentTtlNode implements Closeable
+{
+    public static final String DEFAULT_CHILD_NODE_NAME = "touch";
+    public static final int DEFAULT_TOUCH_SCHEDULE_FACTOR = 2;
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final PersistentNode node;
+    private final CuratorFramework client;
+    private final long ttlMs;
+    private final int touchScheduleFactor;
+    private final ScheduledExecutorService executorService;
+    private final AtomicReference<Future<?>> futureRef = new AtomicReference<>();
+    private final String childPath;
+
+    /**
+     * @param client the client
+     * @param path path for the parent ZNode
+     * @param ttlMs max ttl for the node in milliseconds
+     * @param initData data for the node
+     */
+    public PersistentTtlNode(CuratorFramework client, String path, long ttlMs, byte[] initData)
+    {
+        this(client, Executors.newSingleThreadScheduledExecutor(ThreadUtils.newThreadFactory("PersistentTtlNode")), path, ttlMs, initData, DEFAULT_CHILD_NODE_NAME, DEFAULT_TOUCH_SCHEDULE_FACTOR);
+    }
+
+    /**
+     * @param client the client
+     * @param path path for the parent ZNode
+     * @param ttlMs max ttl for the node in milliseconds
+     * @param initData data for the node
+     * @param childNodeName name to use for the child node of the node created at <code>path</code>
+     * @param touchScheduleFactor how ofter to set/create the child node as a factor of the ttlMs. i.e.
+     *                            the child is touched every <code>(ttlMs / touchScheduleFactor)</code>
+     */
+    public PersistentTtlNode(CuratorFramework client, ScheduledExecutorService executorService, String path, long ttlMs, byte[] initData, String childNodeName, int touchScheduleFactor)
+    {
+        this.client = Objects.requireNonNull(client, "client cannot be null");
+        this.ttlMs = ttlMs;
+        this.touchScheduleFactor = touchScheduleFactor;
+        node = new PersistentNode(client, CreateMode.CONTAINER, false, path, initData)
+        {
+            @Override
+            protected void deleteNode()
+            {
+                // NOP
+            }
+        };
+        this.executorService = Objects.requireNonNull(executorService, "executorService cannot be null");
+        childPath = ZKPaths.makePath(Objects.requireNonNull(path, "path cannot be null"), childNodeName);
+    }
+
+    /**
+     * You must call start() to initiate the persistent ttl node
+     */
+    public void start()
+    {
+        node.start();
+
+        Runnable touchTask = new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                try
+                {
+                    try
+                    {
+                        client.setData().forPath(childPath);
+                    }
+                    catch ( KeeperException.NoNodeException e )
+                    {
+                        client.create().orSetData().withTtl(ttlMs).withMode(CreateMode.PERSISTENT_WITH_TTL).forPath(childPath);
+                    }
+                }
+                catch ( KeeperException.NoNodeException ignore )
+                {
+                    // ignore
+                }
+                catch ( Exception e )
+                {
+                    if ( !ThreadUtils.checkInterrupted(e) )
+                    {
+                        log.debug("Could not touch child node", e);
+                    }
+                }
+            }
+        };
+        Future<?> future = executorService.scheduleAtFixedRate(touchTask, ttlMs / touchScheduleFactor, ttlMs / touchScheduleFactor, TimeUnit.MILLISECONDS);
+        futureRef.set(future);
+    }
+
+    /**
+     * Block until the either initial node creation initiated by {@link #start()} succeeds or
+     * the timeout elapses.
+     *
+     * @param timeout the maximum time to wait
+     * @param unit    time unit
+     * @return if the node was created before timeout
+     * @throws InterruptedException if the thread is interrupted
+     */
+    public boolean waitForInitialCreate(long timeout, TimeUnit unit) throws InterruptedException
+    {
+        return node.waitForInitialCreate(timeout, unit);
+    }
+
+    /**
+     * Set data that node should set in ZK also writes the data to the node. NOTE: it
+     * is an error to call this method after {@link #start()} but before the initial create
+     * has completed. Use {@link #waitForInitialCreate(long, TimeUnit)} to ensure initial
+     * creation.
+     *
+     * @param data new data value
+     * @throws Exception errors
+     */
+    public void setData(byte[] data) throws Exception
+    {
+        node.setData(data);
+    }
+
+    /**
+     * Return the current value of our data
+     *
+     * @return our data
+     */
+    public byte[] getData()
+    {
+        return node.getData();
+    }
+
+    /**
+     * Call when you are done with the PersistentTtlNode. Note: the ZNode is <em>not</em> immediately
+     * deleted. However, if no other PersistentTtlNode with the same path is running the node will get deleted
+     * based on the ttl.
+     */
+    @Override
+    public void close()
+    {
+        Future<?> future = futureRef.getAndSet(null);
+        if ( future != null )
+        {
+            future.cancel(true);
+        }
+        try
+        {
+            node.close();
+        }
+        catch ( IOException e )
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-recipes/src/site/confluence/index.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/index.confluence b/curator-recipes/src/site/confluence/index.confluence
index 01dfc7e..4cf2cde 100644
--- a/curator-recipes/src/site/confluence/index.confluence
+++ b/curator-recipes/src/site/confluence/index.confluence
@@ -29,7 +29,8 @@ regarding "Curator Recipes Own Their ZNode/Paths".
 |[[Tree Cache|tree-cache.html]] \- A utility that attempts to keep all data from all children of a ZK path locally cached. This class will watch the ZK path, respond to update/create/delete events, pull down the data, etc. You can register a listener that will get notified when changes occur.|
 
 ||Nodes||
-|[[Persistent Ephemeral Node|persistent-ephemeral-node.html]] \- An ephemeral node that attempts to stay present in ZooKeeper, even through connection and session interruptions.|
+|[[Persistent Node|persistent-ephemeral-node.html]] \- A node that attempts to stay present in ZooKeeper, even through connection and session interruptions.|
+|[[Persistent TTL Node|persistent-ttl-node.html]] \- Useful when you need to create a TTL node but don't want to keep it alive manually by periodically setting data.|
 |[Group Member|group-member.html]] \- Group membership management. Adds this instance into a group and keeps a cache of members in the group.|
 
 ||Queues||

http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence b/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
index aeb9e10..5baca09 100644
--- a/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
+++ b/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
@@ -1,4 +1,4 @@
-h1. Persistent Ephemeral Node
+h1. Persistent Node
 
 h2. Description
 A persistent node is a node that attempts to stay present in ZooKeeper, even through connection and session interruptions.

http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/persistent-ttl-node.confluence b/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
new file mode 100644
index 0000000..acd2d33
--- /dev/null
+++ b/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
@@ -0,0 +1,39 @@
+h1. Persistent TTL Node
+
+h2. Description
+PersistentTtlNode is useful when you need to create a TTL node but don't want to keep it alive manually by periodically setting data \-
+PersistentTtlNode does that for you. Further the keep\-alive is done in a way that does not generate watch triggers on the parent node.
+
+h2. Participating Classes
+* PersistentNode
+* PersistentTtlNode
+
+h2. Usage
+h3. Creating a PersistentTtlNode
+{code}
+public PersistentTtlNode(CuratorFramework client,
+                               String path,
+                               long ttlMs,
+                               byte[] initData)
+Parameters:
+client - client instance
+path path for the parent ZNode
+ttlMs max ttl for the node in milliseconds
+initData - initData for the node
+{code}
+
+h3. General Usage
+PersistentTtlNode must be started:
+{code}
+node.start();
+{code}
+
+When you are through with the PersistentTtlNode instance, you should call close:
+{code}
+node.close();
+{code}
+
+NOTE: this will NOT delete the node immediately. The node will get deleted based on the ttl.
+
+h2. Error Handling
+PersistentTtlNode instances internally handle all error states recreating the node as necessary.

http://git-wip-us.apache.org/repos/asf/curator/blob/994778c3/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
new file mode 100644
index 0000000..9ae8df9
--- /dev/null
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -0,0 +1,58 @@
+package org.apache.curator.framework.recipes.nodes;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.Timing;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.util.concurrent.TimeUnit;
+
+public class TestPersistentTtlNode extends BaseClassForTests
+{
+    private final Timing timing = new Timing();
+
+    @BeforeMethod
+    @Override
+    public void setup() throws Exception
+    {
+        System.setProperty("znode.container.checkIntervalMs", "1");
+        super.setup();
+    }
+
+    @AfterMethod
+    @Override
+    public void teardown() throws Exception
+    {
+        System.clearProperty("znode.container.checkIntervalMs");
+        super.teardown();
+    }
+
+    @Test
+    public void testBasic() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)))
+        {
+            client.start();
+
+            try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 10, new byte[0]))
+            {
+                node.start();
+                node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
+
+                for ( int i = 0; i < 10; ++i )
+                {
+                    Thread.sleep(10);
+                    Assert.assertNotNull(client.checkExists().forPath("/test"));
+                }
+            }
+
+            timing.sleepABit();
+
+            Assert.assertNull(client.checkExists().forPath("/test"));
+        }
+    }
+}


[20/20] curator git commit: Major rework of caching. Having the wrapped caches adds little value. Focus on the integrated caching in the modeled client instance

Posted by ra...@apache.org.
Major rework of caching. Having the wrapped caches adds little value. Focus on the integrated caching in the modeled client instance


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

Branch: refs/heads/CURATOR-397
Commit: 2cbbf999294181dbefeccebabd9c6de867142e2c
Parents: e40ed18
Author: randgalt <ra...@apache.org>
Authored: Mon May 1 22:56:36 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon May 1 22:56:36 2017 -0500

----------------------------------------------------------------------
 .../main/java/modeled/ModeledCacheExamples.java |  57 ----
 .../modeled/CachedModeledCuratorFramework.java  |  43 ---
 .../async/modeled/ModeledCuratorFramework.java  |  14 +-
 .../apache/curator/x/async/modeled/ZPath.java   |   9 +
 .../cached/CachedModeledCuratorFramework.java   |  48 ++++
 .../x/async/modeled/cached/ModeledCache.java    |  53 ++++
 .../modeled/cached/ModeledCacheEventType.java   |  42 +++
 .../modeled/cached/ModeledCacheListener.java    |  58 ++++
 .../async/modeled/cached/ModeledCachedNode.java |  49 ++++
 .../CachedModeledCuratorFrameworkImpl.java      | 140 +++-------
 .../x/async/modeled/details/ModelStage.java     |   6 +
 .../async/modeled/details/ModeledCacheImpl.java | 173 ++++++++++++
 .../details/ModeledCuratorFrameworkImpl.java    |  34 +--
 .../x/async/modeled/details/ZPathImpl.java      |  13 +-
 .../details/recipes/ModeledCachedNodeImpl.java  | 107 --------
 .../details/recipes/ModeledNodeCacheImpl.java   | 232 ----------------
 .../recipes/ModeledPathChildrenCacheImpl.java   | 266 -------------------
 .../details/recipes/ModeledTreeCacheImpl.java   | 179 -------------
 .../x/async/modeled/recipes/ModeledCache.java   |  51 ----
 .../modeled/recipes/ModeledCacheEvent.java      |  39 ---
 .../modeled/recipes/ModeledCacheEventType.java  |  57 ----
 .../modeled/recipes/ModeledCacheListener.java   |  84 ------
 .../modeled/recipes/ModeledCachedNode.java      |  49 ----
 .../async/modeled/recipes/ModeledNodeCache.java |  91 -------
 .../recipes/ModeledPathChildrenCache.java       | 114 --------
 .../async/modeled/recipes/ModeledTreeCache.java |  70 -----
 .../TestCachedModeledCuratorFramework.java      |   4 +-
 .../details/recipes/TestEventTypeMappings.java  |  44 ---
 .../modeled/recipes/TestModeledCaches.java      | 193 --------------
 29 files changed, 497 insertions(+), 1822 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-examples/src/main/java/modeled/ModeledCacheExamples.java
----------------------------------------------------------------------
diff --git a/curator-examples/src/main/java/modeled/ModeledCacheExamples.java b/curator-examples/src/main/java/modeled/ModeledCacheExamples.java
deleted file mode 100644
index 3157f7e..0000000
--- a/curator-examples/src/main/java/modeled/ModeledCacheExamples.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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 modeled;
-
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.x.async.modeled.JacksonModelSerializer;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEventType;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheListener;
-import org.apache.curator.x.async.modeled.recipes.ModeledTreeCache;
-import java.util.function.Consumer;
-
-public class ModeledCacheExamples
-{
-    public static ModeledTreeCache<PersonModel> wrap(TreeCache cache)
-    {
-        JacksonModelSerializer<PersonModel> serializer = JacksonModelSerializer.build(PersonModel.class);
-
-        // wrap a TreeCache instance so that it can be used "modeled".
-        return ModeledTreeCache.wrap(cache, serializer);
-    }
-
-    public static void watchForChanges(TreeCache cache, Consumer<PersonModel> deletePersonReceiver, Consumer<PersonModel> updatedPersonReceiver)
-    {
-        ModeledTreeCache<PersonModel> modeledCache = wrap(cache);
-        ModeledCacheListener<PersonModel> listener = event -> {
-            PersonModel person = event.getNode().getModel();
-            if ( event.getType() == ModeledCacheEventType.NODE_REMOVED )
-            {
-                deletePersonReceiver.accept(person);
-            }
-            else
-            {
-                updatedPersonReceiver.accept(person);
-            }
-        };
-
-        // take a standard listener and filter so that only events that have a valid model instance are sent to the listener
-        ModeledCacheListener<PersonModel> filteredListener = ModeledCacheListener.filtered(listener, ModeledCacheListener.hasModelFilter());
-        modeledCache.getListenable().addListener(filteredListener);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/CachedModeledCuratorFramework.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/CachedModeledCuratorFramework.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/CachedModeledCuratorFramework.java
deleted file mode 100644
index b4c6fef..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/CachedModeledCuratorFramework.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.x.async.modeled;
-
-import org.apache.curator.x.async.modeled.recipes.ModeledCache;
-import java.io.Closeable;
-
-public interface CachedModeledCuratorFramework<T> extends ModeledCuratorFramework<T>, Closeable
-{
-    /**
-     * Return the cache instance
-     *
-     * @return cache
-     */
-    ModeledCache<T> getCache();
-
-    /**
-     * Start the internally created via {@link #cached()}
-     */
-    void start();
-
-    /**
-     * Close the internally created via {@link #cached()}
-     */
-    @Override
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledCuratorFramework.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledCuratorFramework.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledCuratorFramework.java
index ac4fd8d..31b630d 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledCuratorFramework.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledCuratorFramework.java
@@ -22,7 +22,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.transaction.CuratorOp;
 import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
 import org.apache.curator.x.async.AsyncStage;
-import org.apache.curator.x.async.modeled.recipes.ModeledCache;
+import org.apache.curator.x.async.modeled.cached.CachedModeledCuratorFramework;
 import org.apache.zookeeper.data.Stat;
 import java.util.List;
 import java.util.Map;
@@ -54,19 +54,9 @@ public interface ModeledCuratorFramework<T>
     }
 
     /**
-     * Use the given cache as a front for this modeled instance. All read APIs check the cache
-     * first and, if available, return the values from the cache.
-     * the cache
-     *
-     * @param cache cache to use
-     * @return wrapped instance
-     */
-    CachedModeledCuratorFramework<T> cached(ModeledCache<T> cache);
-
-    /**
      * Use an internally created cache as a front for this modeled instance. All read APIs check the cache
      * first and, if available, return the values from the cache. Note: you must call
-     * {@link org.apache.curator.x.async.modeled.CachedModeledCuratorFramework#start()} and
+     * {@link org.apache.curator.x.async.modeled.cached.CachedModeledCuratorFramework#start()} and
      * {@link CachedModeledCuratorFramework#close()} to start/stop
      *
      * @return wrapped instance

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
index 78cd40f..1b10a40 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ZPath.java
@@ -176,6 +176,15 @@ public interface ZPath
     boolean isRoot();
 
     /**
+     * Return true if this path starts with the given path. i.e.
+     * <code>ZPath.from("/one/two/three").startsWith(ZPath.from("/one/two"))</code> returns true
+     *
+     * @param path base path
+     * @return true/false
+     */
+    boolean startsWith(ZPath path);
+
+    /**
      * The string full path that this ZPath represents
      *
      * @return full path

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/CachedModeledCuratorFramework.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/CachedModeledCuratorFramework.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/CachedModeledCuratorFramework.java
new file mode 100644
index 0000000..8718e60
--- /dev/null
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/CachedModeledCuratorFramework.java
@@ -0,0 +1,48 @@
+/**
+ * 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.x.async.modeled.cached;
+
+import org.apache.curator.x.async.modeled.ModeledCuratorFramework;
+import java.io.Closeable;
+
+public interface CachedModeledCuratorFramework<T> extends ModeledCuratorFramework<T>, Closeable
+{
+    /**
+     * Return the cache instance
+     *
+     * @return cache
+     */
+    ModeledCache<T> getCache();
+
+    /**
+     * Start the internally created via {@link #cached()}
+     */
+    void start();
+
+    /**
+     * Close the internally created via {@link #cached()}
+     */
+    @Override
+    void close();
+
+    /**
+     * {@inheritDoc}
+     */
+    CachedModeledCuratorFramework<T> at(String child);
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCache.java
new file mode 100644
index 0000000..3536a65
--- /dev/null
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCache.java
@@ -0,0 +1,53 @@
+/**
+ * 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.x.async.modeled.cached;
+
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.x.async.modeled.ZPath;
+import java.util.Map;
+import java.util.Optional;
+
+public interface ModeledCache<T>
+{
+    /**
+     * Return the modeled current data for the given path. There are no guarantees of accuracy. This is
+     * merely the most recent view of the data. If there is no node at the given path,
+     * {@link java.util.Optional#empty()} is returned.
+     *
+     * @param path path to the node to check
+     * @return data if the node is alive, or null
+     */
+    Optional<ModeledCachedNode<T>> getCurrentData(ZPath path);
+
+    /**
+     * Return the modeled current set of children at the given path, mapped by child name. There are no
+     * guarantees of accuracy; this is merely the most recent view of the data.
+     *
+     * @param path path to the node to check
+     * @return a possibly-empty list of children if the node is alive, or null
+     */
+    Map<ZPath, ModeledCachedNode<T>> getCurrentChildren(ZPath path);
+
+    /**
+     * Return the listener container so that you can add/remove listeners
+     *
+     * @return listener container
+     */
+    Listenable<ModeledCacheListener<T>> getListenable();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheEventType.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheEventType.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheEventType.java
new file mode 100644
index 0000000..e7754ea
--- /dev/null
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheEventType.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.x.async.modeled.cached;
+
+public enum ModeledCacheEventType
+{
+    /**
+     * A child was added to the path
+     */
+    NODE_ADDED,
+
+    /**
+     * A child's data was changed
+     */
+    NODE_UPDATED,
+
+    /**
+     * A child was removed from the path
+     */
+    NODE_REMOVED,
+
+    /**
+     * Signals that the initial cache has been populated.
+     */
+    INITIALIZED
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheListener.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheListener.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheListener.java
new file mode 100644
index 0000000..9ddef87
--- /dev/null
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCacheListener.java
@@ -0,0 +1,58 @@
+package org.apache.curator.x.async.modeled.cached;
+
+import org.apache.curator.x.async.modeled.ZPath;
+import org.apache.zookeeper.data.Stat;
+
+@FunctionalInterface
+public interface ModeledCacheListener<T>
+{
+    /**
+     * The given path was added, updated or removed
+     *
+     * @param type action type
+     * @param path the path
+     * @param stat the node's stat (previous stat for removal)
+     * @param model the node's model (previous model for removal)
+     */
+    void accept(ModeledCacheEventType type, ZPath path, Stat stat, T model);
+
+    /**
+     * The cache has finished initializing
+     */
+    default void initialized()
+    {
+        // NOP
+    }
+
+    /**
+     * Returns a version of this listener that only begins calling
+     * {@link #accept(ModeledCacheEventType, org.apache.curator.x.async.modeled.ZPath, org.apache.zookeeper.data.Stat, Object)}
+     * once {@link #initialized()} has been called. i.e. changes that occur as the cache is initializing are not sent
+     * to the listener
+     *
+     * @return wrapped listener
+     */
+    default ModeledCacheListener<T> postInitializedOnly()
+    {
+        return new ModeledCacheListener<T>()
+        {
+            private volatile boolean isInitialized = false;
+
+            @Override
+            public void accept(ModeledCacheEventType type, ZPath path, Stat stat, T model)
+            {
+                if ( isInitialized )
+                {
+                    ModeledCacheListener.this.accept(type, path, stat, model);
+                }
+            }
+
+            @Override
+            public void initialized()
+            {
+                isInitialized = true;
+                ModeledCacheListener.this.initialized();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCachedNode.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCachedNode.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCachedNode.java
new file mode 100644
index 0000000..3a8e742
--- /dev/null
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/cached/ModeledCachedNode.java
@@ -0,0 +1,49 @@
+/**
+ * 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.x.async.modeled.cached;
+
+import org.apache.curator.x.async.modeled.ZPath;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Abstracts a cached node
+ */
+public interface ModeledCachedNode<T>
+{
+    /**
+     * The path of the node
+     *
+     * @return path
+     */
+    ZPath getPath();
+
+    /**
+     * The node's last known stat if available
+     *
+     * @return stat
+     */
+    Stat getStat();
+
+    /**
+     * The node's current model
+     *
+     * @return model
+     */
+    T getModel();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledCuratorFrameworkImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledCuratorFrameworkImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledCuratorFrameworkImpl.java
index 6192f3a..3318403 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledCuratorFrameworkImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/CachedModeledCuratorFrameworkImpl.java
@@ -18,44 +18,43 @@
  */
 package org.apache.curator.x.async.modeled.details;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.transaction.CuratorOp;
 import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
 import org.apache.curator.x.async.AsyncStage;
-import org.apache.curator.x.async.modeled.CachedModeledCuratorFramework;
+import org.apache.curator.x.async.api.CreateOption;
 import org.apache.curator.x.async.modeled.CuratorModelSpec;
 import org.apache.curator.x.async.modeled.ModeledCuratorFramework;
 import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.recipes.ModeledCache;
-import org.apache.curator.x.async.modeled.recipes.ModeledCachedNode;
+import org.apache.curator.x.async.modeled.cached.CachedModeledCuratorFramework;
+import org.apache.curator.x.async.modeled.cached.ModeledCache;
+import org.apache.curator.x.async.modeled.cached.ModeledCachedNode;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.DataTree;
 import java.util.AbstractMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Optional;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 class CachedModeledCuratorFrameworkImpl<T> implements CachedModeledCuratorFramework<T>
 {
     private final ModeledCuratorFramework<T> client;
-    private final ModeledCache<T> cache;
-    private final ZPath path;
+    private final ModeledCacheImpl<T> cache;
 
-    CachedModeledCuratorFrameworkImpl(ModeledCuratorFramework<T> client, ModeledCache<T> cache, ZPath path)
+    CachedModeledCuratorFrameworkImpl(ModeledCuratorFramework<T> client)
     {
-        this.client = Objects.requireNonNull(client, "client cannot be null");
-        this.cache = Objects.requireNonNull(cache, "cache cannot be null");
-        this.path = Objects.requireNonNull(path, "path cannot be null");
+        this(client, new ModeledCacheImpl<>(client.unwrap(), client.modelSpec().path(), client.modelSpec().serializer(), client.modelSpec().createOptions().contains(CreateOption.compress)));
     }
 
-    @VisibleForTesting
-    volatile AtomicInteger debugCachedReadCount = null;
+    private CachedModeledCuratorFrameworkImpl(ModeledCuratorFramework<T> client, ModeledCacheImpl<T> cache)
+    {
+        this.client = client;
+        this.cache = cache;
+    }
 
     @Override
     public ModeledCache<T> getCache()
@@ -66,55 +65,49 @@ class CachedModeledCuratorFrameworkImpl<T> implements CachedModeledCuratorFramew
     @Override
     public void start()
     {
-        throw new UnsupportedOperationException();
+        cache.start();
     }
 
     @Override
     public void close()
     {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public CuratorModelSpec<T> modelSpec()
-    {
-        return client.modelSpec();
+        cache.close();
     }
 
     @Override
-    public CachedModeledCuratorFramework<T> cached(ModeledCache<T> cache)
+    public CachedModeledCuratorFramework<T> cached()
     {
-        throw new UnsupportedOperationException();
+        return this;
     }
 
     @Override
-    public CachedModeledCuratorFramework<T> cached()
+    public CuratorFramework unwrap()
     {
-        throw new UnsupportedOperationException();
+        return client.unwrap();
     }
 
     @Override
-    public CuratorFramework unwrap()
+    public CuratorModelSpec<T> modelSpec()
     {
-        return client.unwrap();
+        return client.modelSpec();
     }
 
     @Override
-    public ModeledCuratorFramework<T> at(String child)
+    public CachedModeledCuratorFramework<T> at(String child)
     {
-        return new CachedModeledCuratorFrameworkImpl<>(client.at(child), cache, path.at(child));
+        return new CachedModeledCuratorFrameworkImpl<>(client.at(child), cache);
     }
 
     @Override
     public AsyncStage<String> set(T model)
     {
-        return client.set(model);
+        return client.set(model);   // TODO - update cache?
     }
 
     @Override
     public AsyncStage<String> set(T model, Stat storingStatIn)
     {
-        return client.set(model, storingStatIn);
+        return client.set(model, storingStatIn);   // TODO - update cache?
     }
 
     @Override
@@ -126,25 +119,15 @@ class CachedModeledCuratorFrameworkImpl<T> implements CachedModeledCuratorFramew
     @Override
     public AsyncStage<T> read(Stat storingStatIn)
     {
+        ZPath path = client.modelSpec().path();
         Optional<ModeledCachedNode<T>> data = cache.getCurrentData(path);
-        if ( data.isPresent() )
-        {
-            ModeledCachedNode<T> localData = data.get();
-            T model = localData.getModel();
-            if ( model != null )
+        return data.map(node -> {
+            if ( storingStatIn != null )
             {
-                if ( (storingStatIn != null) && (localData.getStat() != null) )
-                {
-                    DataTree.copyStat(localData.getStat(), storingStatIn);
-                }
-                if ( debugCachedReadCount != null )
-                {
-                    debugCachedReadCount.incrementAndGet();
-                }
-                return new ModelStage<>(model);
+                DataTree.copyStat(node.getStat(), storingStatIn);
             }
-        }
-        return (storingStatIn != null) ? client.read(storingStatIn) : client.read();
+            return new ModelStage<>(node.getModel());
+        }).orElseGet(() -> new ModelStage<>(new KeeperException.NoNodeException(path.fullPath())));
     }
 
     @Override
@@ -174,67 +157,26 @@ class CachedModeledCuratorFrameworkImpl<T> implements CachedModeledCuratorFramew
     @Override
     public AsyncStage<Stat> checkExists()
     {
+        ZPath path = client.modelSpec().path();
         Optional<ModeledCachedNode<T>> data = cache.getCurrentData(path);
-        return data.map(node -> {
-            AsyncStage<Stat> stage = new ModelStage<>(node.getStat());
-            if ( debugCachedReadCount != null )
-            {
-                debugCachedReadCount.incrementAndGet();
-            }
-            return stage;
-        }).orElseGet(client::checkExists);
+        return data.map(node -> new ModelStage<>(node.getStat())).orElseGet(() -> new ModelStage<>((Stat)null));
     }
 
     @Override
     public AsyncStage<List<ZPath>> getChildren()
     {
-        Map<ZPath, ModeledCachedNode<T>> currentChildren = cache.getCurrentChildren(path);
-        if ( currentChildren != cache.noChildrenValue() )
-        {
-            if ( debugCachedReadCount != null )
-            {
-                debugCachedReadCount.incrementAndGet();
-            }
-            return new ModelStage<>(Lists.newArrayList(currentChildren.keySet()));
-        }
-        return client.getChildren();
+        Set<ZPath> paths = cache.getCurrentChildren(client.modelSpec().path()).keySet();
+        return new ModelStage<>(Lists.newArrayList(paths));
     }
 
     @Override
     public AsyncStage<Map<ZPath, AsyncStage<T>>> readChildren()
     {
-        Map<ZPath, ModeledCachedNode<T>> currentChildren = cache.getCurrentChildren(path);
-        if ( currentChildren != cache.noChildrenValue() )
-        {
-            if ( debugCachedReadCount != null )
-            {
-                debugCachedReadCount.incrementAndGet();
-            }
-            Map<ZPath, AsyncStage<T>> children = currentChildren.entrySet()
-                .stream()
-                .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().getModel()))
-                .filter(e -> e.getValue() != null)
-                .collect(Collectors.toMap(Map.Entry::getKey, e -> new ModelStage<>(e.getValue())));
-            return new ModelStage<>(children);
-        }
-
-        ModelStage<Map<ZPath, AsyncStage<T>>> modelStage = new ModelStage<>();
-        client.getChildren().whenComplete((children, e) -> {
-            if ( e != null )
-            {
-                modelStage.completeExceptionally(e);
-            }
-            else
-            {
-                if ( debugCachedReadCount != null )
-                {
-                    debugCachedReadCount.incrementAndGet();
-                }
-                Map<ZPath, AsyncStage<T>> map = children.stream().collect(Collectors.toMap(Function.identity(), path1 -> at(path1.nodeName()).read()));
-                modelStage.complete(map);
-            }
-        });
-        return modelStage;
+        Map<ZPath, AsyncStage<T>> map = cache.getCurrentChildren(client.modelSpec().path()).entrySet()
+            .stream()
+            .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new ModelStage<>(entry.getValue().getModel())))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+        return new ModelStage<>(map);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelStage.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelStage.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelStage.java
index c28b133..77caed1 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelStage.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModelStage.java
@@ -43,6 +43,12 @@ class ModelStage<T> extends CompletableFuture<T> implements AsyncStage<T>
         complete(value);
     }
 
+    ModelStage(Exception e)
+    {
+        event = null;
+        completeExceptionally(e);
+    }
+
     @Override
     public CompletionStage<WatchedEvent> event()
     {

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCacheImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCacheImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCacheImpl.java
new file mode 100644
index 0000000..c1ab8cd
--- /dev/null
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCacheImpl.java
@@ -0,0 +1,173 @@
+package org.apache.curator.x.async.modeled.details;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.framework.listen.ListenerContainer;
+import org.apache.curator.framework.recipes.cache.TreeCache;
+import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
+import org.apache.curator.framework.recipes.cache.TreeCacheListener;
+import org.apache.curator.x.async.modeled.ModelSerializer;
+import org.apache.curator.x.async.modeled.ZPath;
+import org.apache.curator.x.async.modeled.cached.ModeledCache;
+import org.apache.curator.x.async.modeled.cached.ModeledCacheEventType;
+import org.apache.curator.x.async.modeled.cached.ModeledCacheListener;
+import org.apache.curator.x.async.modeled.cached.ModeledCachedNode;
+import org.apache.zookeeper.data.Stat;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+class ModeledCacheImpl<T> implements TreeCacheListener, ModeledCache<T>
+{
+    private final TreeCache cache;
+    private final Map<ZPath, Entry<T>> entries = new ConcurrentHashMap<>();
+    private final ModelSerializer<T> serializer;
+    private final ListenerContainer<ModeledCacheListener<T>> listenerContainer = new ListenerContainer<>();
+
+    private static final class Entry<T>
+    {
+        final Stat stat;
+        final T model;
+
+        Entry(Stat stat, T model)
+        {
+            this.stat = stat;
+            this.model = model;
+        }
+    }
+
+    ModeledCacheImpl(CuratorFramework client, ZPath path, ModelSerializer<T> serializer, boolean compressed)
+    {
+        this.serializer = serializer;
+        cache = TreeCache.newBuilder(client, path.fullPath())
+            .setCacheData(false)
+            .setDataIsCompressed(compressed)
+            .build();
+    }
+
+    public void start()
+    {
+        try
+        {
+            cache.start();
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void close()
+    {
+        cache.close();
+        entries.clear();
+    }
+
+    @Override
+    public Optional<ModeledCachedNode<T>> getCurrentData(ZPath path)
+    {
+        Entry<T> entry = entries.remove(path);
+        if ( entry != null )
+        {
+            return Optional.of(new InternalCachedNode<>(path, entry));
+        }
+        return Optional.empty();
+    }
+
+    @Override
+    public Map<ZPath, ModeledCachedNode<T>> getCurrentChildren(ZPath path)
+    {
+        return entries.entrySet()
+            .stream()
+            .filter(entry -> entry.getKey().startsWith(path))
+            .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), new InternalCachedNode<>(entry.getKey(), entry.getValue())))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    @Override
+    public Listenable<ModeledCacheListener<T>> getListenable()
+    {
+        return listenerContainer;
+    }
+
+    @Override
+    public void childEvent(CuratorFramework client, TreeCacheEvent event) throws Exception
+    {
+        switch ( event.getType() )
+        {
+        case NODE_ADDED:
+        case NODE_UPDATED:
+        {
+            ZPath path = ZPath.from(event.toString());
+            T model = serializer.deserialize(event.getData().getData());
+            entries.put(path, new Entry<>(event.getData().getStat(), model));
+            ModeledCacheEventType type = (event.getType() == TreeCacheEvent.Type.NODE_ADDED) ? ModeledCacheEventType.NODE_ADDED : ModeledCacheEventType.NODE_UPDATED;
+            accept(type, path, event.getData().getStat(), model);
+            break;
+        }
+
+        case NODE_REMOVED:
+        {
+            ZPath path = ZPath.from(event.toString());
+            Entry<T> entry = entries.remove(path);
+            T model = (entry != null) ? entry.model : serializer.deserialize(event.getData().getData());
+            Stat stat = (entry != null) ? entry.stat : event.getData().getStat();
+            accept(ModeledCacheEventType.NODE_REMOVED, path, stat, model);
+            break;
+        }
+
+        case INITIALIZED:
+        {
+            listenerContainer.forEach(l -> {
+                l.initialized();
+                return null;
+            });
+            break;
+        }
+
+        default:
+            // ignore
+            break;
+        }
+    }
+
+    private void accept(ModeledCacheEventType type, ZPath path, Stat stat, T model)
+    {
+        listenerContainer.forEach(l -> {
+            l.accept(type, path, stat, model);
+            return null;
+        });
+    }
+
+    private static class InternalCachedNode<U> implements ModeledCachedNode<U>
+    {
+        private final ZPath path;
+        private final Entry<U> entry;
+
+        private InternalCachedNode(ZPath path, Entry<U> entry)
+        {
+            this.path = path;
+            this.entry = entry;
+        }
+
+        @Override
+        public ZPath getPath()
+        {
+            return path;
+        }
+
+        @Override
+        public Stat getStat()
+        {
+            return entry.stat;
+        }
+
+        @Override
+        public U getModel()
+        {
+            return entry.model;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCuratorFrameworkImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCuratorFrameworkImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCuratorFrameworkImpl.java
index 6ed0649..aba87f3 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCuratorFrameworkImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ModeledCuratorFrameworkImpl.java
@@ -23,7 +23,6 @@ import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.framework.api.transaction.CuratorOp;
 import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
-import org.apache.curator.framework.recipes.cache.TreeCache;
 import org.apache.curator.x.async.AsyncCuratorFramework;
 import org.apache.curator.x.async.AsyncStage;
 import org.apache.curator.x.async.WatchMode;
@@ -33,12 +32,10 @@ import org.apache.curator.x.async.api.AsyncPathable;
 import org.apache.curator.x.async.api.AsyncTransactionSetDataBuilder;
 import org.apache.curator.x.async.api.CreateOption;
 import org.apache.curator.x.async.api.WatchableAsyncCuratorFramework;
-import org.apache.curator.x.async.modeled.CachedModeledCuratorFramework;
 import org.apache.curator.x.async.modeled.CuratorModelSpec;
 import org.apache.curator.x.async.modeled.ModeledCuratorFramework;
 import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.recipes.ModeledCache;
-import org.apache.curator.x.async.modeled.recipes.ModeledTreeCache;
+import org.apache.curator.x.async.modeled.cached.CachedModeledCuratorFramework;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
@@ -98,36 +95,9 @@ public class ModeledCuratorFrameworkImpl<T> implements ModeledCuratorFramework<T
     }
 
     @Override
-    public CachedModeledCuratorFramework<T> cached(ModeledCache<T> cache)
-    {
-        return new CachedModeledCuratorFrameworkImpl<>(this, cache, modelSpec.path());
-    }
-
-    @Override
     public CachedModeledCuratorFramework<T> cached()
     {
-        TreeCache.Builder builder = TreeCache.newBuilder(client.unwrap(), modelSpec.path().fullPath());
-        builder = builder.setCacheData(true);
-        if ( modelSpec.createOptions().contains(CreateOption.compress) )
-        {
-            builder = builder.setDataIsCompressed(true);
-        }
-        TreeCache cache = builder.build();
-        ModeledTreeCache<T> wrapped = ModeledTreeCache.wrap(cache, modelSpec.serializer());
-        return new CachedModeledCuratorFrameworkImpl<T>(this, wrapped, modelSpec.path())
-        {
-            @Override
-            public void start()
-            {
-                wrapped.start();
-            }
-
-            @Override
-            public void close()
-            {
-                wrapped.close();
-            }
-        };
+        return new CachedModeledCuratorFrameworkImpl<>(this);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
index 8072785..36c9d1a 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/ZPathImpl.java
@@ -73,7 +73,7 @@ public class ZPathImpl implements ZPath
     {
         names = Objects.requireNonNull(names, "names cannot be null");
         names.forEach(ZPathImpl::validate);
-        ImmutableList.Builder<String> builder = ImmutableList.<String>builder();
+        ImmutableList.Builder<String> builder = ImmutableList.builder();
         if ( base != null )
         {
             if ( base instanceof ZPathImpl )
@@ -113,6 +113,17 @@ public class ZPathImpl implements ZPath
     }
 
     @Override
+    public boolean startsWith(ZPath path)
+    {
+        if ( path instanceof ZPathImpl )
+        {
+            ZPathImpl rhs = (ZPathImpl)path;
+            return (nodes.size() >= rhs.nodes.size()) && nodes.subList(0, rhs.nodes.size()).equals(rhs);
+        }
+        return false;
+    }
+
+    @Override
     public Pattern toSchemaPathPattern()
     {
         return Pattern.compile(fullPath() + ZKPaths.PATH_SEPARATOR + ".*");

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledCachedNodeImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledCachedNodeImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledCachedNodeImpl.java
deleted file mode 100644
index e66fd8a..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledCachedNodeImpl.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/**
- * 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.x.async.modeled.details.recipes;
-
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.recipes.ModeledCachedNode;
-import org.apache.zookeeper.data.Stat;
-import java.util.Objects;
-
-public class ModeledCachedNodeImpl<T> implements ModeledCachedNode<T>
-{
-    private final ZPath path;
-    private final Stat stat;
-    private final T data;
-
-    public ModeledCachedNodeImpl(ZPath path)
-    {
-        this(path, null, new Stat());
-    }
-
-    public ModeledCachedNodeImpl(ZPath path, T data)
-    {
-        this(path, data, new Stat());
-    }
-
-    public ModeledCachedNodeImpl(ZPath path, T data, Stat stat)
-    {
-        this.path = Objects.requireNonNull(path, "path cannot be null");
-        this.stat = Objects.requireNonNull(stat, "stat cannot be null");
-        this.data = data;
-    }
-
-    @Override
-    public ZPath getPath()
-    {
-        return path;
-    }
-
-    @Override
-    public Stat getStat()
-    {
-        return stat;
-    }
-
-    @Override
-    public T getModel()
-    {
-        return data;
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if ( this == o )
-        {
-            return true;
-        }
-        if ( o == null || getClass() != o.getClass() )
-        {
-            return false;
-        }
-
-        ModeledCachedNodeImpl<?> that = (ModeledCachedNodeImpl<?>)o;
-
-        if ( !path.equals(that.path) )
-        {
-            return false;
-        }
-        //noinspection SimplifiableIfStatement
-        if ( !stat.equals(that.stat) )
-        {
-            return false;
-        }
-        return data != null ? data.equals(that.data) : that.data == null;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = path.hashCode();
-        result = 31 * result + stat.hashCode();
-        result = 31 * result + (data != null ? data.hashCode() : 0);
-        return result;
-    }
-
-    @Override
-    public String toString()
-    {
-        return "ModeledCachedNode{" + "stat=" + stat + ", data=" + data + '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledNodeCacheImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledNodeCacheImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledNodeCacheImpl.java
deleted file mode 100644
index 5b89faf..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledNodeCacheImpl.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/**
- * 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.x.async.modeled.details.recipes;
-
-import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.ChildData;
-import org.apache.curator.framework.recipes.cache.NodeCache;
-import org.apache.curator.framework.recipes.cache.NodeCacheListener;
-import org.apache.curator.framework.state.ConnectionState;
-import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.curator.utils.CloseableUtils;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEvent;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEventType;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheListener;
-import org.apache.curator.x.async.modeled.recipes.ModeledCachedNode;
-import org.apache.curator.x.async.modeled.recipes.ModeledNodeCache;
-import org.apache.zookeeper.data.Stat;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-
-public class ModeledNodeCacheImpl<T> implements ModeledNodeCache<T>, ConnectionStateListener
-{
-    private final NodeCache cache;
-    private final ModelSerializer<T> serializer;
-    private final ZPath path;
-    private final Map<ModeledCacheListener<T>, NodeCacheListener> listenerMap = new ConcurrentHashMap<>();
-
-    public ModeledNodeCacheImpl(NodeCache cache, ModelSerializer<T> serializer)
-    {
-        this.cache = Objects.requireNonNull(cache, "cache cannot be null");
-        this.serializer = Objects.requireNonNull(serializer, "serializer cannot be null");
-        path = ZPath.parse(cache.getPath());
-    }
-
-    @Override
-    public void stateChanged(CuratorFramework client, ConnectionState newState)
-    {
-        ModeledCacheEventType mappedType;
-        switch ( newState )
-        {
-            default:
-            {
-                mappedType = null;
-                break;
-            }
-
-            case RECONNECTED:
-            case CONNECTED:
-            {
-                mappedType = ModeledCacheEventType.CONNECTION_RECONNECTED;
-                break;
-            }
-
-            case SUSPENDED:
-            {
-                mappedType = ModeledCacheEventType.CONNECTION_SUSPENDED;
-                break;
-            }
-
-            case LOST:
-            {
-                mappedType = ModeledCacheEventType.CONNECTION_LOST;
-                break;
-            }
-        }
-
-        if ( mappedType != null )
-        {
-            ModeledCacheEvent<T> event = new ModeledCacheEvent<T>()
-            {
-                @Override
-                public ModeledCacheEventType getType()
-                {
-                    return mappedType;
-                }
-
-                @Override
-                public ModeledCachedNode<T> getNode()
-                {
-                    return null;
-                }
-            };
-            listenerMap.keySet().forEach(l -> l.event(null));
-        }
-    }
-
-    @Override
-    public NodeCache unwrap()
-    {
-        return cache;
-    }
-
-    @Override
-    public void start()
-    {
-        try
-        {
-            cache.start();
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("Could not start", e);
-        }
-        cache.getClient().getConnectionStateListenable().addListener(this);
-    }
-
-    @Override
-    public void start(boolean buildInitial)
-    {
-        cache.getClient().getConnectionStateListenable().removeListener(this);
-        try
-        {
-            cache.start(buildInitial);
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("Could not start", e);
-        }
-    }
-
-    @Override
-    public void rebuild()
-    {
-        try
-        {
-            cache.rebuild();
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("Could not rebuild", e);
-        }
-    }
-
-    @Override
-    public Listenable<ModeledCacheListener<T>> getListenable()
-    {
-        return new Listenable<ModeledCacheListener<T>>()
-        {
-            @Override
-            public void addListener(ModeledCacheListener<T> listener)
-            {
-                addListener(listener, MoreExecutors.sameThreadExecutor());
-            }
-
-            @Override
-            public void addListener(ModeledCacheListener<T> listener, Executor executor)
-            {
-                NodeCacheListener nodeCacheListener = () ->
-                {
-                    Optional<ModeledCachedNode<T>> currentData = getCurrentData();
-                    ModeledCacheEvent<T> event = new ModeledCacheEvent<T>()
-                    {
-                        @Override
-                        public ModeledCacheEventType getType()
-                        {
-                            return currentData.isPresent() ? ModeledCacheEventType.NODE_UPDATED : ModeledCacheEventType.NODE_REMOVED;
-                        }
-
-                        @Override
-                        public ModeledCachedNode<T> getNode()
-                        {
-                            return currentData.orElse(null);
-                        }
-                    };
-                    listener.event(event);
-                };
-                listenerMap.put(listener, nodeCacheListener);
-                cache.getListenable().addListener(nodeCacheListener, executor);
-            }
-
-            @Override
-            public void removeListener(ModeledCacheListener<T> listener)
-            {
-                NodeCacheListener nodeCacheListener = listenerMap.remove(listener);
-                if ( nodeCacheListener != null )
-                {
-                    cache.getListenable().removeListener(nodeCacheListener);
-                }
-            }
-        };
-    }
-
-    @Override
-    public Optional<ModeledCachedNode<T>> getCurrentData()
-    {
-        ChildData currentData = cache.getCurrentData();
-        if ( currentData == null )
-        {
-            return Optional.empty();
-        }
-        byte[] data = currentData.getData();
-        Stat stat = currentData.getStat();
-        if ( stat == null )
-        {
-            stat = new Stat();
-        }
-        if ( (data == null) || (data.length == 0) )
-        {
-            return Optional.of(new ModeledCachedNodeImpl<T>(path, null, stat));
-        }
-        return Optional.of(new ModeledCachedNodeImpl<>(path, serializer.deserialize(data), stat));
-    }
-
-    @Override
-    public void close()
-    {
-        CloseableUtils.closeQuietly(cache);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledPathChildrenCacheImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledPathChildrenCacheImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledPathChildrenCacheImpl.java
deleted file mode 100644
index ed86404..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledPathChildrenCacheImpl.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/**
- * 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.x.async.modeled.details.recipes;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.ChildData;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
-import org.apache.curator.utils.CloseableUtils;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEvent;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEventType;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheListener;
-import org.apache.curator.x.async.modeled.recipes.ModeledCachedNode;
-import org.apache.curator.x.async.modeled.recipes.ModeledPathChildrenCache;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.stream.Collectors;
-
-public class ModeledPathChildrenCacheImpl<T> implements ModeledPathChildrenCache<T>
-{
-    private final PathChildrenCache cache;
-    private final Map<ModeledCacheListener<T>, PathChildrenCacheListener> listenerMap = new ConcurrentHashMap<>();
-    private final ModelSerializer<T> serializer;
-
-    public ModeledPathChildrenCacheImpl(PathChildrenCache cache, ModelSerializer<T> serializer)
-    {
-        this.cache = Objects.requireNonNull(cache, "cache cannot be null");
-        this.serializer = Objects.requireNonNull(serializer, "serializer cannot be null");
-    }
-
-    @Override
-    public PathChildrenCache unwrap()
-    {
-        return cache;
-    }
-
-    @Override
-    public void start()
-    {
-        try
-        {
-            cache.start();
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("can't start cache", e);
-        }
-    }
-
-    @Override
-    public void start(PathChildrenCache.StartMode mode)
-    {
-        try
-        {
-            cache.start(mode);
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("can't start cache", e);
-        }
-    }
-
-    @Override
-    public void rebuild()
-    {
-        try
-        {
-            cache.rebuild();
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("can't rebuild cache", e);
-        }
-    }
-
-    @Override
-    public void rebuildNode(ZPath fullPath)
-    {
-        try
-        {
-            cache.rebuildNode(fullPath.fullPath());
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("can't rebuild cache at " + fullPath, e);
-        }
-    }
-
-    @Override
-    public Listenable<ModeledCacheListener<T>> getListenable()
-    {
-        return new Listenable<ModeledCacheListener<T>>()
-        {
-            @Override
-            public void addListener(ModeledCacheListener<T> listener)
-            {
-                addListener(listener, MoreExecutors.sameThreadExecutor());
-            }
-
-            @Override
-            public void addListener(ModeledCacheListener<T> listener, Executor executor)
-            {
-                PathChildrenCacheListener pathChildrenCacheListener = (client, event) -> {
-                    ModeledCacheEventType eventType = toType(event.getType());
-                    ModeledCachedNode<T> node = from(serializer, event.getData());
-                    ModeledCacheEvent<T> modeledEvent = new ModeledCacheEvent<T>()
-                    {
-                        @Override
-                        public ModeledCacheEventType getType()
-                        {
-                            return eventType;
-                        }
-
-                        @Override
-                        public ModeledCachedNode<T> getNode()
-                        {
-                            return node;
-                        }
-                    };
-                    listener.event(modeledEvent);
-                };
-                listenerMap.put(listener, pathChildrenCacheListener);
-                cache.getListenable().addListener(pathChildrenCacheListener);
-            }
-
-            @Override
-            public void removeListener(ModeledCacheListener listener)
-            {
-                PathChildrenCacheListener pathChildrenCacheListener = listenerMap.remove(listener);
-                if ( pathChildrenCacheListener != null )
-                {
-                    cache.getListenable().removeListener(pathChildrenCacheListener);
-                }
-            }
-        };
-    }
-
-    @Override
-    public List<ModeledCachedNode<T>> getCurrentData()
-    {
-        return cache.getCurrentData().stream()
-            .map(data -> from(serializer, data))
-            .collect(Collectors.toList());
-    }
-
-    @Override
-    public Map<ZPath, ModeledCachedNode<T>> getCurrentChildren(ZPath fullPath)
-    {
-        ChildData currentData = cache.getCurrentData(fullPath.fullPath());
-        if ( currentData == null )
-        {
-            return noChildrenValue();
-        }
-        Map<ZPath, ModeledCachedNode<T>> map = Maps.newHashMap();
-        map.put(fullPath, from(serializer, currentData));
-        return map;
-    }
-
-    @Override
-    public Optional<ModeledCachedNode<T>> getCurrentData(ZPath fullPath)
-    {
-        return Optional.ofNullable(from(serializer, cache.getCurrentData(fullPath.fullPath())));
-    }
-
-    @Override
-    public void clearDataBytes(ZPath fullPath)
-    {
-        cache.clearDataBytes(fullPath.fullPath());
-    }
-
-    @Override
-    public boolean clearDataBytes(ZPath fullPath, int ifVersion)
-    {
-        return cache.clearDataBytes(fullPath.fullPath(), ifVersion);
-    }
-
-    @Override
-    public void clearAndRefresh()
-    {
-        try
-        {
-            cache.clearAndRefresh();
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("could not clear and refresh", e);
-        }
-    }
-
-    @Override
-    public void clear()
-    {
-        cache.clear();
-    }
-
-    @Override
-    public void close()
-    {
-        CloseableUtils.closeQuietly(cache);
-    }
-
-    static <T> ModeledCachedNode<T> from(ModelSerializer<T> serializer, ChildData data)
-    {
-        if ( data == null )
-        {
-            return null;
-        }
-        T model = ((data.getData() != null) && (data.getData().length > 0)) ? serializer.deserialize(data.getData()) : null;
-        return new ModeledCachedNodeImpl<>(ZPath.parse(data.getPath()), model, data.getStat());
-    }
-
-    @VisibleForTesting
-    static ModeledCacheEventType toType(PathChildrenCacheEvent.Type type)
-    {
-        switch ( type )
-        {
-            case CHILD_ADDED:
-                return ModeledCacheEventType.NODE_ADDED;
-
-            case CHILD_UPDATED:
-                return ModeledCacheEventType.NODE_UPDATED;
-
-            case CHILD_REMOVED:
-                return ModeledCacheEventType.NODE_REMOVED;
-
-            case CONNECTION_SUSPENDED:
-                return ModeledCacheEventType.CONNECTION_SUSPENDED;
-
-            case CONNECTION_RECONNECTED:
-                return ModeledCacheEventType.CONNECTION_RECONNECTED;
-
-            case CONNECTION_LOST:
-                return ModeledCacheEventType.CONNECTION_LOST;
-
-            case INITIALIZED:
-                return ModeledCacheEventType.INITIALIZED;
-        }
-        throw new UnsupportedOperationException("Unknown type: " + type);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledTreeCacheImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledTreeCacheImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledTreeCacheImpl.java
deleted file mode 100644
index 7f0aecc..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/details/recipes/ModeledTreeCacheImpl.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * 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.x.async.modeled.details.recipes;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.ChildData;
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
-import org.apache.curator.framework.recipes.cache.TreeCacheListener;
-import org.apache.curator.utils.CloseableUtils;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEvent;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheEventType;
-import org.apache.curator.x.async.modeled.recipes.ModeledCacheListener;
-import org.apache.curator.x.async.modeled.recipes.ModeledCachedNode;
-import org.apache.curator.x.async.modeled.recipes.ModeledTreeCache;
-import java.util.AbstractMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.stream.Collectors;
-
-import static org.apache.curator.x.async.modeled.details.recipes.ModeledPathChildrenCacheImpl.from;
-
-public class ModeledTreeCacheImpl<T> implements ModeledTreeCache<T>
-{
-    private final TreeCache cache;
-    private final Map<ModeledCacheListener<T>, TreeCacheListener> listenerMap = new ConcurrentHashMap<>();
-    private final ModelSerializer<T> serializer;
-
-    public ModeledTreeCacheImpl(TreeCache cache, ModelSerializer<T> serializer)
-    {
-        this.cache = Objects.requireNonNull(cache, "cache cannot be null");
-        this.serializer = Objects.requireNonNull(serializer, "serializer cannot be null");
-    }
-
-    @Override
-    public TreeCache unwrap()
-    {
-        return cache;
-    }
-
-    @Override
-    public void start()
-    {
-        try
-        {
-            cache.start();
-        }
-        catch ( Exception e )
-        {
-            throw new RuntimeException("Could not start", e);
-        }
-    }
-
-    @Override
-    public void close()
-    {
-        CloseableUtils.closeQuietly(cache);
-    }
-
-    @Override
-    public Listenable<ModeledCacheListener<T>> getListenable()
-    {
-        return new Listenable<ModeledCacheListener<T>>()
-        {
-            @Override
-            public void addListener(ModeledCacheListener<T> listener)
-            {
-                addListener(listener, MoreExecutors.sameThreadExecutor());
-            }
-
-            @Override
-            public void addListener(ModeledCacheListener<T> listener, Executor executor)
-            {
-                TreeCacheListener treeCacheListener = (client, event) -> {
-                    ModeledCacheEventType eventType = toType(event.getType());
-                    ModeledCachedNode<T> node = from(serializer, event.getData());
-                    ModeledCacheEvent<T> wrappedEvent = new ModeledCacheEvent<T>()
-                    {
-                        @Override
-                        public ModeledCacheEventType getType()
-                        {
-                            return eventType;
-                        }
-
-                        @Override
-                        public ModeledCachedNode<T> getNode()
-                        {
-                            return node;
-                        }
-                    };
-                    listener.event(wrappedEvent);
-                };
-                listenerMap.put(listener, treeCacheListener);
-                cache.getListenable().addListener(treeCacheListener, executor);
-            }
-
-            @Override
-            public void removeListener(ModeledCacheListener<T> listener)
-            {
-                TreeCacheListener treeCacheListener = listenerMap.remove(listener);
-                if ( treeCacheListener != null )
-                {
-                    cache.getListenable().removeListener(treeCacheListener);
-                }
-            }
-        };
-    }
-
-    @Override
-    public Map<ZPath, ModeledCachedNode<T>> getCurrentChildren(ZPath fullPath)
-    {
-        Map<String, ChildData> currentChildren = cache.getCurrentChildren(fullPath.fullPath());
-        if ( currentChildren == null )
-        {
-            return noChildrenValue();
-        }
-        return currentChildren.entrySet().stream()
-            .map(entry -> new AbstractMap.SimpleEntry<>(ZPath.parse(entry.getKey()), from(serializer, entry.getValue())))
-            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
-    }
-
-    @Override
-    public Optional<ModeledCachedNode<T>> getCurrentData(ZPath fullPath)
-    {
-        return Optional.ofNullable(from(serializer, cache.getCurrentData(fullPath.fullPath())));
-    }
-
-    @VisibleForTesting
-    static ModeledCacheEventType toType(TreeCacheEvent.Type type)
-    {
-        switch ( type )
-        {
-            case NODE_ADDED:
-                return ModeledCacheEventType.NODE_ADDED;
-
-            case NODE_UPDATED:
-                return ModeledCacheEventType.NODE_UPDATED;
-
-            case NODE_REMOVED:
-                return ModeledCacheEventType.NODE_REMOVED;
-
-            case CONNECTION_SUSPENDED:
-                return ModeledCacheEventType.CONNECTION_SUSPENDED;
-
-            case CONNECTION_RECONNECTED:
-                return ModeledCacheEventType.CONNECTION_RECONNECTED;
-
-            case CONNECTION_LOST:
-                return ModeledCacheEventType.CONNECTION_LOST;
-
-            case INITIALIZED:
-                return ModeledCacheEventType.INITIALIZED;
-        }
-        throw new UnsupportedOperationException("Unknown type: " + type);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCache.java
deleted file mode 100644
index b810512..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCache.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import org.apache.curator.x.async.modeled.ZPath;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Optional;
-
-public interface ModeledCache<T>
-{
-    default Map<ZPath, ModeledCachedNode<T>> noChildrenValue()
-    {
-        return Collections.emptyMap();
-    }
-
-    /**
-     * Return the modeled current data for the given path. There are no guarantees of accuracy. This is
-     * merely the most recent view of the data. If there is no node at the given path,
-     * {@link java.util.Optional#empty()} is returned.
-     *
-     * @param fullPath full path to the node to check
-     * @return data if the node is alive, or null
-     */
-    Optional<ModeledCachedNode<T>> getCurrentData(ZPath fullPath);
-
-    /**
-     * Return the modeled current set of children at the given path, mapped by child name. There are no
-     * guarantees of accuracy; this is merely the most recent view of the data.
-     *
-     * @param fullPath full path to the node to check
-     * @return a possibly-empty list of children if the node is alive, or null
-     */
-    Map<ZPath, ModeledCachedNode<T>> getCurrentChildren(ZPath fullPath);
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEvent.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEvent.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEvent.java
deleted file mode 100644
index 469d8d6..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEvent.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-/**
- * Abstracts a cache event
- */
-public interface ModeledCacheEvent<T>
-{
-    /**
-     * The event type
-     *
-     * @return event type
-     */
-    ModeledCacheEventType getType();
-
-    /**
-     * Cached node if appropriate for the event (i.e. NODE_* events)
-     *
-     * @return node
-     */
-    ModeledCachedNode<T> getNode();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEventType.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEventType.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEventType.java
deleted file mode 100644
index bfdf57d..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheEventType.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-public enum ModeledCacheEventType
-{
-    /**
-     * A child was added to the path
-     */
-    NODE_ADDED,
-
-    /**
-     * A child's data was changed
-     */
-    NODE_UPDATED,
-
-    /**
-     * A child was removed from the path
-     */
-    NODE_REMOVED,
-
-    /**
-     * Called when the connection has changed to {@link org.apache.curator.framework.state.ConnectionState#SUSPENDED}
-     */
-    CONNECTION_SUSPENDED,
-
-    /**
-     * Called when the connection has changed to {@link org.apache.curator.framework.state.ConnectionState#RECONNECTED}
-     */
-    CONNECTION_RECONNECTED,
-
-    /**
-     * Called when the connection has changed to {@link org.apache.curator.framework.state.ConnectionState#LOST}
-     */
-    CONNECTION_LOST,
-
-    /**
-     * Signals that the initial cache has been populated.
-     */
-    INITIALIZED
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheListener.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheListener.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheListener.java
deleted file mode 100644
index 7b82e72..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCacheListener.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import java.util.function.Predicate;
-
-/**
- * Event listener
- */
-@FunctionalInterface
-public interface ModeledCacheListener<T>
-{
-    /**
-     * Receive an event
-     *
-     * @param event the event
-     */
-    void event(ModeledCacheEvent<T> event);
-
-    /**
-     * Wrap this listener with a filter
-     *
-     * @param filter test for events. Only events that pass the filter are sent to the listener
-     * @return filtered version of this listener
-     */
-    static <T> ModeledCacheListener<T> filtered(ModeledCacheListener<T> listener, Predicate<ModeledCacheEvent<T>> filter)
-    {
-        return event -> {
-            if ( filter.test(event) )
-            {
-                listener.event(event);
-            }
-        };
-    }
-
-    /**
-     * Filters out all but CRUD events
-     *
-     * @return predicate
-     */
-    static <T> Predicate<ModeledCacheEvent<T>> nodeEventFilter()
-    {
-        return event -> (event.getType() == ModeledCacheEventType.NODE_ADDED)
-            || (event.getType() == ModeledCacheEventType.NODE_UPDATED)
-            || (event.getType() == ModeledCacheEventType.NODE_REMOVED)
-            ;
-    }
-
-    /**
-     * Filters out all but {@link ModeledCacheEventType#NODE_REMOVED} events
-     *
-     * @return predicate
-     */
-    static <T> Predicate<ModeledCacheEvent<T>> nodeRemovedFilter()
-    {
-        return event -> event.getType() == ModeledCacheEventType.NODE_REMOVED;
-    }
-
-    /**
-     * Filters out all but events that have valid model instances
-     *
-     * @return predicate
-     */
-    static <T> Predicate<ModeledCacheEvent<T>> hasModelFilter()
-    {
-        return event -> (event.getNode() != null) && (event.getNode().getModel() != null);
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCachedNode.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCachedNode.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCachedNode.java
deleted file mode 100644
index a54f7b7..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledCachedNode.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.zookeeper.data.Stat;
-
-/**
- * Abstracts a cached node
- */
-public interface ModeledCachedNode<T>
-{
-    /**
-     * The path of the node
-     *
-     * @return path
-     */
-    ZPath getPath();
-
-    /**
-     * The node's last known stat if available
-     *
-     * @return stat
-     */
-    Stat getStat();
-
-    /**
-     * The node's current model
-     *
-     * @return model
-     */
-    T getModel();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledNodeCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledNodeCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledNodeCache.java
deleted file mode 100644
index a5fb598..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledNodeCache.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.NodeCache;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.details.recipes.ModeledNodeCacheImpl;
-import java.io.Closeable;
-import java.util.Optional;
-
-/**
- * Wraps a {@link org.apache.curator.framework.recipes.cache.NodeCache} so that
- * node data can be viewed as strongly typed models.
- */
-public interface ModeledNodeCache<T> extends Closeable
-{
-    /**
-     * Return a newly wrapped cache
-     *
-     * @param cache the cache to wrap
-     * @param serializer for the model
-     * @return new wrapped cache
-     */
-    static <T> ModeledNodeCache<T> wrap(NodeCache cache, ModelSerializer<T> serializer)
-    {
-        return new ModeledNodeCacheImpl<>(cache, serializer);
-    }
-
-    /**
-     * Return the original cache that was wrapped
-     *
-     * @return cache
-     */
-    NodeCache unwrap();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.NodeCache#start()}
-     */
-    void start();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.NodeCache#start(boolean)}
-     */
-    void start(boolean buildInitial);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.NodeCache#rebuild()}
-     */
-    void rebuild();
-
-    /**
-     * Return the listener container so that you can add/remove listeners. Note:
-     * {@link org.apache.curator.x.async.modeled.recipes.ModeledCacheEventType#INITIALIZED}
-     * and {@link org.apache.curator.x.async.modeled.recipes.ModeledCacheEventType#NODE_ADDED} are not
-     * used.
-     *
-     * @return listener container
-     */
-    Listenable<ModeledCacheListener<T>> getListenable();
-
-    /**
-     * Return the modeled current data. There are no guarantees of accuracy. This is
-     * merely the most recent view of the data. If the node does not exist,
-     * this returns {@link java.util.Optional#empty()} is returned
-     *
-     * @return node data
-     */
-    Optional<ModeledCachedNode<T>> getCurrentData();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.NodeCache#close()}
-     */
-    void close();
-}


[19/20] curator git commit: Major rework of caching. Having the wrapped caches adds little value. Focus on the integrated caching in the modeled client instance

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java
deleted file mode 100644
index 7b54fb7..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledPathChildrenCache.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.details.recipes.ModeledPathChildrenCacheImpl;
-import java.io.Closeable;
-import java.util.List;
-import java.util.Optional;
-
-/**
- * Wraps a {@link org.apache.curator.framework.recipes.cache.PathChildrenCache} so that
- * node data can be viewed as strongly typed models.
- */
-public interface ModeledPathChildrenCache<T> extends ModeledCache<T>, Closeable
-{
-    /**
-     * Return a newly wrapped cache
-     *
-     * @param cache the cache to wrap
-     * @param serializer model serializer
-     * @return new wrapped cache
-     */
-    static <T> ModeledPathChildrenCache<T> wrap(PathChildrenCache cache, ModelSerializer<T> serializer)
-    {
-        return new ModeledPathChildrenCacheImpl<>(cache, serializer);
-    }
-
-    /**
-     * Return the original cache that was wrapped
-     *
-     * @return cache
-     */
-    PathChildrenCache unwrap();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#start()}
-     */
-    void start();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#start(org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode)}
-     */
-    void start(PathChildrenCache.StartMode mode);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#rebuild()}
-     */
-    void rebuild();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#rebuildNode(String)}
-     */
-    void rebuildNode(ZPath fullPath);
-
-    /**
-     * Return the listener container so that you can add/remove listeners
-     *
-     * @return listener container
-     */
-    Listenable<ModeledCacheListener<T>> getListenable();
-
-    /**
-     * Return the modeled current data. There are no guarantees of accuracy. This is
-     * merely the most recent view of the data. The data is returned in sorted order.
-     *
-     * @return list of children and data
-     */
-    List<ModeledCachedNode<T>> getCurrentData();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clearDataBytes(String)}
-     */
-    void clearDataBytes(ZPath fullPath);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clearDataBytes(String, int)}
-     */
-    boolean clearDataBytes(ZPath fullPath, int ifVersion);
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clearAndRefresh()}
-     */
-    void clearAndRefresh();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#clear()}
-     */
-    void clear();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.PathChildrenCache#rebuildNode(String)}
-     */
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java b/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java
deleted file mode 100644
index 07eb191..0000000
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/modeled/recipes/ModeledTreeCache.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import org.apache.curator.framework.listen.Listenable;
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.x.async.modeled.ModelSerializer;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.details.recipes.ModeledTreeCacheImpl;
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- * Wraps a {@link org.apache.curator.framework.recipes.cache.TreeCache} so that
- * node data can be viewed as strongly typed models.
- */
-public interface ModeledTreeCache<T> extends ModeledCache<T>, Closeable
-{
-    /**
-     * Return a newly wrapped cache
-     *
-     * @param cache the cache to wrap
-     * @param serializer model serializer
-     * @return new wrapped cache
-     */
-    static <T> ModeledTreeCache<T> wrap(TreeCache cache, ModelSerializer<T> serializer)
-    {
-        return new ModeledTreeCacheImpl<>(cache, serializer);
-    }
-
-    /**
-     * Return the original cache that was wrapped
-     *
-     * @return cache
-     */
-    TreeCache unwrap();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.TreeCache#start()}
-     */
-    void start();
-
-    /**
-     * Forwards to {@link org.apache.curator.framework.recipes.cache.TreeCache#close()}
-     */
-    void close();
-
-    /**
-     * Return the listener container so that you can add/remove listeners
-     *
-     * @return listener container
-     */
-    Listenable<ModeledCacheListener<T>> getListenable();
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
index 4a0bed1..e4f0158 100644
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
+++ b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/TestCachedModeledCuratorFramework.java
@@ -23,7 +23,7 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.x.async.CompletableBaseClassForTests;
-import org.apache.curator.x.async.modeled.CachedModeledCuratorFramework;
+import org.apache.curator.x.async.modeled.cached.CachedModeledCuratorFramework;
 import org.apache.curator.x.async.modeled.CuratorModelSpec;
 import org.apache.curator.x.async.modeled.JacksonModelSerializer;
 import org.apache.curator.x.async.modeled.ModelSerializer;
@@ -69,7 +69,7 @@ public class TestCachedModeledCuratorFramework extends CompletableBaseClassForTe
         client.start();
 
         AtomicInteger counter = new AtomicInteger();
-        ((CachedModeledCuratorFrameworkImpl)client).debugCachedReadCount = counter;
+//        ((CachedModeledCuratorFrameworkImpl)client).debugCachedReadCount = counter;
 
         complete(client.read());
         Assert.assertEquals(counter.get(), 0);

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java
deleted file mode 100644
index 73c634f..0000000
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/details/recipes/TestEventTypeMappings.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.x.async.modeled.details.recipes;
-
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
-import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
-import org.testng.annotations.Test;
-
-public class TestEventTypeMappings
-{
-    @Test
-    public void testPathChildrenCacheTypes()
-    {
-        for ( PathChildrenCacheEvent.Type type : PathChildrenCacheEvent.Type.values() )
-        {
-            ModeledPathChildrenCacheImpl.toType(type);  // throws an exception on unknown types
-        }
-    }
-
-    @Test
-    public void testTreeCacheTypes()
-    {
-        for ( TreeCacheEvent.Type type : TreeCacheEvent.Type.values() )
-        {
-            ModeledTreeCacheImpl.toType(type);  // throws an exception on unknown types
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/2cbbf999/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java b/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java
deleted file mode 100644
index 0a32e9a..0000000
--- a/curator-x-async/src/test/java/org/apache/curator/x/async/modeled/recipes/TestModeledCaches.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/**
- * 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.x.async.modeled.recipes;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.recipes.cache.NodeCache;
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.utils.CloseableUtils;
-import org.apache.curator.x.async.CompletableBaseClassForTests;
-import org.apache.curator.x.async.modeled.CuratorModelSpec;
-import org.apache.curator.x.async.modeled.JacksonModelSerializer;
-import org.apache.curator.x.async.modeled.ModeledCuratorFramework;
-import org.apache.curator.x.async.modeled.ZPath;
-import org.apache.curator.x.async.modeled.models.TestModel;
-import org.apache.zookeeper.data.Stat;
-import org.testng.Assert;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-import java.math.BigInteger;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-public class TestModeledCaches extends CompletableBaseClassForTests
-{
-    private CuratorFramework client;
-    private JacksonModelSerializer<TestModel> serializer;
-    private ZPath path;
-    private ModeledCuratorFramework<TestModel> modeled;
-
-    @BeforeMethod
-    @Override
-    public void setup() throws Exception
-    {
-        super.setup();
-
-        client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
-        client.start();
-
-        serializer = JacksonModelSerializer.build(TestModel.class);
-
-        path = ZPath.parse("/test/path");
-        modeled = ModeledCuratorFramework.wrap(client, CuratorModelSpec.builder(path, serializer).build());
-    }
-
-    @AfterMethod
-    @Override
-    public void teardown() throws Exception
-    {
-        CloseableUtils.closeQuietly(client);
-
-        super.teardown();
-    }
-
-    @Test
-    public void testModeledNodeCache() throws InterruptedException
-    {
-        try ( ModeledNodeCache<TestModel> cache = ModeledNodeCache.wrap(new NodeCache(client, path.fullPath()), serializer) )
-        {
-            cache.start(true);
-
-            BlockingQueue<ModeledCacheEvent<TestModel>> events = new LinkedBlockingQueue<>();
-            ModeledCacheListener<TestModel> listener = events::add;
-            cache.getListenable().addListener(listener);
-
-            TestModel model1 = new TestModel("a", "b", "c", 1, BigInteger.TEN);
-            TestModel model2 = new TestModel("d", "e", "f", 10, BigInteger.ONE);
-
-            Stat stat = new Stat();
-            modeled.set(model1, stat);
-            ModeledCacheEvent<TestModel> event = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event);
-            Assert.assertEquals(event.getType(), ModeledCacheEventType.NODE_UPDATED);
-            Assert.assertNotNull(event.getNode());
-            Assert.assertEquals(event.getNode().getPath(), path);
-            Assert.assertEquals(event.getNode().getModel(), model1);
-            Assert.assertEquals(event.getNode().getStat(), stat);
-
-            timing.sleepABit();
-            Assert.assertEquals(events.size(), 0);
-
-            modeled.update(model2);
-            event = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event.getNode());
-            Assert.assertEquals(event.getNode().getPath(), path);
-            Assert.assertEquals(event.getNode().getModel(), model2);
-
-            modeled.delete();
-            event = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNull(event.getNode());
-        }
-    }
-
-    @Test
-    public void testModeledPathChildrenCache() throws InterruptedException
-    {
-        try ( ModeledPathChildrenCache<TestModel> cache = ModeledPathChildrenCache.wrap(new PathChildrenCache(client, path.fullPath(), true), serializer) )
-        {
-            cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
-
-            BlockingQueue<ModeledCacheEvent<TestModel>> events = new LinkedBlockingQueue<>();
-            ModeledCacheListener<TestModel> listener = events::add;
-            cache.getListenable().addListener(listener);
-
-            TestModel model1 = new TestModel("a", "b", "c", 1, BigInteger.TEN);
-            TestModel model2 = new TestModel("d", "e", "f", 10, BigInteger.ONE);
-            TestModel model3 = new TestModel("g", "h", "i", 100, BigInteger.ZERO);
-
-            modeled.at("1").set(model1).thenApply(__ -> modeled.at("2").set(model2));
-            ModeledCacheEvent<TestModel> event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            ModeledCacheEvent<TestModel> event2 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertNotNull(event2);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals(event2.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals((event1.getNode() != null) ? event1.getNode().getModel() : null, model1);
-            Assert.assertEquals((event2.getNode() != null) ? event2.getNode().getModel() : null, model2);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("1"));
-            Assert.assertEquals(event2.getNode().getPath(), path.at("2"));
-
-            modeled.at("1").delete();
-            event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_REMOVED);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("1"));
-
-            modeled.at("2").update(model3);
-            event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_UPDATED);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("2"));
-            Assert.assertEquals((event1.getNode() != null) ? event1.getNode().getModel() : null, model3);
-
-            cache.getListenable().removeListener(listener);
-            modeled.at("2").delete();
-            Assert.assertNull(events.poll(timing.forSleepingABit().milliseconds(), TimeUnit.MILLISECONDS));  // listener is removed - shouldn't get an event
-        }
-    }
-
-    @Test
-    public void testModeledTreeCache() throws Exception
-    {
-        try (ModeledTreeCache<TestModel> cache = ModeledTreeCache.wrap(TreeCache.newBuilder(client, path.fullPath()).build(), serializer) )
-        {
-            BlockingQueue<ModeledCacheEvent<TestModel>> events = new LinkedBlockingQueue<>();
-            ModeledCacheListener<TestModel> listener = ModeledCacheListener.filtered(events::add, ModeledCacheListener.<TestModel>nodeRemovedFilter().or(ModeledCacheListener.hasModelFilter()));
-            cache.getListenable().addListener(listener);
-
-            cache.start();
-
-            TestModel model1 = new TestModel("a", "b", "c", 1, BigInteger.TEN);
-            TestModel model2 = new TestModel("d", "e", "f", 10, BigInteger.ONE);
-            TestModel model3 = new TestModel("g", "h", "i", 100, BigInteger.ZERO);
-
-            modeled.at("1").set(model1).thenApply(__ -> modeled.at("1").at("2").set(model2).thenApply(___ -> modeled.at("1").at("2").at("3").set(model3)));
-            ModeledCacheEvent<TestModel> event1 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            ModeledCacheEvent<TestModel> event2 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            ModeledCacheEvent<TestModel> event3 = events.poll(timing.milliseconds(), TimeUnit.MILLISECONDS);
-            Assert.assertNotNull(event1);
-            Assert.assertNotNull(event2);
-            Assert.assertNotNull(event3);
-            Assert.assertEquals(event1.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals(event2.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals(event3.getType(), ModeledCacheEventType.NODE_ADDED);
-            Assert.assertEquals((event1.getNode() != null) ? event1.getNode().getModel() : null, model1);
-            Assert.assertEquals((event2.getNode() != null) ? event2.getNode().getModel() : null, model2);
-            Assert.assertEquals((event3.getNode() != null) ? event3.getNode().getModel() : null, model3);
-            Assert.assertEquals(event1.getNode().getPath(), path.at("1"));
-            Assert.assertEquals(event2.getNode().getPath(), path.at("1").at("2"));
-            Assert.assertEquals(event3.getNode().getPath(), path.at("1").at("2").at("3"));
-        }
-    }
-}


[15/20] curator git commit: assertTrue for node.waitForInitialCreate

Posted by ra...@apache.org.
assertTrue for node.waitForInitialCreate


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

Branch: refs/heads/CURATOR-397
Commit: b84d351a5f3c44ab5450c690b462388cb0a90697
Parents: da7f18c
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 17 20:26:59 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 17 20:26:59 2017 -0500

----------------------------------------------------------------------
 .../curator/framework/recipes/nodes/TestPersistentTtlNode.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/b84d351a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
index 2224bbf..43f5cc0 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -66,7 +66,7 @@ public class TestPersistentTtlNode extends BaseClassForTests
             try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 100, new byte[0]))
             {
                 node.start();
-                node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
+                Assert.assertTrue(node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS));
 
                 for ( int i = 0; i < 10; ++i )
                 {
@@ -91,7 +91,7 @@ public class TestPersistentTtlNode extends BaseClassForTests
             try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 10, new byte[0]))
             {
                 node.start();
-                node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
+                Assert.assertTrue(node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS));
 
                 for ( int i = 0; i < 10; ++i )
                 {
@@ -131,7 +131,7 @@ public class TestPersistentTtlNode extends BaseClassForTests
                     cache.getListenable().addListener(listener);
 
                     node.start();
-                    node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
+                    Assert.assertTrue(node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS));
                     cache.start(BUILD_INITIAL_CACHE);
 
                     Assert.assertEquals(changes.availablePermits(), 0);


[18/20] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-397

Posted by ra...@apache.org.
Merge branch 'CURATOR-3.0' into CURATOR-397


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

Branch: refs/heads/CURATOR-397
Commit: e40ed181655a83e705e6d74704d2fa2d1f93a7bc
Parents: 3d59310 35f5d27
Author: randgalt <ra...@apache.org>
Authored: Mon May 1 14:55:59 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon May 1 14:55:59 2017 -0500

----------------------------------------------------------------------
 .../org/apache/curator/utils/ThreadUtils.java   |   4 +-
 .../curator/framework/api/CreateBuilder.java    |  17 +-
 .../curator/framework/api/CreateBuilder2.java   |  34 +++
 .../framework/imps/CreateBuilderImpl.java       | 134 +++++-------
 .../curator/framework/imps/TestTtlNodes.java    |  88 ++++++++
 .../framework/recipes/nodes/PersistentNode.java |  50 +++--
 .../recipes/nodes/PersistentTtlNode.java        | 209 +++++++++++++++++++
 .../src/site/confluence/index.confluence        |   3 +-
 .../persistent-ephemeral-node.confluence        |  39 ----
 .../site/confluence/persistent-node.confluence  |  39 ++++
 .../confluence/persistent-ttl-node.confluence   |  41 ++++
 .../recipes/nodes/TestPersistentTtlNode.java    | 153 ++++++++++++++
 .../curator/x/async/api/AsyncCreateBuilder.java |  29 +++
 .../x/async/details/AsyncCreateBuilderImpl.java |  22 +-
 pom.xml                                         |   2 +-
 15 files changed, 727 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/e40ed181/pom.xml
----------------------------------------------------------------------


[06/20] curator git commit: more doc and tests

Posted by ra...@apache.org.
more doc and tests


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

Branch: refs/heads/CURATOR-397
Commit: 7aa5617e299a6259de3360bef307a231b426bcfb
Parents: 994778c
Author: randgalt <ra...@apache.org>
Authored: Fri Dec 9 15:35:50 2016 +0100
Committer: randgalt <ra...@apache.org>
Committed: Fri Dec 9 15:35:50 2016 +0100

----------------------------------------------------------------------
 .../src/site/confluence/index.confluence        |  2 +-
 .../persistent-ephemeral-node.confluence        | 39 --------------
 .../site/confluence/persistent-node.confluence  | 39 ++++++++++++++
 .../confluence/persistent-ttl-node.confluence   |  2 +
 .../recipes/nodes/TestPersistentTtlNode.java    | 55 ++++++++++++++++++++
 5 files changed, 97 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/7aa5617e/curator-recipes/src/site/confluence/index.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/index.confluence b/curator-recipes/src/site/confluence/index.confluence
index 4cf2cde..08ef762 100644
--- a/curator-recipes/src/site/confluence/index.confluence
+++ b/curator-recipes/src/site/confluence/index.confluence
@@ -29,7 +29,7 @@ regarding "Curator Recipes Own Their ZNode/Paths".
 |[[Tree Cache|tree-cache.html]] \- A utility that attempts to keep all data from all children of a ZK path locally cached. This class will watch the ZK path, respond to update/create/delete events, pull down the data, etc. You can register a listener that will get notified when changes occur.|
 
 ||Nodes||
-|[[Persistent Node|persistent-ephemeral-node.html]] \- A node that attempts to stay present in ZooKeeper, even through connection and session interruptions.|
+|[[Persistent Node|persistent-node.html]] \- A node that attempts to stay present in ZooKeeper, even through connection and session interruptions.|
 |[[Persistent TTL Node|persistent-ttl-node.html]] \- Useful when you need to create a TTL node but don't want to keep it alive manually by periodically setting data.|
 |[Group Member|group-member.html]] \- Group membership management. Adds this instance into a group and keeps a cache of members in the group.|
 

http://git-wip-us.apache.org/repos/asf/curator/blob/7aa5617e/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence b/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
deleted file mode 100644
index 5baca09..0000000
--- a/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
+++ /dev/null
@@ -1,39 +0,0 @@
-h1. Persistent Node
-
-h2. Description
-A persistent node is a node that attempts to stay present in ZooKeeper, even through connection and session interruptions.
-
-h2. Participating Classes
-* PersistentNode
-
-h2. Usage
-h3. Creating a PersistentNode
-{code}
-public PersistentNode(CuratorFramework client,
-                               CreateMode mode,
-                               boolean useProtection,
-                               String basePath,
-                               byte[] data)
-Parameters:
-client - client instance
-mode - creation mode
-useProtection - if true, call CreateBuilder.withProtection()
-basePath - the base path for the node
-data - data for the node
-{code}
-
-h3. General Usage
-PersistentNodes must be started:
-{code}
-node.start();
-{code}
-
-When you are through with the PersistentNode instance, you should call close:
-{code}
-node.close();
-{code}
-
-NOTE: this will delete the node
-
-h2. Error Handling
-PersistentNode instances internally handle all error states recreating the node as necessary.

http://git-wip-us.apache.org/repos/asf/curator/blob/7aa5617e/curator-recipes/src/site/confluence/persistent-node.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/persistent-node.confluence b/curator-recipes/src/site/confluence/persistent-node.confluence
new file mode 100644
index 0000000..5baca09
--- /dev/null
+++ b/curator-recipes/src/site/confluence/persistent-node.confluence
@@ -0,0 +1,39 @@
+h1. Persistent Node
+
+h2. Description
+A persistent node is a node that attempts to stay present in ZooKeeper, even through connection and session interruptions.
+
+h2. Participating Classes
+* PersistentNode
+
+h2. Usage
+h3. Creating a PersistentNode
+{code}
+public PersistentNode(CuratorFramework client,
+                               CreateMode mode,
+                               boolean useProtection,
+                               String basePath,
+                               byte[] data)
+Parameters:
+client - client instance
+mode - creation mode
+useProtection - if true, call CreateBuilder.withProtection()
+basePath - the base path for the node
+data - data for the node
+{code}
+
+h3. General Usage
+PersistentNodes must be started:
+{code}
+node.start();
+{code}
+
+When you are through with the PersistentNode instance, you should call close:
+{code}
+node.close();
+{code}
+
+NOTE: this will delete the node
+
+h2. Error Handling
+PersistentNode instances internally handle all error states recreating the node as necessary.

http://git-wip-us.apache.org/repos/asf/curator/blob/7aa5617e/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/persistent-ttl-node.confluence b/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
index acd2d33..f123c1d 100644
--- a/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
+++ b/curator-recipes/src/site/confluence/persistent-ttl-node.confluence
@@ -3,6 +3,8 @@ h1. Persistent TTL Node
 h2. Description
 PersistentTtlNode is useful when you need to create a TTL node but don't want to keep it alive manually by periodically setting data \-
 PersistentTtlNode does that for you. Further the keep\-alive is done in a way that does not generate watch triggers on the parent node.
+It also provides similar guarantees that a [[Persistent Node|persistent-node.html]] does: the node attempts to stay present in ZooKeeper,
+even through connection and session interruptions.
 
 h2. Participating Classes
 * PersistentNode

http://git-wip-us.apache.org/repos/asf/curator/blob/7aa5617e/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
index 9ae8df9..848c034 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -2,6 +2,9 @@ package org.apache.curator.framework.recipes.nodes;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.recipes.cache.PathChildrenCache;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
@@ -9,8 +12,11 @@ import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode.BUILD_INITIAL_CACHE;
+
 public class TestPersistentTtlNode extends BaseClassForTests
 {
     private final Timing timing = new Timing();
@@ -55,4 +61,53 @@ public class TestPersistentTtlNode extends BaseClassForTests
             Assert.assertNull(client.checkExists().forPath("/test"));
         }
     }
+
+    @Test
+    public void testEventsOnParent() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)))
+        {
+            client.start();
+
+            try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 10, new byte[0]))
+            {
+                try(PathChildrenCache cache = new PathChildrenCache(client, "/", true))
+                {
+                    final Semaphore changes = new Semaphore(0);
+                    PathChildrenCacheListener listener = new PathChildrenCacheListener()
+                    {
+                        @Override
+                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+                        {
+                            if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_UPDATED )
+                            {
+                                if ( event.getData().getPath().equals("/test") )
+                                {
+                                    changes.release();
+                                }
+                            }
+                        }
+                    };
+                    cache.getListenable().addListener(listener);
+
+                    node.start();
+                    node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
+                    cache.start(BUILD_INITIAL_CACHE);
+
+                    Assert.assertEquals(changes.availablePermits(), 0);
+                    timing.sleepABit();
+                    Assert.assertEquals(changes.availablePermits(), 0);
+
+                    client.setData().forPath("/test", "changed".getBytes());
+                    Assert.assertTrue(timing.acquireSemaphore(changes));
+                    timing.sleepABit();
+                    Assert.assertEquals(changes.availablePermits(), 0);
+                }
+            }
+
+            timing.sleepABit();
+
+            Assert.assertNull(client.checkExists().forPath("/test"));
+        }
+    }
 }


[08/20] curator git commit: more testing

Posted by ra...@apache.org.
more testing


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

Branch: refs/heads/CURATOR-397
Commit: a9cbdd31d44872d5fb0831f1b883431abe4e6994
Parents: be4960a
Author: randgalt <ra...@apache.org>
Authored: Fri Dec 9 16:07:11 2016 +0100
Committer: randgalt <ra...@apache.org>
Committed: Fri Dec 9 16:07:11 2016 +0100

----------------------------------------------------------------------
 .../recipes/nodes/TestPersistentTtlNode.java    | 33 +++++++++++++++++---
 1 file changed, 29 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/a9cbdd31/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
index be38f6e..e522103 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -8,6 +8,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.utils.ZKPaths;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
@@ -44,25 +45,49 @@ public class TestPersistentTtlNode extends BaseClassForTests
         {
             client.start();
 
-            try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 10, new byte[0]))
+            try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 100, new byte[0]))
             {
                 node.start();
                 node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
 
                 for ( int i = 0; i < 10; ++i )
                 {
-                    Thread.sleep(10);
+                    Thread.sleep(110);  // sleep a bit more than the TTL
                     Assert.assertNotNull(client.checkExists().forPath("/test"));
                 }
             }
+            Assert.assertNotNull(client.checkExists().forPath("/test"));
 
             timing.sleepABit();
-
             Assert.assertNull(client.checkExists().forPath("/test"));
         }
     }
 
     @Test
+    public void testForcedDeleteOfTouchNode() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)))
+        {
+            client.start();
+
+            try (PersistentTtlNode node = new PersistentTtlNode(client, "/test", 10, new byte[0]))
+            {
+                node.start();
+                node.waitForInitialCreate(timing.session(), TimeUnit.MILLISECONDS);
+
+                for ( int i = 0; i < 10; ++i )
+                {
+                    Thread.sleep(10);
+                    client.delete().quietly().forPath(ZKPaths.makePath("test", PersistentTtlNode.DEFAULT_CHILD_NODE_NAME));
+                }
+
+                timing.sleepABit();
+                Assert.assertNotNull(client.checkExists().forPath("/test"));
+            }
+        }
+    }
+
+    @Test
     public void testEventsOnParent() throws Exception
     {
         try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)))
@@ -79,7 +104,7 @@ public class TestPersistentTtlNode extends BaseClassForTests
                         @Override
                         public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                         {
-                            if ( (event.getData() != null) && "/test".equals(event.getData().getPath()) )
+                            if ( (event.getType() == PathChildrenCacheEvent.Type.CHILD_UPDATED) && "/test".equals(event.getData().getPath()) )
                             {
                                 changes.release();
                             }


[02/20] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-351

Posted by ra...@apache.org.
Merge branch 'CURATOR-3.0' into CURATOR-351


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

Branch: refs/heads/CURATOR-397
Commit: 786d69587d183250b4108a9e18898ad4b2e94659
Parents: 521652d d034aea
Author: randgalt <ra...@apache.org>
Authored: Fri Oct 28 10:05:52 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Oct 28 10:05:52 2016 -0500

----------------------------------------------------------------------
 .../org/apache/curator/ConnectionState.java     |  36 ++++-
 .../apache/curator/CuratorZookeeperClient.java  |  13 +-
 .../main/java/org/apache/curator/RetryLoop.java |   5 +-
 .../curator/drivers/AdvancedTracerDriver.java   |  50 ++++++
 .../org/apache/curator/drivers/EventTrace.java  |  53 ++++++
 .../apache/curator/drivers/OperationTrace.java  | 162 +++++++++++++++++++
 .../framework/imps/BackgroundSyncImpl.java      |   9 +-
 .../framework/imps/CreateBuilderImpl.java       |  66 ++++----
 .../framework/imps/CuratorFrameworkImpl.java    |   4 +-
 .../imps/CuratorMultiTransactionImpl.java       |   4 +-
 .../framework/imps/DeleteBuilderImpl.java       |  12 +-
 .../curator/framework/imps/EnsembleTracker.java |   5 +-
 .../framework/imps/ExistsBuilderImpl.java       |  16 +-
 .../FindAndDeleteProtectedNodeInBackground.java |   6 +-
 .../framework/imps/GetACLBuilderImpl.java       |  10 +-
 .../framework/imps/GetChildrenBuilderImpl.java  |  12 +-
 .../framework/imps/GetDataBuilderImpl.java      |  12 +-
 .../framework/imps/GzipCompressionProvider.java |  26 +--
 .../framework/imps/SetACLBuilderImpl.java       |  10 +-
 .../framework/imps/SetDataBuilderImpl.java      |  13 +-
 .../curator/framework/imps/SyncBuilderImpl.java |   6 +-
 .../framework/imps/TempGetDataBuilderImpl.java  |   6 +-
 .../apache/curator/framework/schema/Schema.java |  22 +--
 .../curator/framework/schema/SchemaSet.java     |   2 +-
 .../framework/schema/SchemaViolation.java       | 101 ++++++++++++
 src/site/confluence/logging.confluence          |   2 +-
 26 files changed, 541 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/786d6958/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index 6ec2943,4bed564..cc2b541
--- 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
@@@ -545,52 -535,83 +545,52 @@@ class CreateBuilderImpl implements Crea
      {
          try
          {
-             final TimeTrace trace = client.getZookeeperClient().startTracer("CreateBuilderImpl-Background");
+             final OperationTrace trace = client.getZookeeperClient().startAdvancedTracer("CreateBuilderImpl-Background");
+             final byte[] data = operationAndData.getData().getData();
  
 -            if(storingStat == null)
 -            {
 -                client.getZooKeeper().create
 -                (
 -                    operationAndData.getData().getPath(),
 -                    data,
 -                    acling.getAclList(operationAndData.getData().getPath()),
 -                    createMode,
 -                    new AsyncCallback.StringCallback()
 -                    {
 -                        @Override
 -                        public void processResult(int rc, String path, Object ctx, String name)
 -                        {
 -                            trace.setReturnCode(rc).setRequestBytesLength(data).setPath(path).commit();
 +            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();
++                        trace.setReturnCode(rc).setRequestBytesLength(data).setPath(path).commit();
  
 -                            if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
 -                            {
 -                                backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
 -                            }
 -                            else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
 -                            {
 -                                backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
 -                            }
 -                            else
 -                            {
 -                                sendBackgroundResponse(rc, path, ctx, name, null, operationAndData);
 -                            }
 +                        if ( (stat != null) && (storingStat != null) )
 +                        {
 +                            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());
                          }
 -                    },
 -                    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();
 -
 -                            if ( stat != null )
 -                            {
 -                                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);
 -                            }
 +                        if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
 +                        {
 +                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
                          }
 -                    },
 -                    backgrounding.getContext()
 -                );
 -            }
 +                        else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
 +                        {
 +                            backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
 +                        }
 +                        else
 +                        {
 +                            sendBackgroundResponse(rc, path, ctx, name, stat, operationAndData);
 +                        }
 +                    }
 +                },
 +                backgrounding.getContext(),
 +                ttl
 +            );
          }
          catch ( Throwable e )
          {


[11/20] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-351

Posted by ra...@apache.org.
Merge branch 'CURATOR-3.0' into CURATOR-351


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

Branch: refs/heads/CURATOR-397
Commit: 1826b66d72f6e543a57aaf5f3acc3cd9ca6e965d
Parents: e7d57ec b3939ac
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 3 10:54:50 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 3 10:54:50 2017 -0500

----------------------------------------------------------------------
 README                                          |   7 -
 README.md                                       |  11 +
 curator-client/pom.xml                          |  44 +-
 .../org/apache/curator/ConnectionState.java     |  18 +-
 .../ensemble/fixed/FixedEnsembleProvider.java   |   4 +-
 .../org/apache/curator/utils/DebugUtils.java    |   1 +
 curator-examples/pom.xml                        |   2 +-
 curator-framework/pom.xml                       |   4 +-
 .../curator/framework/CuratorFramework.java     |   1 +
 .../curator/framework/api/CreateBuilder.java    |  10 +
 .../curator/framework/api/CreateBuilder2.java   |  10 +
 .../curator/framework/api/ExistsBuilder.java    |   7 +
 .../curator/framework/imps/Backgrounding.java   |  10 +-
 .../framework/imps/CreateBuilderImpl.java       |  24 +-
 .../imps/CuratorMultiTransactionImpl.java       |   6 +
 .../framework/imps/DeleteBuilderImpl.java       |  12 +-
 .../curator/framework/imps/EnsembleTracker.java |   4 +-
 .../framework/imps/ExistsBuilderImpl.java       |  29 +-
 .../framework/imps/GetACLBuilderImpl.java       |   9 +-
 .../framework/imps/GetChildrenBuilderImpl.java  |  10 +-
 .../framework/imps/GetConfigBuilderImpl.java    |   8 +
 .../framework/imps/GetDataBuilderImpl.java      |  11 +-
 .../curator/framework/imps/NamespaceImpl.java   |   2 +-
 .../framework/imps/ReconfigBuilderImpl.java     |  13 +-
 .../imps/RemoveWatchesBuilderImpl.java          |  12 +
 .../framework/imps/SetACLBuilderImpl.java       |  10 +-
 .../framework/imps/SetDataBuilderImpl.java      |  10 +-
 .../curator/framework/imps/SyncBuilderImpl.java |   6 +
 .../src/site/confluence/index.confluence        |   2 +
 .../framework/imps/TestNamespaceFacade.java     |  10 +
 curator-recipes/pom.xml                         |  10 +-
 .../recipes/cache/PathChildrenCache.java        |   2 +-
 .../framework/recipes/cache/TreeCache.java      |  67 +-
 .../recipes/leader/LeaderSelector.java          |  48 +-
 .../recipes/locks/InterProcessSemaphore.java    |   6 +
 .../locks/InterProcessSemaphoreMutex.java       |  13 +-
 .../recipes/locks/InterProcessSemaphoreV2.java  |   5 +
 .../curator/framework/recipes/locks/Lease.java  |   8 +
 .../framework/recipes/nodes/PersistentNode.java |  28 +-
 .../cache/TestPathChildrenCacheInCluster.java   |  60 ++
 .../recipes/leader/TestLeaderAcls.java          | 133 ++++
 .../recipes/leader/TestLeaderSelector.java      |   2 +-
 .../leader/TestLeaderSelectorParticipants.java  |   2 +-
 .../locks/TestInterProcessSemaphore.java        |  11 +-
 .../recipes/nodes/TestGroupMember.java          |  28 +-
 .../nodes/TestPersistentEphemeralNode.java      |  52 +-
 curator-test/pom.xml                            |  51 +-
 .../apache/curator/test/BaseClassForTests.java  | 148 ++++-
 .../org/apache/curator/test/InstanceSpec.java   |  58 +-
 .../curator/test/QuorumConfigBuilder.java       |   9 +-
 .../curator/test/TestQuorumConfigBuilder.java   |  49 ++
 curator-x-async/pom.xml                         |  46 ++
 .../curator/x/async/AsyncCuratorFramework.java  | 110 ++++
 .../curator/x/async/AsyncEventException.java    |  46 ++
 .../org/apache/curator/x/async/AsyncResult.java | 118 ++++
 .../org/apache/curator/x/async/AsyncStage.java  |  37 ++
 .../org/apache/curator/x/async/WatchMode.java   |  43 ++
 .../curator/x/async/api/AsyncCreateBuilder.java | 144 +++++
 .../x/async/api/AsyncCuratorFrameworkDsl.java   | 116 ++++
 .../curator/x/async/api/AsyncDeleteBuilder.java |  55 ++
 .../curator/x/async/api/AsyncEnsemblable.java   |  32 +
 .../curator/x/async/api/AsyncExistsBuilder.java |  37 ++
 .../curator/x/async/api/AsyncGetACLBuilder.java |  38 ++
 .../x/async/api/AsyncGetChildrenBuilder.java    |  37 ++
 .../x/async/api/AsyncGetConfigBuilder.java      |  36 ++
 .../x/async/api/AsyncGetDataBuilder.java        |  53 ++
 .../x/async/api/AsyncMultiTransaction.java      |  39 ++
 .../x/async/api/AsyncPathAndBytesable.java      |  36 ++
 .../curator/x/async/api/AsyncPathable.java      |  34 +
 .../x/async/api/AsyncReconfigBuilder.java       | 118 ++++
 .../x/async/api/AsyncRemoveWatchesBuilder.java  | 126 ++++
 .../curator/x/async/api/AsyncSetACLBuilder.java |  49 ++
 .../x/async/api/AsyncSetDataBuilder.java        |  54 ++
 .../curator/x/async/api/AsyncSyncBuilder.java   |  29 +
 .../async/api/AsyncTransactionCheckBuilder.java |  35 +
 .../api/AsyncTransactionCreateBuilder.java      |  66 ++
 .../api/AsyncTransactionDeleteBuilder.java      |  35 +
 .../curator/x/async/api/AsyncTransactionOp.java |  54 ++
 .../api/AsyncTransactionSetDataBuilder.java     |  51 ++
 .../curator/x/async/api/CreateOption.java       |  76 +++
 .../curator/x/async/api/DeleteOption.java       |  44 ++
 .../curator/x/async/api/ExistsOption.java       |  35 +
 .../x/async/api/RemoveWatcherOption.java        |  45 ++
 .../api/WatchableAsyncCuratorFramework.java     |  54 ++
 .../x/async/details/AsyncCreateBuilderImpl.java | 164 +++++
 .../details/AsyncCuratorFrameworkImpl.java      | 228 +++++++
 .../x/async/details/AsyncDeleteBuilderImpl.java |  75 +++
 .../x/async/details/AsyncExistsBuilderImpl.java |  69 ++
 .../details/AsyncGetChildrenBuilderImpl.java    |  61 ++
 .../details/AsyncGetConfigBuilderImpl.java      |  60 ++
 .../async/details/AsyncGetDataBuilderImpl.java  |  76 +++
 .../async/details/AsyncReconfigBuilderImpl.java | 125 ++++
 .../details/AsyncRemoveWatchesBuilderImpl.java  | 174 +++++
 .../x/async/details/AsyncResultImpl.java        | 142 +++++
 .../x/async/details/AsyncSetACLBuilderImpl.java |  68 ++
 .../async/details/AsyncSetDataBuilderImpl.java  |  84 +++
 .../x/async/details/AsyncTransactionOpImpl.java | 233 +++++++
 .../curator/x/async/details/BackgroundProc.java |  27 +
 .../x/async/details/BackgroundProcs.java        |  83 +++
 .../curator/x/async/details/BuilderCommon.java  |  41 ++
 .../apache/curator/x/async/details/Filters.java |  53 ++
 .../x/async/details/InternalCallback.java       |  55 ++
 .../x/async/details/InternalWatcher.java        |  96 +++
 .../src/site/confluence/index.confluence        | 224 +++++++
 curator-x-async/src/site/site.xml               |  33 +
 .../curator/framework/imps/TestFramework.java   | 631 +++++++++++++++++++
 .../framework/imps/TestFrameworkBackground.java | 290 +++++++++
 .../curator/x/async/TestBasicOperations.java    | 216 +++++++
 curator-x-discovery-server/pom.xml              |   4 +-
 curator-x-discovery/pom.xml                     |  10 +-
 .../curator/x/discovery/ServiceInstance.java    |  32 +-
 .../x/discovery/ServiceInstanceBuilder.java     |   9 +-
 .../discovery/details/ServiceProviderImpl.java  |   8 +
 .../x/discovery/TestJsonInstanceSerializer.java |  35 +-
 .../discovery/details/TestServiceProvider.java  | 109 ++++
 curator-x-rpc/pom.xml                           |   4 +-
 curator-x-rpc/src/site/site.xml                 |   2 +-
 pom.xml                                         |  63 +-
 src/site/confluence/index.confluence            |   2 +
 src/site/site.xml                               |   1 +
 120 files changed, 6435 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
index 431a945,564d11b..59b3510
--- 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
@@@ -18,9 -18,8 +18,19 @@@
   */
  package org.apache.curator.framework.api;
  
 -public interface CreateBuilder extends
 -    CreateBuilderMain
 +public interface CreateBuilder extends CreateBuilderMain
  {
 -    CreateBuilderMain orSetData();
++    /**
++     * Specify a TTL when mode is {@link org.apache.zookeeper.CreateMode#PERSISTENT_WITH_TTL} or
++     * {@link org.apache.zookeeper.CreateMode#PERSISTENT_SEQUENTIAL_WITH_TTL}. If
++     * the znode has not been modified within the given TTL, it will be deleted once it has no
++     * children. The TTL unit is milliseconds and must be greater than 0 and less than or equal to
++     * EphemeralType.MAX_TTL.
++     *
++     * @param ttl the ttl
++     * @return this for chaining
++     */
 +    CreateBuilderMain withTtl(long ttl);
 +
 +    CreateBuilder2 orSetData();
  }

http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
index 7e5c89a,0000000..ddc27c3
mode 100644,000000..100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
@@@ -1,24 -1,0 +1,34 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + */
 +package org.apache.curator.framework.api;
 +
 +public interface CreateBuilder2 extends CreateBuilderMain
 +{
++    /**
++     * Specify a TTL when mode is {@link org.apache.zookeeper.CreateMode#PERSISTENT_WITH_TTL} or
++     * {@link org.apache.zookeeper.CreateMode#PERSISTENT_SEQUENTIAL_WITH_TTL}. If
++     * the znode has not been modified within the given TTL, it will be deleted once it has no
++     * children. The TTL unit is milliseconds and must be greater than 0 and less than or equal to
++     * EphemeralType.MAX_TTL.
++     *
++     * @param ttl the ttl
++     * @return this for chaining
++     */
 +    CreateBuilderMain withTtl(long ttl);
 +}

http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --cc curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index b3817bd,bbb98ea..b7d68e8
--- 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
@@@ -41,7 -41,7 +41,7 @@@ import java.util.concurrent.Callable
  import java.util.concurrent.Executor;
  import java.util.concurrent.atomic.AtomicBoolean;
  
- class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, BackgroundOperation<PathAndBytes>, ErrorListenerPathAndBytesable<String>
 -public class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndBytes>, ErrorListenerPathAndBytesable<String>
++public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, BackgroundOperation<PathAndBytes>, ErrorListenerPathAndBytesable<String>
  {
      private final CuratorFrameworkImpl client;
      private CreateMode createMode;
@@@ -75,11 -74,25 +75,27 @@@
          setDataIfExists = false;
          protectedId = null;
          storingStat = null;
 +        ttl = -1;
      }
  
 -    public CreateBuilderImpl(CuratorFrameworkImpl client, CreateMode createMode, Backgrounding backgrounding, boolean createParentsIfNeeded, boolean createParentsAsContainers, boolean doProtected, boolean compress, boolean setDataIfExists, List<ACL> aclList, Stat storingStat)
++    public CreateBuilderImpl(CuratorFrameworkImpl client, CreateMode createMode, Backgrounding backgrounding, boolean createParentsIfNeeded, boolean createParentsAsContainers, boolean doProtected, boolean compress, boolean setDataIfExists, List<ACL> aclList, Stat storingStat, long ttl)
+     {
+         this.client = client;
+         this.createMode = createMode;
+         this.backgrounding = backgrounding;
+         this.createParentsIfNeeded = createParentsIfNeeded;
+         this.createParentsAsContainers = createParentsAsContainers;
+         this.doProtected = doProtected;
+         this.compress = compress;
+         this.setDataIfExists = setDataIfExists;
+         protectedId = null;
+         this.acling = new ACLing(client.getAclProvider(), aclList);
+         this.storingStat = storingStat;
++        this.ttl = ttl;
+     }
+ 
      @Override
 -    public CreateBuilderMain orSetData()
 +    public CreateBuilder2 orSetData()
      {
          setDataIfExists = true;
          return this;

http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
----------------------------------------------------------------------
diff --cc curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
index e956266,b88a548..a45d0a8
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
@@@ -66,7 -65,6 +66,7 @@@ public class PersistentNode implements 
      private final AtomicReference<String> nodePath = new AtomicReference<String>(null);
      private final String basePath;
      private final CreateMode mode;
++    private final long ttl;
      private final AtomicReference<byte[]> data = new AtomicReference<byte[]>();
      private final AtomicReference<State> state = new AtomicReference<State>(State.LATENT);
      private final AtomicBoolean authFailure = new AtomicBoolean(false);
@@@ -182,6 -173,6 +188,7 @@@
          this.client = Preconditions.checkNotNull(givenClient, "client cannot be null").newWatcherRemoveCuratorFramework();
          this.basePath = PathUtils.validatePath(basePath);
          this.mode = Preconditions.checkNotNull(mode, "mode cannot be null");
++        this.ttl = ttl;
          final byte[] data = Preconditions.checkNotNull(initData, "data cannot be null");
  
          backgroundCallback = new BackgroundCallback()
@@@ -426,7 -415,17 +431,18 @@@
          {
              String existingPath = nodePath.get();
              String createPath = (existingPath != null && !useProtection) ? existingPath : basePath;
-             createMethod.withMode(getCreateMode(existingPath != null)).inBackground(backgroundCallback).forPath(createPath, data.get());
+ 
+             CreateModable<ACLBackgroundPathAndBytesable<String>> localCreateMethod = createMethod.get();
+             if ( localCreateMethod == null )
+             {
 -                CreateModable<ACLBackgroundPathAndBytesable<String>> tempCreateMethod = useProtection ? client.create().creatingParentContainersIfNeeded().withProtection() : client.create().creatingParentContainersIfNeeded();
++                CreateBuilderMain createBuilder = mode.isTTL() ? client.create().withTtl(ttl) : client.create();
++                CreateModable<ACLBackgroundPathAndBytesable<String>> tempCreateMethod = useProtection ? createBuilder.creatingParentContainersIfNeeded().withProtection() : createBuilder.creatingParentContainersIfNeeded();
+                 if ( createMethod.compareAndSet(null, tempCreateMethod) )
+                 {
+                     localCreateMethod = tempCreateMethod;
+                 }
+             }
+             localCreateMethod.withMode(getCreateMode(existingPath != null)).inBackground(backgroundCallback).forPath(createPath, data.get());
          }
          catch ( Exception e )
          {

http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncCreateBuilder.java
----------------------------------------------------------------------
diff --cc curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncCreateBuilder.java
index 0000000,6f077bb..e5f2d8c
mode 000000,100644..100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncCreateBuilder.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncCreateBuilder.java
@@@ -1,0 -1,115 +1,144 @@@
+ /**
+  * 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.x.async.api;
+ 
+ import org.apache.curator.x.async.AsyncStage;
+ import org.apache.zookeeper.CreateMode;
+ import org.apache.zookeeper.data.ACL;
+ import org.apache.zookeeper.data.Stat;
+ import java.util.List;
+ import java.util.Set;
+ 
+ /**
+  * Builder for ZNode creates
+  */
+ public interface AsyncCreateBuilder extends AsyncPathAndBytesable<AsyncStage<String>>
+ {
+     /**
+      * Have the operation fill the provided stat object
+      *
+      * @param stat the stat to have filled in
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> storingStatIn(Stat stat);
+ 
+     /**
+      * Use the given create mode. The default is {@link org.apache.zookeeper.CreateMode#PERSISTENT}
+      *
+      * @param createMode mode to use
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withMode(CreateMode createMode);
+ 
+     /**
+      * Set an ACL list (default is {@link org.apache.zookeeper.ZooDefs.Ids#OPEN_ACL_UNSAFE})
+      *
+      * @param aclList the ACL list to use
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withACL(List<ACL> aclList);
+ 
+     /**
++     * Specify a TTL when mode is {@link org.apache.zookeeper.CreateMode#PERSISTENT_WITH_TTL} or
++     * {@link org.apache.zookeeper.CreateMode#PERSISTENT_SEQUENTIAL_WITH_TTL}. If
++     * the znode has not been modified within the given TTL, it will be deleted once it has no
++     * children. The TTL unit is milliseconds and must be greater than 0 and less than or equal to
++     * EphemeralType.MAX_TTL.
++     *
++     * @param ttl the ttl
++     * @return this for chaining
++     */
++    AsyncPathAndBytesable<AsyncStage<String>> withTtl(long ttl);
++
++    /**
+      * Options to change how the ZNode is created
+      *
+      * @param options options
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options);
+ 
+     /**
+      * set options and ACLs
+      *
+      * @param options options
+      * @param aclList the ACL list to use
+      * @see #withOptions(java.util.Set)
+      * @see #withACL(java.util.List)
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, List<ACL> aclList);
+ 
+     /**
+      * set options, mode and ACLs
+      *
+      * @param options options
+      * @param createMode mode to use
+      * @param aclList the ACL list to use
+      * @see #withACL(java.util.List)
+      * @see #withOptions(java.util.Set)
+      * @see #withMode(org.apache.zookeeper.CreateMode)
+      * @see #withACL(java.util.List)
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode, List<ACL> aclList);
+ 
+     /**
+      * set options and mode
+      *
+      * @param options options
+      * @param createMode mode to use
+      * @see #withOptions(java.util.Set)
+      * @see #withMode(org.apache.zookeeper.CreateMode)
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode);
+ 
+     /**
+      * set options, mode, ACLs, and stat
+      *
+      * @param options options
+      * @param createMode mode to use
+      * @param aclList the ACL list to use
+      * @param stat the stat to have filled in
+      * @see #withOptions(java.util.Set)
+      * @see #withMode(org.apache.zookeeper.CreateMode)
+      * @see #withACL(java.util.List)
+      * @see #storingStatIn(org.apache.zookeeper.data.Stat)
+      * @return this
+      */
+     AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode, List<ACL> aclList, Stat stat);
++
++    /**
++     * set options, mode, ACLs, and stat
++     *
++     * @param options options
++     * @param createMode mode to use
++     * @param aclList the ACL list to use
++     * @param stat the stat to have filled in
++     * @param ttl the ttl or 0
++     * @see #withOptions(java.util.Set)
++     * @see #withMode(org.apache.zookeeper.CreateMode)
++     * @see #withACL(java.util.List)
++     * @see #storingStatIn(org.apache.zookeeper.data.Stat)
++     * @see #withTtl(long)
++     * @return this
++     */
++    AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode, List<ACL> aclList, Stat stat, long ttl);
+ }

http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCreateBuilderImpl.java
----------------------------------------------------------------------
diff --cc curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCreateBuilderImpl.java
index 0000000,b2b9000..e8b1d30
mode 000000,100644..100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCreateBuilderImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCreateBuilderImpl.java
@@@ -1,0 -1,144 +1,164 @@@
+ /**
+  * 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.x.async.details;
+ 
+ import org.apache.curator.framework.imps.CreateBuilderImpl;
+ import org.apache.curator.framework.imps.CuratorFrameworkImpl;
+ import org.apache.curator.x.async.AsyncStage;
+ import org.apache.curator.x.async.api.AsyncCreateBuilder;
+ import org.apache.curator.x.async.api.AsyncPathAndBytesable;
+ import org.apache.curator.x.async.api.CreateOption;
+ import org.apache.zookeeper.CreateMode;
+ import org.apache.zookeeper.data.ACL;
+ import org.apache.zookeeper.data.Stat;
+ import java.util.Collections;
+ import java.util.List;
+ import java.util.Objects;
+ import java.util.Set;
+ 
+ import static org.apache.curator.x.async.details.BackgroundProcs.nameProc;
+ import static org.apache.curator.x.async.details.BackgroundProcs.safeCall;
+ 
+ class AsyncCreateBuilderImpl implements AsyncCreateBuilder
+ {
+     private final CuratorFrameworkImpl client;
+     private final Filters filters;
+     private CreateMode createMode = CreateMode.PERSISTENT;
+     private List<ACL> aclList = null;
+     private Set<CreateOption> options = Collections.emptySet();
+     private Stat stat = null;
++    private long ttl = -1;
+ 
+     AsyncCreateBuilderImpl(CuratorFrameworkImpl client, Filters filters)
+     {
+         this.client = client;
+         this.filters = filters;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> storingStatIn(Stat stat)
+     {
+         this.stat = stat;
+         return this;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withMode(CreateMode createMode)
+     {
+         this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
+         return this;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withACL(List<ACL> aclList)
+     {
+         this.aclList = aclList;
+         return this;
+     }
+ 
+     @Override
++    public AsyncPathAndBytesable<AsyncStage<String>> withTtl(long ttl)
++    {
++        this.ttl = ttl;
++        return this;
++    }
++
++    @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options)
+     {
+         this.options = Objects.requireNonNull(options, "options cannot be null");
+         return this;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, List<ACL> aclList)
+     {
+         this.options = Objects.requireNonNull(options, "options cannot be null");
+         this.aclList = aclList;
+         return this;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode, List<ACL> aclList)
+     {
+         this.options = Objects.requireNonNull(options, "options cannot be null");
+         this.aclList = aclList;
+         this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
+         return this;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode)
+     {
+         this.options = Objects.requireNonNull(options, "options cannot be null");
+         this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
+         return this;
+     }
+ 
+     @Override
+     public AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode, List<ACL> aclList, Stat stat)
+     {
+         this.options = Objects.requireNonNull(options, "options cannot be null");
+         this.aclList = aclList;
+         this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
+         this.stat = stat;
+         return this;
+     }
+ 
+     @Override
++    public AsyncPathAndBytesable<AsyncStage<String>> withOptions(Set<CreateOption> options, CreateMode createMode, List<ACL> aclList, Stat stat, long ttl)
++    {
++        this.options = Objects.requireNonNull(options, "options cannot be null");
++        this.aclList = aclList;
++        this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
++        this.stat = stat;
++        this.ttl = ttl;
++        return this;
++    }
++
++    @Override
+     public AsyncStage<String> forPath(String path)
+     {
+         return internalForPath(path, null, false);
+     }
+ 
+     @Override
+     public AsyncStage<String> forPath(String path, byte[] data)
+     {
+         return internalForPath(path, data, true);
+     }
+ 
+     private AsyncStage<String> internalForPath(String path, byte[] data, boolean useData)
+     {
+         BuilderCommon<String> common = new BuilderCommon<>(filters, nameProc);
+         CreateBuilderImpl builder = new CreateBuilderImpl(client,
+             createMode,
+             common.backgrounding,
+             options.contains(CreateOption.createParentsIfNeeded) || options.contains(CreateOption.createParentsAsContainers),
+             options.contains(CreateOption.createParentsAsContainers),
+             options.contains(CreateOption.doProtected),
+             options.contains(CreateOption.compress),
+             options.contains(CreateOption.setDataIfExists),
+             aclList,
 -            stat
++            stat,
++            ttl
+         );
+         return safeCall(common.internalCallback, () -> useData ? builder.forPath(path, data) : builder.forPath(path));
+     }
+ }

http://git-wip-us.apache.org/repos/asf/curator/blob/1826b66d/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 4027ae7,34e724c..764ef2c
--- a/pom.xml
+++ b/pom.xml
@@@ -59,9 -59,9 +59,9 @@@
          <jdk-version>1.7</jdk-version>
  
          <!-- versions -->
 -        <zookeeper-version>3.5.1-alpha</zookeeper-version>
 +        <zookeeper-version>3.5.3-beta</zookeeper-version>
          <maven-project-info-reports-plugin-version>2.7</maven-project-info-reports-plugin-version>
-         <maven-bundle-plugin-version>2.3.7</maven-bundle-plugin-version>
+         <maven-bundle-plugin-version>3.2.0</maven-bundle-plugin-version>
          <maven-javadoc-plugin-version>2.10.3</maven-javadoc-plugin-version>
          <doxia-module-confluence-version>1.6</doxia-module-confluence-version>
          <maven-license-plugin-version>1.9.0</maven-license-plugin-version>


[04/20] curator git commit: make sure you can use TTLs with orSetData

Posted by ra...@apache.org.
make sure you can use TTLs with orSetData


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

Branch: refs/heads/CURATOR-397
Commit: d0299661e87418aeb97ea50658a927c3647b2112
Parents: 44eb60c
Author: randgalt <ra...@apache.org>
Authored: Thu Dec 8 14:14:31 2016 +0100
Committer: randgalt <ra...@apache.org>
Committed: Thu Dec 8 14:14:31 2016 +0100

----------------------------------------------------------------------
 .../curator/framework/api/CreateBuilder.java    |  2 +-
 .../curator/framework/api/CreateBuilder2.java   | 24 ++++++++++++++++++++
 .../framework/imps/CreateBuilderImpl.java       |  4 ++--
 3 files changed, 27 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/d0299661/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 8c98e0d..431a945 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
@@ -22,5 +22,5 @@ public interface CreateBuilder extends CreateBuilderMain
 {
     CreateBuilderMain withTtl(long ttl);
 
-    CreateBuilderMain orSetData();
+    CreateBuilder2 orSetData();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/d0299661/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
new file mode 100644
index 0000000..7e5c89a
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder2.java
@@ -0,0 +1,24 @@
+/**
+ * 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 CreateBuilder2 extends CreateBuilderMain
+{
+    CreateBuilderMain withTtl(long ttl);
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/d0299661/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 cc2b541..b3817bd 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
@@ -41,7 +41,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndBytes>, ErrorListenerPathAndBytesable<String>
+class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, BackgroundOperation<PathAndBytes>, ErrorListenerPathAndBytesable<String>
 {
     private final CuratorFrameworkImpl client;
     private CreateMode createMode;
@@ -79,7 +79,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     }
 
     @Override
-    public CreateBuilderMain orSetData()
+    public CreateBuilder2 orSetData()
     {
         setDataIfExists = true;
         return this;


[17/20] curator git commit: intermediate AsyncTransactionCreateBuilder DSL methods should return AsyncPathAndBytesable not AsynccPathable

Posted by ra...@apache.org.
intermediate AsyncTransactionCreateBuilder DSL methods should return AsyncPathAndBytesable not AsynccPathable


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

Branch: refs/heads/CURATOR-397
Commit: 35f5d274e572b91332ab04f3d03f79938cd216f3
Parents: 086c5b4
Author: randgalt <ra...@apache.org>
Authored: Sat Apr 29 14:10:25 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Sat Apr 29 14:10:25 2017 -0500

----------------------------------------------------------------------
 .../curator/x/async/api/AsyncTransactionCreateBuilder.java   | 8 ++++----
 .../curator/x/async/details/AsyncTransactionOpImpl.java      | 8 ++++----
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/35f5d274/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionCreateBuilder.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionCreateBuilder.java b/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionCreateBuilder.java
index 439db81..81da5c0 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionCreateBuilder.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionCreateBuilder.java
@@ -34,7 +34,7 @@ public interface AsyncTransactionCreateBuilder extends AsyncPathAndBytesable<Cur
      * @param createMode mode
      * @return this
      */
-    AsyncPathable<CuratorOp> withMode(CreateMode createMode);
+    AsyncPathAndBytesable<CuratorOp> withMode(CreateMode createMode);
 
     /**
      * Set an ACL list (default is {@link org.apache.zookeeper.ZooDefs.Ids#OPEN_ACL_UNSAFE})
@@ -42,14 +42,14 @@ public interface AsyncTransactionCreateBuilder extends AsyncPathAndBytesable<Cur
      * @param aclList the ACL list to use
      * @return this
      */
-    AsyncPathable<CuratorOp> withACL(List<ACL> aclList);
+    AsyncPathAndBytesable<CuratorOp> withACL(List<ACL> aclList);
 
     /**
      * Cause the data to be compressed using the configured compression provider
      *
      * @return this
      */
-    AsyncPathable<CuratorOp> compressed();
+    AsyncPathAndBytesable<CuratorOp> compressed();
 
     /**
      * Specify mode, acl list and compression
@@ -62,5 +62,5 @@ public interface AsyncTransactionCreateBuilder extends AsyncPathAndBytesable<Cur
      * @see #compressed()
      * @return this
      */
-    AsyncPathable<CuratorOp> withOptions(CreateMode createMode, List<ACL> aclList, boolean compressed);
+    AsyncPathAndBytesable<CuratorOp> withOptions(CreateMode createMode, List<ACL> aclList, boolean compressed);
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/35f5d274/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncTransactionOpImpl.java
----------------------------------------------------------------------
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncTransactionOpImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncTransactionOpImpl.java
index 89f0a22..0be720f 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncTransactionOpImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncTransactionOpImpl.java
@@ -56,28 +56,28 @@ class AsyncTransactionOpImpl implements AsyncTransactionOp
             private boolean compressed = false;
 
             @Override
-            public AsyncPathable<CuratorOp> withMode(CreateMode createMode)
+            public AsyncPathAndBytesable<CuratorOp> withMode(CreateMode createMode)
             {
                 this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
                 return this;
             }
 
             @Override
-            public AsyncPathable<CuratorOp> withACL(List<ACL> aclList)
+            public AsyncPathAndBytesable<CuratorOp> withACL(List<ACL> aclList)
             {
                 this.aclList = aclList;
                 return this;
             }
 
             @Override
-            public AsyncPathable<CuratorOp> compressed()
+            public AsyncPathAndBytesable<CuratorOp> compressed()
             {
                 compressed = true;
                 return this;
             }
 
             @Override
-            public AsyncPathable<CuratorOp> withOptions(CreateMode createMode, List<ACL> aclList, boolean compressed)
+            public AsyncPathAndBytesable<CuratorOp> withOptions(CreateMode createMode, List<ACL> aclList, boolean compressed)
             {
                 this.createMode = Objects.requireNonNull(createMode, "createMode cannot be null");
                 this.aclList = aclList;


[13/20] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-351

Posted by ra...@apache.org.
Merge branch 'CURATOR-3.0' into CURATOR-351


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

Branch: refs/heads/CURATOR-397
Commit: e3fec5bc7fcd1091c3f07317039d1d5ed20f2d66
Parents: f89c1ea 1939389
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 17 13:27:13 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 17 13:27:13 2017 -0500

----------------------------------------------------------------------
 doap.rdf | 4 +++-
 pom.xml  | 2 +-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/e3fec5bc/pom.xml
----------------------------------------------------------------------


[16/20] curator git commit: added missing doc for executorService

Posted by ra...@apache.org.
added missing doc for executorService


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

Branch: refs/heads/CURATOR-397
Commit: 086c5b4152cdb045e7bb4818d9cef925e88ed313
Parents: b84d351
Author: randgalt <ra...@apache.org>
Authored: Tue Apr 25 09:49:47 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Tue Apr 25 09:49:47 2017 -0500

----------------------------------------------------------------------
 .../apache/curator/framework/recipes/nodes/PersistentTtlNode.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/086c5b41/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
index e1e8bf4..91f5f71 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
@@ -81,6 +81,7 @@ public class PersistentTtlNode implements Closeable
 
     /**
      * @param client the client
+     * @param executorService  ExecutorService to use for background thread. This service should be single threaded, otherwise you may see inconsistent results.
      * @param path path for the parent ZNode
      * @param ttlMs max ttl for the node in milliseconds
      * @param initData data for the node


[12/20] curator git commit: use local var data

Posted by ra...@apache.org.
use local var data


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

Branch: refs/heads/CURATOR-397
Commit: f89c1ea5336bbde932638a2a28d678b231a2e49b
Parents: 1826b66
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 3 11:31:49 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 3 11:31:49 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/curator/framework/imps/CreateBuilderImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/f89c1ea5/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 b7d68e8..718dded 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
@@ -567,7 +567,7 @@ public class CreateBuilderImpl implements CreateBuilder, CreateBuilder2, Backgro
             client.getZooKeeper().create
             (
                 operationAndData.getData().getPath(),
-                operationAndData.getData().getData(),
+                data,
                 acling.getAclList(operationAndData.getData().getPath()),
                 createMode,
                 new AsyncCallback.Create2Callback() {


[14/20] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-351

Posted by ra...@apache.org.
Merge branch 'CURATOR-3.0' into CURATOR-351


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

Branch: refs/heads/CURATOR-397
Commit: da7f18c618eb51e6aaaff7b10f79240cefaf0183
Parents: e3fec5b 00ffe77
Author: randgalt <ra...@apache.org>
Authored: Mon Apr 17 19:08:15 2017 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Apr 17 19:08:15 2017 -0500

----------------------------------------------------------------------
 .../apache/curator/framework/recipes/nodes/PersistentNode.java | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/da7f18c6/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
----------------------------------------------------------------------
diff --cc curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
index a45d0a8,6c0c22c..0cda2a1
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
@@@ -435,12 -419,9 +435,10 @@@ public class PersistentNode implements 
              CreateModable<ACLBackgroundPathAndBytesable<String>> localCreateMethod = createMethod.get();
              if ( localCreateMethod == null )
              {
 -                CreateModable<ACLBackgroundPathAndBytesable<String>> tempCreateMethod = useProtection ? client.create().creatingParentContainersIfNeeded().withProtection() : client.create().creatingParentContainersIfNeeded();
 +                CreateBuilderMain createBuilder = mode.isTTL() ? client.create().withTtl(ttl) : client.create();
 +                CreateModable<ACLBackgroundPathAndBytesable<String>> tempCreateMethod = useProtection ? createBuilder.creatingParentContainersIfNeeded().withProtection() : createBuilder.creatingParentContainersIfNeeded();
-                 if ( createMethod.compareAndSet(null, tempCreateMethod) )
-                 {
-                     localCreateMethod = tempCreateMethod;
-                 }
+                 createMethod.compareAndSet(null, tempCreateMethod);
+                 localCreateMethod = createMethod.get();
              }
              localCreateMethod.withMode(getCreateMode(existingPath != null)).inBackground(backgroundCallback).forPath(createPath, data.get());
          }


[09/20] curator git commit: add licenses

Posted by ra...@apache.org.
add licenses


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

Branch: refs/heads/CURATOR-397
Commit: 9654778a6a5c329d1f36ebdea32d68290850f96d
Parents: a9cbdd3
Author: randgalt <ra...@apache.org>
Authored: Sat Dec 10 10:14:09 2016 -0500
Committer: randgalt <ra...@apache.org>
Committed: Sat Dec 10 10:14:09 2016 -0500

----------------------------------------------------------------------
 .../recipes/nodes/PersistentTtlNode.java          | 18 ++++++++++++++++++
 .../recipes/nodes/TestPersistentTtlNode.java      | 18 ++++++++++++++++++
 2 files changed, 36 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/9654778a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
index 3eda9e2..e1e8bf4 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentTtlNode.java
@@ -1,3 +1,21 @@
+/**
+ * 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.nodes;
 
 import org.apache.curator.framework.CuratorFramework;

http://git-wip-us.apache.org/repos/asf/curator/blob/9654778a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
index e522103..2224bbf 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentTtlNode.java
@@ -1,3 +1,21 @@
+/**
+ * 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.nodes;
 
 import org.apache.curator.framework.CuratorFramework;


[03/20] curator git commit: TTL node support for PersistentNode

Posted by ra...@apache.org.
TTL node support for PersistentNode


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

Branch: refs/heads/CURATOR-397
Commit: 44eb60c92fd29c440b0a17c85fbe0f6ec8fbda6e
Parents: 786d695
Author: randgalt <ra...@apache.org>
Authored: Wed Dec 7 15:51:40 2016 +0100
Committer: randgalt <ra...@apache.org>
Committed: Wed Dec 7 15:51:40 2016 +0100

----------------------------------------------------------------------
 .../framework/recipes/nodes/PersistentNode.java | 46 ++++++++++++++------
 1 file changed, 33 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/44eb60c9/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
index 8375967..eaa91b7 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
@@ -26,6 +26,7 @@ import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.ACLBackgroundPathAndBytesable;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CreateBuilder;
+import org.apache.curator.framework.api.CreateBuilderMain;
 import org.apache.curator.framework.api.CreateModable;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorWatcher;
@@ -164,6 +165,19 @@ public class PersistentNode implements Closeable
      */
     public PersistentNode(CuratorFramework givenClient, final CreateMode mode, boolean useProtection, final String basePath, byte[] initData)
     {
+        this(givenClient, mode, useProtection, basePath, initData, -1);
+    }
+
+    /**
+     * @param givenClient        client instance
+     * @param mode          creation mode
+     * @param useProtection if true, call {@link CreateBuilder#withProtection()}
+     * @param basePath the base path for the node
+     * @param initData data for the node
+     * @param ttl for ttl modes, the ttl to use
+     */
+    public PersistentNode(CuratorFramework givenClient, final CreateMode mode, boolean useProtection, final String basePath, byte[] initData, long ttl)
+    {
         this.useProtection = useProtection;
         this.client = Preconditions.checkNotNull(givenClient, "client cannot be null").newWatcherRemoveCuratorFramework();
         this.basePath = PathUtils.validatePath(basePath);
@@ -186,7 +200,8 @@ public class PersistentNode implements Closeable
             }
         };
 
-        createMethod = useProtection ? client.create().creatingParentContainersIfNeeded().withProtection() : client.create().creatingParentContainersIfNeeded();
+        CreateBuilderMain createBuilder = mode.isTTL() ? client.create().withTtl(ttl) : client.create();
+        createMethod = useProtection ? createBuilder.creatingParentContainersIfNeeded().withProtection() : createBuilder.creatingParentContainersIfNeeded();
         this.data.set(Arrays.copyOf(data, data.length));
     }
 
@@ -426,20 +441,25 @@ public class PersistentNode implements Closeable
         {
             switch ( mode )
             {
-            default:
-            {
-                break;
-            }
+                default:
+                {
+                    break;
+                }
 
-            case EPHEMERAL_SEQUENTIAL:
-            {
-                return CreateMode.EPHEMERAL;    // protection case - node already set
-            }
+                case EPHEMERAL_SEQUENTIAL:
+                {
+                    return CreateMode.EPHEMERAL;    // protection case - node already set
+                }
 
-            case PERSISTENT_SEQUENTIAL:
-            {
-                return CreateMode.PERSISTENT;    // protection case - node already set
-            }
+                case PERSISTENT_SEQUENTIAL:
+                {
+                    return CreateMode.PERSISTENT;    // protection case - node already set
+                }
+
+                case PERSISTENT_SEQUENTIAL_WITH_TTL:
+                {
+                    return CreateMode.PERSISTENT_WITH_TTL;    // protection case - node already set
+                }
             }
         }
         return mode;