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 2019/11/03 04:20:45 UTC

[curator] branch curator-v2-recipe created (now f2b0a78)

This is an automated email from the ASF dual-hosted git repository.

randgalt pushed a change to branch curator-v2-recipe
in repository https://gitbox.apache.org/repos/asf/curator.git.


      at f2b0a78  wip - create new recipes that use persistent/recurisive watches. A PersistentWatch and a new replacement cache.

This branch includes the following new commits:

     new 5e8ccf1  wip - new module that will be used to support features in ZK 3.6+ while maintaining background compatability with previous versions of ZK in the other modules. At some point in the future, this can be merged into the main modules and then removed
     new d127cc4  wip - new module that will be used to support features in ZK 3.6+ while maintaining background compatability with previous versions of ZK in the other modules. At some point in the future, this can be merged into the main modules and then removed
     new f2b0a78  wip - create new recipes that use persistent/recurisive watches. A PersistentWatch and a new replacement cache.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[curator] 02/03: wip - new module that will be used to support features in ZK 3.6+ while maintaining background compatability with previous versions of ZK in the other modules. At some point in the future, this can be merged into the main modules and then removed

Posted by ra...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

randgalt pushed a commit to branch curator-v2-recipe
in repository https://gitbox.apache.org/repos/asf/curator.git

commit d127cc472f3c0fa289d266324d046e3f48f0bc10
Author: randgalt <ra...@apache.org>
AuthorDate: Sat Nov 2 11:40:44 2019 -0500

    wip - new module that will be used to support features in ZK 3.6+ while maintaining background compatability with previous versions of ZK in the other modules. At some point in the future, this can be merged into the main modules and then removed
---
 .../framework/v2/TestAsyncCuratorFrameworkV2.java  | 57 ++++++++++++++++++++++
 .../curator/framework/v2/TestFrameworkV2.java      | 57 ++++++++++++++++++++++
 2 files changed, 114 insertions(+)

diff --git a/curator-v2/src/test/java/org/apache/curator/framework/v2/TestAsyncCuratorFrameworkV2.java b/curator-v2/src/test/java/org/apache/curator/framework/v2/TestAsyncCuratorFrameworkV2.java
new file mode 100644
index 0000000..4785787
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/framework/v2/TestAsyncCuratorFrameworkV2.java
@@ -0,0 +1,57 @@
+/**
+ * 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.v2;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.curator.v2.AsyncCuratorFrameworkV2;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.concurrent.CountDownLatch;
+
+public class TestAsyncCuratorFrameworkV2 extends CuratorTestBase
+{
+    @Test
+    public void testPersistentRecursiveWatch() throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
+        {
+            AsyncCuratorFrameworkV2 async = AsyncCuratorFrameworkV2.wrap(client);
+
+            client.start();
+            client.blockUntilConnected();
+
+            CountDownLatch latch = new CountDownLatch(5);
+            Watcher watcher = event -> latch.countDown();
+            async.addWatch().withMode(AddWatchMode.PERSISTENT_RECURSIVE).usingWatcher(watcher).forPath("/test").toCompletableFuture().get();
+
+            client.create().forPath("/test");
+            client.create().forPath("/test/a");
+            client.create().forPath("/test/a/b");
+            client.create().forPath("/test/a/b/c");
+            client.create().forPath("/test/a/b/c/d");
+
+            Assert.assertTrue(timing.awaitLatch(latch));
+        }
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/framework/v2/TestFrameworkV2.java b/curator-v2/src/test/java/org/apache/curator/framework/v2/TestFrameworkV2.java
new file mode 100644
index 0000000..f00dab3
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/framework/v2/TestFrameworkV2.java
@@ -0,0 +1,57 @@
+/**
+ * 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.v2;
+
+import org.apache.curator.v2.CuratorFrameworkV2;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.curator.v2.CuratorFrameworkV2.wrap;
+
+public class TestFrameworkV2 extends CuratorTestBase
+{
+    @Test
+    public void testPersistentRecursiveWatch() throws Exception
+    {
+        try ( CuratorFrameworkV2 client = wrap(CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1))) )
+        {
+            client.start();
+            client.blockUntilConnected();
+
+            CountDownLatch latch = new CountDownLatch(5);
+            Watcher watcher = event -> latch.countDown();
+            client.watches().add().withMode(AddWatchMode.PERSISTENT_RECURSIVE).usingWatcher(watcher).forPath("/test");
+
+            client.create().forPath("/test");
+            client.create().forPath("/test/a");
+            client.create().forPath("/test/a/b");
+            client.create().forPath("/test/a/b/c");
+            client.create().forPath("/test/a/b/c/d");
+
+            Assert.assertTrue(timing.awaitLatch(latch));
+        }
+    }
+}


[curator] 03/03: wip - create new recipes that use persistent/recurisive watches. A PersistentWatch and a new replacement cache.

Posted by ra...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

randgalt pushed a commit to branch curator-v2-recipe
in repository https://gitbox.apache.org/repos/asf/curator.git

commit f2b0a7883c375458331bceef51e762cbcacf11fe
Author: randgalt <ra...@apache.org>
AuthorDate: Sat Nov 2 23:16:18 2019 -0500

    wip - create new recipes that use persistent/recurisive watches. A PersistentWatch and a new replacement cache.
---
 curator-examples/pom.xml                           |   2 +-
 .../src/main/java/cache/CuratorCacheExample.java   | 102 ++++++
 .../src/site/confluence/index.confluence           |   2 +-
 .../src/site/confluence/curator-cache.confluence   |  36 ++
 .../src/site/confluence/index.confluence           |   7 +-
 .../framework/recipes/cache/BaseTestTreeCache.java |  12 +-
 curator-v2/pom.xml                                 |  23 +-
 .../curator/v2/recipes/cache/CuratorCache.java     | 140 ++++++++
 .../v2/recipes/cache/CuratorCacheBuilder.java      |  63 ++++
 .../v2/recipes/cache/CuratorCacheBuilderImpl.java  |  80 +++++
 .../curator/v2/recipes/cache/CuratorCacheImpl.java | 304 +++++++++++++++++
 .../v2/recipes/cache/CuratorCacheListener.java     |  80 +++++
 .../recipes/cache/CuratorCacheListenerBuilder.java | 133 ++++++++
 .../cache/CuratorCacheListenerBuilderImpl.java     | 165 +++++++++
 .../v2/recipes/cache/CuratorCacheStorage.java      | 115 +++++++
 .../v2/recipes/cache/NodeCacheListenerWrapper.java |  46 +++
 .../cache/PathChildrenCacheListenerWrapper.java    |  81 +++++
 .../recipes/cache/StandardCuratorCacheStorage.java |  87 +++++
 .../v2/recipes/cache/TreeCacheListenerWrapper.java |  81 +++++
 .../v2/recipes/watch/PersistentWatcher.java        | 180 ++++++++++
 .../framework/v2/TestAsyncCuratorFrameworkV2.java  |  57 ----
 .../curator/framework/v2/TestFrameworkV2.java      |  57 ----
 .../curator/v2/recipes/cache/TestCuratorCache.java | 248 ++++++++++++++
 .../recipes/cache/TestCuratorCacheConsistency.java | 373 +++++++++++++++++++++
 .../cache/TestCuratorCacheEventOrdering.java       |  50 +++
 .../v2/recipes/cache/TestCuratorCacheWrappers.java | 166 +++++++++
 .../v2/recipes/cache/TestWrappedNodeCache.java     | 178 ++++++++++
 .../v2/recipes/watch/TestPersistentWatcher.java    | 104 ++++++
 pom.xml                                            |   6 +
 29 files changed, 2841 insertions(+), 137 deletions(-)

diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index 7d6ffb9..5b27cd6 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -36,7 +36,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.curator</groupId>
-            <artifactId>curator-recipes</artifactId>
+            <artifactId>curator-v2</artifactId>
         </dependency>
 
         <dependency>
diff --git a/curator-examples/src/main/java/cache/CuratorCacheExample.java b/curator-examples/src/main/java/cache/CuratorCacheExample.java
new file mode 100644
index 0000000..4b7d282
--- /dev/null
+++ b/curator-examples/src/main/java/cache/CuratorCacheExample.java
@@ -0,0 +1,102 @@
+/**
+ * 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 cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.TestingServer;
+import org.apache.curator.v2.recipes.cache.CuratorCache;
+import org.apache.curator.v2.recipes.cache.CuratorCacheListener;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Very simple example of creating a CuratorCache that listens to events and logs the changes
+ * to standard out. A loop of random changes is run to exercise the cache.
+ */
+public class CuratorCacheExample
+{
+    private static final String PATH = "/example/cache";
+
+    public static void main(String[] args) throws Exception
+    {
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+        try (TestingServer server = new TestingServer())
+        {
+            try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new ExponentialBackoffRetry(1000, 3)))
+            {
+                client.start();
+                try (CuratorCache cache = CuratorCache.build(client, PATH))
+                {
+                    // there are several ways to set a listener on a CuratorCache. You can watch for individual events
+                    // or for all events. Here, we'll watch all events and merely log that they occurred
+                    CuratorCacheListener listener = (type, oldNode, node) -> {
+                        switch ( type )
+                        {
+                            case NODE_CREATED:
+                                System.out.println(String.format("Node created: [%s]", node));
+                                break;
+
+                            case NODE_CHANGED:
+                                System.out.println(String.format("Node changed. Old: [%s] New: [%s]", oldNode, node));
+                                break;
+
+                            case NODE_DELETED:
+                                System.out.println(String.format("Node deleted. Old value: [%s]", oldNode));
+                                break;
+                        }
+                    };
+
+                    // register the listener
+                    cache.listenable().addListener(listener);
+
+                    // the cache must be started
+                    cache.start();
+
+                    // now randomly create/change/delete nodes
+                    for ( int i = 0; i < 1000; ++i )
+                    {
+                        int depth = random.nextInt(1, 4);
+                        String path = makeRandomPath(random, depth);
+                        if ( random.nextBoolean() )
+                        {
+                            client.create().orSetData().creatingParentsIfNeeded().forPath(path, Long.toString(random.nextLong()).getBytes());
+                        }
+                        else
+                        {
+                            client.delete().quietly().deletingChildrenIfNeeded().forPath(path);
+                        }
+
+                        Thread.sleep(5);
+                    }
+                }
+            }
+        }
+    }
+
+    private static String makeRandomPath(ThreadLocalRandom random, int depth)
+    {
+        if ( depth == 0 )
+        {
+            return PATH;
+        }
+        return makeRandomPath(random, depth - 1) + "/" + random.nextInt(3);
+    }
+}
\ No newline at end of file
diff --git a/curator-examples/src/site/confluence/index.confluence b/curator-examples/src/site/confluence/index.confluence
index f9be506..6c5a30d 100644
--- a/curator-examples/src/site/confluence/index.confluence
+++ b/curator-examples/src/site/confluence/index.confluence
@@ -3,7 +3,7 @@ h1. Examples
 This module contains example usages of various Curator features. Each directory in the module is a separate example.
 
 |/leader|Example leader selector code|
-|/cache|Example PathChildrenCache usage|
+|/cache|Example CuratorCache usage|
 |/locking|Example of using InterProcessMutex|
 |/discovery|Example usage of the Curator's ServiceDiscovery|
 |/framework|A few examples of how to use the CuratorFramework class|
diff --git a/curator-recipes/src/site/confluence/curator-cache.confluence b/curator-recipes/src/site/confluence/curator-cache.confluence
new file mode 100644
index 0000000..2eeb500
--- /dev/null
+++ b/curator-recipes/src/site/confluence/curator-cache.confluence
@@ -0,0 +1,36 @@
+h1. Curator Cache
+
+*Note: * CuratorCache requires ZooKeeper 3.6\+.
+
+h2. Description
+A utility that attempts to keep the data from a node locally cached. Optionally the entire tree of children below the node can also be cached.
+Will respond to update/create/delete events, pull down the data, etc. You can register listeners that will get notified when changes occur.
+
+h2. Participating Classes
+* CuratorCache
+* CuratorCacheListener
+* ChildData
+
+h2. Usage
+h3. Creating a CuratorCache
+{code}
+CuratorCache.build(CuratorFramework client, String path, Options... options)
+
+Parameters:
+client - the client
+path - path to watch
+options - empty or one or more options
+{code}
+
+Note: there is a builder factory available for additional options when building the cache instance. See {{CuratorCacheBuilder}} for details.
+
+h2. General Usage
+The cache must be started by calling {{start()}}. Call {{close()}} when you are through with the cache.
+
+At any time, call {{storage()}} to get the current state cache. You can also register to be notified when a change occurs by calling {{listenable()}} and then registering
+a listener for events.
+
+See the examples for an example usage.
+
+h2. Error Handling
+CuratorCache instances internally monitor connection losses, etc. automatically rebuilding the cache on reconnection.
diff --git a/curator-recipes/src/site/confluence/index.confluence b/curator-recipes/src/site/confluence/index.confluence
index 08ef762..d96b5ce 100644
--- a/curator-recipes/src/site/confluence/index.confluence
+++ b/curator-recipes/src/site/confluence/index.confluence
@@ -24,9 +24,10 @@ regarding "Curator Recipes Own Their ZNode/Paths".
 |[[Distributed Atomic Long|distributed-atomic-long.html]] \- A counter that attempts atomic increments. It first tries using optimistic locking. If that fails, an optional InterProcessMutex is taken. For both optimistic and mutex, a retry policy is used to retry the increment.|
 
 ||Caches||
-|[[Path Cache|path-cache.html]] \- A Path Cache is used to watch a ZNode. Whenever a child is added, updated or removed, the Path Cache will change its state to contain the current set of children, the children's data and the children's state. Path caches in the Curator Framework are provided by the PathChildrenCache class. Changes to the path are passed to registered PathChildrenCacheListener instances.|
-|[[Node Cache|node-cache.html]] \- A utility that attempts to keep the data from a node locally cached. This class will watch the node, respond to update/create/delete events, pull down the data, etc. You can register a listener that will get notified when changes occur.|
-|[[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.|
+|[[Curator Cache|curator-cache.html]] \- A utility that attempts to keep the data from a node locally cached. Optionally the entire tree of children below the node can also be cached. Will respond to update/create/delete events, pull down the data, etc. You can register listeners that will get notified when changes occur.|
+|[[Path Cache|path-cache.html]] \- (For pre-ZooKeeper 3.6.x) A Path Cache is used to watch a ZNode. Whenever a child is added, updated or removed, the Path Cache will change its state to contain the current set of children, the children's data and the children's state. Path caches in the Curator Framework are provided by the PathChildrenCache class. Changes to the path are passed to registered PathChildrenCacheListener instances.|
+|[[Node Cache|node-cache.html]] \- (For pre-ZooKeeper 3.6.x) A utility that attempts to keep the data from a node locally cached. This class will watch the node, respond to update/create/delete events, pull down the data, etc. You can register a listener that will get notified when changes occur.|
+|[[Tree Cache|tree-cache.html]] \- (For pre-ZooKeeper 3.6.x) 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-node.html]] \- A node that attempts to stay present in ZooKeeper, even through connection and session interruptions.|
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java
index 246704f..828c5d3 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java
@@ -46,7 +46,7 @@ public class BaseTestTreeCache extends BaseClassForTests
     /**
      * Automatically records all events into an easily testable event stream.
      */
-    final TreeCacheListener eventListener = new TreeCacheListener()
+    public final TreeCacheListener eventListener = new TreeCacheListener()
     {
         @Override
         public void childEvent(CuratorFramework client, TreeCacheEvent event) throws Exception
@@ -136,7 +136,7 @@ public class BaseTestTreeCache extends BaseClassForTests
     /**
      * Asserts the event queue is empty.
      */
-    void assertNoMoreEvents() throws InterruptedException
+    public void assertNoMoreEvents() throws InterruptedException
     {
         timing.sleepABit();
         Assert.assertTrue(events.isEmpty(), String.format("Expected no events, found %d; first event: %s", events.size(), events.peek()));
@@ -145,7 +145,7 @@ public class BaseTestTreeCache extends BaseClassForTests
     /**
      * Asserts the given event is next in the queue, and consumes it from the queue.
      */
-    TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType) throws InterruptedException
+    public TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType) throws InterruptedException
     {
         return assertEvent(expectedType, null);
     }
@@ -153,7 +153,7 @@ public class BaseTestTreeCache extends BaseClassForTests
     /**
      * Asserts the given event is next in the queue, and consumes it from the queue.
      */
-    TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType, String expectedPath) throws InterruptedException
+    public TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType, String expectedPath) throws InterruptedException
     {
         return assertEvent(expectedType, expectedPath, null);
     }
@@ -161,12 +161,12 @@ public class BaseTestTreeCache extends BaseClassForTests
     /**
      * Asserts the given event is next in the queue, and consumes it from the queue.
      */
-    TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType, String expectedPath, byte[] expectedData) throws InterruptedException
+    public TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType, String expectedPath, byte[] expectedData) throws InterruptedException
     {
         return assertEvent(expectedType, expectedPath, expectedData, false);
     }
 
-    TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType, String expectedPath, byte[] expectedData, boolean ignoreConnectionEvents) throws InterruptedException
+    public TreeCacheEvent assertEvent(TreeCacheEvent.Type expectedType, String expectedPath, byte[] expectedData, boolean ignoreConnectionEvents) throws InterruptedException
     {
         TreeCacheEvent event = events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS);
         Assert.assertNotNull(event, String.format("Expected type: %s, path: %s", expectedType, expectedPath));
diff --git a/curator-v2/pom.xml b/curator-v2/pom.xml
index 0ff537a..bef972d 100644
--- a/curator-v2/pom.xml
+++ b/curator-v2/pom.xml
@@ -27,6 +27,17 @@
 
         <dependency>
             <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
             <artifactId>curator-x-async</artifactId>
             <exclusions>
                 <exclusion>
@@ -67,18 +78,6 @@
 
         <dependency>
             <groupId>org.apache.curator</groupId>
-            <artifactId>curator-recipes</artifactId>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.zookeeper</groupId>
-                    <artifactId>zookeeper</artifactId>
-                </exclusion>
-            </exclusions>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.curator</groupId>
             <artifactId>curator-test</artifactId>
             <exclusions>
                 <exclusion>
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCache.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCache.java
new file mode 100644
index 0000000..8702f21
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCache.java
@@ -0,0 +1,140 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.v2.CuratorFrameworkV2;
+import java.io.Closeable;
+
+/**
+ * <p>
+ *     A utility that attempts to keep the data from a node locally cached. Optionally the entire
+ *     tree of children below the node can also be cached. Will respond to update/create/delete events, pull
+ *     down the data, etc. You can register listeners that will get notified when changes occur.
+ * </p>
+ *
+ * <p>
+ *     <b>IMPORTANT</b> - it's not possible to stay transactionally in sync. Users of this class must
+ *     be prepared for false-positives and false-negatives. Additionally, always use the version number
+ *     when updating data to avoid overwriting another process' change.
+ * </p>
+ */
+public interface CuratorCache extends Closeable
+{
+    /**
+     * cache build options
+     */
+    enum Options
+    {
+        /**
+         * Normally the entire tree of nodes starting at the given node are cached. This option
+         * causes only the given node to be cached (i.e. a single node cache)
+         */
+        SINGLE_NODE_CACHE,
+
+        /**
+         * Decompress data via {@link org.apache.curator.framework.api.GetDataBuilder#decompressed()}
+         */
+        COMPRESSED_DATA,
+
+        /**
+         * Normally, when the cache is closed via {@link CuratorCache#close()}, the storage is cleared
+         * via {@link CuratorCacheStorage#clear()}. This option prevents the storage from being cleared.
+         */
+        DO_NOT_CLEAR_ON_CLOSE
+    }
+
+    /**
+     * Return a Curator Cache for the given path with the given options using a standard storage instance
+     *
+     * @param client Curator client
+     * @param path path to cache
+     * @param options any options
+     * @return cache (note it must be started via {@link #start()}
+     */
+    static CuratorCache build(CuratorFramework client, String path, Options... options)
+    {
+        return builder(client, path).withOptions(options).build();
+    }
+
+    /**
+     * Start a Curator Cache builder
+     *
+     * @param client Curator client
+     * @param path path to cache
+     * @return builder
+     */
+    static CuratorCacheBuilder builder(CuratorFramework client, String path)
+    {
+        return new CuratorCacheBuilderImpl(client, path);
+    }
+
+    /**
+     * Return a Curator Cache for the given path with the given options using a standard storage instance
+     *
+     * @param client Curator client
+     * @param path path to cache
+     * @param options any options
+     * @return cache (note it must be started via {@link #start()}
+     */
+    static CuratorCache build(CuratorFrameworkV2 client, String path, Options... options)
+    {
+        return builder(client, path).withOptions(options).build();
+    }
+
+    /**
+     * Start a Curator Cache builder
+     *
+     * @param client Curator client
+     * @param path path to cache
+     * @return builder
+     */
+    static CuratorCacheBuilder builder(CuratorFrameworkV2 client, String path)
+    {
+        return new CuratorCacheBuilderImpl(client, path);
+    }
+
+    /**
+     * Start the cache. This will cause a complete refresh from the cache's root node and generate
+     * events for all nodes found, etc.
+     */
+    void start();
+
+    /**
+     * Close the cache, stop responding to events, etc.
+     */
+    @Override
+    void close();
+
+    /**
+     * Return the storage instance being used
+     *
+     * @return storage
+     */
+    CuratorCacheStorage storage();
+
+    /**
+     * Return the listener container so that listeners can be registered to be notified of changes to the cache
+     *
+     * @return listener container
+     */
+    Listenable<CuratorCacheListener> listenable();
+}
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheBuilder.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheBuilder.java
new file mode 100644
index 0000000..0590f24
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheBuilder.java
@@ -0,0 +1,63 @@
+/**
+ * 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.v2.recipes.cache;
+
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+
+public interface CuratorCacheBuilder
+{
+    /**
+     * @param options any options
+     * @return this
+     */
+    CuratorCacheBuilder withOptions(CuratorCache.Options... options);
+
+    /**
+     * Alternate storage to use. If not specified, {@link StandardCuratorCacheStorage#standard()} is used
+     *
+     * @param storage storage instance to use
+     * @return this
+     */
+    CuratorCacheBuilder withStorage(CuratorCacheStorage storage);
+
+    /**
+     * By default any unexpected exception is handled by logging the exception. You can change
+     * so that a handler is called instead. Under normal circumstances, this shouldn't be necessary.
+     *
+     * @param exceptionHandler exception handler to use
+     */
+    CuratorCacheBuilder withExceptionHandler(Consumer<Exception> exceptionHandler);
+
+    /**
+     * Normally, listeners are wrapped in {@link org.apache.curator.framework.CuratorFramework#runSafe(Runnable)}. Use this
+     * method to set a different executor.
+     *
+     * @param executor to use
+     */
+    CuratorCacheBuilder withExecutor(Executor executor);
+
+    /**
+     * Return a new Curator Cache based on the builder methods that have been called
+     *
+     * @return new Curator Cache
+     */
+    CuratorCache build();
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheBuilderImpl.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheBuilderImpl.java
new file mode 100644
index 0000000..325a71e
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheBuilderImpl.java
@@ -0,0 +1,80 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.v2.CuratorFrameworkV2;
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+
+class CuratorCacheBuilderImpl implements CuratorCacheBuilder
+{
+    private final CuratorFrameworkV2 client;
+    private final String path;
+    private CuratorCacheStorage storage;
+    private Consumer<Exception> exceptionHandler;
+    private Executor executor;
+    private CuratorCache.Options[] options;
+
+    CuratorCacheBuilderImpl(CuratorFramework client, String path)
+    {
+        this(CuratorFrameworkV2.wrap(client), path);
+    }
+
+    CuratorCacheBuilderImpl(CuratorFrameworkV2 client, String path)
+    {
+        this.client = client;
+        this.path = path;
+    }
+
+    @Override
+    public CuratorCacheBuilder withOptions(CuratorCache.Options... options)
+    {
+        this.options = options;
+        return this;
+    }
+
+    @Override
+    public CuratorCacheBuilder withStorage(CuratorCacheStorage storage)
+    {
+        this.storage = storage;
+        return this;
+    }
+
+    @Override
+    public CuratorCacheBuilder withExceptionHandler(Consumer<Exception> exceptionHandler)
+    {
+        this.exceptionHandler = exceptionHandler;
+        return this;
+    }
+
+    @Override
+    public CuratorCacheBuilder withExecutor(Executor executor)
+    {
+        this.executor = executor;
+        return this;
+    }
+
+    @Override
+    public CuratorCache build()
+    {
+        return new CuratorCacheImpl(client, storage, path, options, executor, exceptionHandler);
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheImpl.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheImpl.java
new file mode 100644
index 0000000..193669d
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheImpl.java
@@ -0,0 +1,304 @@
+/**
+ * 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.v2.recipes.cache;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.framework.listen.StandardListenerManager;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.utils.ThreadUtils;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.curator.v2.CuratorFrameworkV2;
+import org.apache.curator.v2.recipes.watch.PersistentWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+
+import static org.apache.curator.v2.recipes.cache.CuratorCacheListener.Type.*;
+import static org.apache.zookeeper.KeeperException.Code.NONODE;
+import static org.apache.zookeeper.KeeperException.Code.OK;
+
+class CuratorCacheImpl implements CuratorCache
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final AtomicReference<State> state = new AtomicReference<>(State.LATENT);
+    private final PersistentWatcher persistentWatcher;
+    private final CuratorFrameworkV2 client;
+    private final CuratorCacheStorage storage;
+    private final String path;
+    private final boolean recursive;
+    private final boolean compressedData;
+    private final boolean clearOnClose;
+    private final StandardListenerManager<CuratorCacheListener> listenerManager = StandardListenerManager.standard();
+    private final Consumer<Exception> exceptionHandler;
+    private final Executor executor;
+
+    private volatile AtomicLong outstandingOps;
+
+    private enum State
+    {
+        LATENT,
+        STARTED,
+        CLOSED
+    }
+
+    CuratorCacheImpl(CuratorFrameworkV2 client, CuratorCacheStorage storage, String path, Options[] optionsArg, Executor executor, Consumer<Exception> exceptionHandler)
+    {
+        Set<Options> options = (optionsArg != null) ? Sets.newHashSet(optionsArg) : Collections.emptySet();
+        this.client = client;
+        this.storage = (storage != null) ? storage : CuratorCacheStorage.standard();
+        this.path = path;
+        recursive = !options.contains(Options.SINGLE_NODE_CACHE);
+        compressedData = options.contains(Options.COMPRESSED_DATA);
+        clearOnClose = !options.contains(Options.DO_NOT_CLEAR_ON_CLOSE);
+        persistentWatcher = new PersistentWatcher(client, path, recursive);
+        persistentWatcher.getListenable().addListener(this::processEvent);
+        persistentWatcher.getResetListenable().addListener(this::rebuild);
+        this.exceptionHandler = (exceptionHandler != null) ? exceptionHandler : e -> log.error("CuratorCache error", e);
+        this.executor = (executor != null) ? executor : client::runSafe;
+    }
+
+    @Override
+    public void start()
+    {
+        Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Already started");
+        outstandingOps = new AtomicLong(0);
+        persistentWatcher.start();
+    }
+
+    @Override
+    public void close()
+    {
+        if ( state.compareAndSet(State.STARTED, State.CLOSED) )
+        {
+            persistentWatcher.close();
+            if ( clearOnClose )
+            {
+                storage.clear();
+            }
+        }
+    }
+
+    @Override
+    public CuratorCacheStorage storage()
+    {
+        return storage;
+    }
+
+    @Override
+    public Listenable<CuratorCacheListener> listenable()
+    {
+        return listenerManager;
+    }
+
+    private void rebuild()
+    {
+        if ( state.get() != State.STARTED )
+        {
+            return;
+        }
+
+        nodeChanged(path);
+        storage.stream()
+            .map(ChildData::getPath)
+            .filter(p -> !p.equals(path))
+            .forEach(this::nodeChanged);
+    }
+
+    private void processEvent(WatchedEvent event)
+    {
+        if ( state.get() != State.STARTED )
+        {
+            return;
+        }
+
+        switch ( event.getType() )
+        {
+        case NodeDataChanged:
+        case NodeCreated:
+        {
+            nodeChanged(event.getPath());
+            break;
+        }
+
+        case NodeDeleted:
+        {
+            removeStorage(event.getPath());
+            break;
+        }
+
+        case NodeChildrenChanged:
+        {
+            nodeChildrenChanged(event.getPath());
+            break;
+        }
+        }
+    }
+
+    private void nodeChildrenChanged(String fromPath)
+    {
+        if ( (state.get() != State.STARTED) || !recursive )
+        {
+            return;
+        }
+
+        try
+        {
+            BackgroundCallback callback = (__, event) -> {
+                if ( event.getResultCode() == OK.intValue() )
+                {
+                    event.getChildren().forEach(child -> nodeChanged(ZKPaths.makePath(fromPath, child)));
+                }
+                else if ( event.getResultCode() == NONODE.intValue() )
+                {
+                    removeStorage(event.getPath());
+                }
+                else
+                {
+                    handleException(event);
+                }
+                checkDecrementOutstandingOps();
+            };
+
+            checkIncrementOutstandingOps();
+            client.getChildren().inBackground(callback).forPath(fromPath);
+        }
+        catch ( Exception e )
+        {
+            handleException(e);
+        }
+    }
+
+    private void nodeChanged(String fromPath)
+    {
+        if ( state.get() != State.STARTED )
+        {
+            return;
+        }
+
+        try
+        {
+            BackgroundCallback callback = (__, event) -> {
+                if ( event.getResultCode() == OK.intValue() )
+                {
+                    putStorage(new ChildData(event.getPath(), event.getStat(), event.getData()));
+                    nodeChildrenChanged(event.getPath());
+                }
+                else if ( event.getResultCode() == NONODE.intValue() )
+                {
+                    removeStorage(event.getPath());
+                }
+                else
+                {
+                    handleException(event);
+                }
+                checkDecrementOutstandingOps();
+            };
+
+            checkIncrementOutstandingOps();
+            if ( compressedData )
+            {
+                client.getData().decompressed().inBackground(callback).forPath(fromPath);
+            }
+            else
+            {
+                client.getData().inBackground(callback).forPath(fromPath);
+            }
+        }
+        catch ( Exception e )
+        {
+            handleException(e);
+        }
+    }
+
+    public static volatile boolean JZTEST = false;
+
+    private void putStorage(ChildData data)
+    {
+        Optional<ChildData> previousData = storage.put(data);
+        if ( previousData.isPresent() )
+        {
+            if ( previousData.get().getStat().getMzxid() != data.getStat().getMzxid() )
+            {
+                callListeners(l -> l.event(NODE_CHANGED, previousData.get(), data));
+            }
+        }
+        else
+        {
+            callListeners(l -> l.event(NODE_CREATED, null, data));
+        }
+    }
+
+    private void removeStorage(String path)
+    {
+        storage.remove(path).ifPresent(previousData -> callListeners(l -> l.event(NODE_DELETED, previousData, null)));
+    }
+
+    private void callListeners(Consumer<CuratorCacheListener> proc)
+    {
+        if ( state.get() == State.STARTED )
+        {
+            executor.execute(() -> listenerManager.forEach(proc));
+        }
+    }
+
+    private void handleException(CuratorEvent event)
+    {
+        handleException(KeeperException.create(KeeperException.Code.get(event.getResultCode())));
+    }
+
+    private void handleException(Exception e)
+    {
+        ThreadUtils.checkInterrupted(e);
+        exceptionHandler.accept(e);
+    }
+
+    private void checkIncrementOutstandingOps()
+    {
+        AtomicLong localOutstandingOps = outstandingOps;
+        if ( localOutstandingOps != null )
+        {
+            localOutstandingOps.incrementAndGet();
+        }
+    }
+
+    private void checkDecrementOutstandingOps()
+    {
+        AtomicLong localOutstandingOps = outstandingOps;
+        if ( localOutstandingOps != null )
+        {
+            if ( localOutstandingOps.decrementAndGet() == 0 )
+            {
+                outstandingOps = null;
+                callListeners(CuratorCacheListener::initialized);
+            }
+        }
+    }
+}
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListener.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListener.java
new file mode 100644
index 0000000..163d168
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListener.java
@@ -0,0 +1,80 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.recipes.cache.ChildData;
+
+/**
+ * Listener for {@link CuratorCache} events. The main functional interface is general purpose
+ * but you can build event specific listeners, etc. using the builder. Note: all listeners
+ * are wrapped in {@link org.apache.curator.framework.CuratorFramework#runSafe(Runnable)} when called.
+ */
+@FunctionalInterface
+public interface CuratorCacheListener
+{
+    /**
+     * An enumerated type that describes a change
+     */
+    enum Type
+    {
+        /**
+         * A new node was added to the cache
+         */
+        NODE_CREATED,
+
+        /**
+         * A node already in the cache has changed
+         */
+        NODE_CHANGED,
+
+        /**
+         * A node already in the cache was deleted
+         */
+        NODE_DELETED
+    }
+
+    /**
+     * Called when a data is created, changed or deleted.
+     *
+     * @param type the type of event
+     * @param oldData the old data or null
+     * @param data the new data or null
+     */
+    void event(Type type, ChildData oldData, ChildData data);
+
+    /**
+     * When the cache is started, the initial nodes are tracked and when they are finished loading
+     * into the cache this method is called.
+     */
+    default void initialized()
+    {
+        // NOP
+    }
+
+    /**
+     * Returns a builder allowing type specific, and special purpose listeners.
+     *
+     * @return builder
+     */
+    static CuratorCacheListenerBuilder builder()
+    {
+        return new CuratorCacheListenerBuilderImpl();
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListenerBuilder.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListenerBuilder.java
new file mode 100644
index 0000000..948b372
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListenerBuilder.java
@@ -0,0 +1,133 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+import org.apache.curator.framework.recipes.cache.TreeCacheListener;
+import org.apache.curator.v2.recipes.cache.CuratorCacheListener.Type;
+import java.util.function.Consumer;
+
+public interface CuratorCacheListenerBuilder
+{
+    /**
+     * Add a standard listener
+     *
+     * @param listener listener to add
+     * @return this
+     */
+    CuratorCacheListenerBuilder forAll(CuratorCacheListener listener);
+
+    /**
+     * Add a listener only for {@link Type#NODE_CREATED}
+     *
+     * @param listener listener to add
+     * @return this
+     */
+    CuratorCacheListenerBuilder forCreates(Consumer<ChildData> listener);
+
+    @FunctionalInterface
+    interface ChangeListener
+    {
+        void event(ChildData oldNode, ChildData node);
+    }
+
+    /**
+     * Add a listener only for {@link Type#NODE_CHANGED}
+     *
+     * @param listener listener to add
+     * @return this
+     */
+    CuratorCacheListenerBuilder forChanges(ChangeListener listener);
+
+    /**
+     * Add a listener only both {@link Type#NODE_CREATED} and {@link Type#NODE_CHANGED}
+     *
+     * @param listener listener to add
+     * @return this
+     */
+    CuratorCacheListenerBuilder forCreatesAndChanges(ChangeListener listener);
+
+    /**
+     * Add a listener only for {@link Type#NODE_DELETED}
+     *
+     * @param listener listener to add
+     * @return this
+     */
+    CuratorCacheListenerBuilder forDeletes(Consumer<ChildData> listener);
+
+    /**
+     * Add a listener only for {@link CuratorCacheListener#initialized()}
+     *
+     * @param listener listener to add
+     * @return this
+     */
+    CuratorCacheListenerBuilder forInitialized(Runnable listener);
+
+    /**
+     * Bridge listener. You can reuse old-style {@link org.apache.curator.framework.recipes.cache.PathChildrenCacheListener}s
+     * with CuratorCache. IMPORTANT: the connection state methods in the listener will never be called as CuratorCache
+     * does not register the listener with the connection state listener container. Also note that CuratorCache
+     * behaves differently than {@link org.apache.curator.framework.recipes.cache.PathChildrenCache} so
+     * things such as event ordering will likely be different.
+     *
+     * @param client the curator client
+     * @param listener the listener to wrap
+     * @return a CuratorCacheListener that forwards to the given listener
+     */
+    CuratorCacheListenerBuilder forPathChildrenCache(CuratorFramework client, PathChildrenCacheListener listener);
+
+    /**
+     * Bridge listener. You can reuse old-style {@link org.apache.curator.framework.recipes.cache.TreeCacheListener}s
+     * with CuratorCache. IMPORTANT: the connection state methods in the listener will never be called as CuratorCache
+     * does not register the listener with the connection state listener container. Also note that CuratorCache
+     * behaves differently than {@link org.apache.curator.framework.recipes.cache.TreeCache} so
+     * things such as event ordering will likely be different.
+     *
+     * @param client the curator client
+     * @param listener the listener to wrap
+     * @return a CuratorCacheListener that forwards to the given listener
+     */
+    CuratorCacheListenerBuilder forTreeCache(CuratorFramework client, TreeCacheListener listener);
+
+    /**
+     * Bridge listener. You can reuse old-style {@link org.apache.curator.framework.recipes.cache.NodeCacheListener}s
+     * with CuratorCache.
+     *
+     * @param listener the listener to wrap
+     * @return a CuratorCacheListener that forwards to the given listener
+     */
+    CuratorCacheListenerBuilder forNodeCache(NodeCacheListener listener);
+
+    /**
+     * Make the built listener so that it only becomes active once {@link CuratorCacheListener#initialized()} has been called.
+     * i.e. changes that occur as the cache is initializing are not sent to the listener
+     */
+    CuratorCacheListenerBuilder afterInitialized();
+
+    /**
+     * Build and return a new listener based on the methods that have been previously called
+     *
+     * @return new listener
+     */
+    CuratorCacheListener build();
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListenerBuilderImpl.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListenerBuilderImpl.java
new file mode 100644
index 0000000..7009797
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheListenerBuilderImpl.java
@@ -0,0 +1,165 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+import org.apache.curator.framework.recipes.cache.TreeCacheListener;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+
+class CuratorCacheListenerBuilderImpl implements CuratorCacheListenerBuilder
+{
+    private final List<CuratorCacheListener> listeners = new ArrayList<>();
+    private boolean afterInitializedOnly = false;
+
+    @Override
+    public CuratorCacheListenerBuilder forAll(CuratorCacheListener listener)
+    {
+        listeners.add(listener);
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forCreates(Consumer<ChildData> listener)
+    {
+        listeners.add((type, oldNode, node) -> {
+            if ( type == CuratorCacheListener.Type.NODE_CREATED )
+            {
+                listener.accept(node);
+            }
+        });
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forChanges(ChangeListener listener)
+    {
+        listeners.add((type, oldNode, node) -> {
+            if ( type == CuratorCacheListener.Type.NODE_CHANGED )
+            {
+                listener.event(oldNode, node);
+            }
+        });
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forCreatesAndChanges(ChangeListener listener)
+    {
+        listeners.add((type, oldNode, node) -> {
+            if ( (type == CuratorCacheListener.Type.NODE_CHANGED) || (type == CuratorCacheListener.Type.NODE_CREATED) )
+            {
+                listener.event(oldNode, node);
+            }
+        });
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forDeletes(Consumer<ChildData> listener)
+    {
+        listeners.add((type, oldNode, node) -> {
+            if ( type == CuratorCacheListener.Type.NODE_DELETED )
+            {
+                listener.accept(oldNode);
+            }
+        });
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forInitialized(Runnable listener)
+    {
+        CuratorCacheListener localListener = new CuratorCacheListener()
+        {
+            @Override
+            public void event(Type type, ChildData oldData, ChildData data)
+            {
+                // NOP
+            }
+
+            @Override
+            public void initialized()
+            {
+                listener.run();
+            }
+        };
+        listeners.add(localListener);
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forPathChildrenCache(CuratorFramework client, PathChildrenCacheListener listener)
+    {
+        listeners.add(new PathChildrenCacheListenerWrapper(client, listener));
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forTreeCache(CuratorFramework client, TreeCacheListener listener)
+    {
+        listeners.add(new TreeCacheListenerWrapper(client, listener));
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder forNodeCache(NodeCacheListener listener)
+    {
+        listeners.add(new NodeCacheListenerWrapper(listener));
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListenerBuilder afterInitialized()
+    {
+        afterInitializedOnly = true;
+        return this;
+    }
+
+    @Override
+    public CuratorCacheListener build()
+    {
+        List<CuratorCacheListener> copy = new ArrayList<>(listeners);
+        return new CuratorCacheListener()
+        {
+            private volatile boolean isInitialized = !afterInitializedOnly;
+
+            @Override
+            public void event(Type type, ChildData oldData, ChildData data)
+            {
+                if ( isInitialized )
+                {
+                    copy.forEach(l -> l.event(type, oldData, data));
+                }
+            }
+
+            @Override
+            public void initialized()
+            {
+                isInitialized = true;
+                copy.forEach(CuratorCacheListener::initialized);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheStorage.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheStorage.java
new file mode 100644
index 0000000..137d7d6
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/CuratorCacheStorage.java
@@ -0,0 +1,115 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.recipes.cache.ChildData;
+import java.util.AbstractMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Interface for maintaining data in a {@link CuratorCache}
+ */
+public interface CuratorCacheStorage
+{
+    /**
+     * Return a new standard storage instance
+     *
+     * @return storage instance
+     */
+    static CuratorCacheStorage standard() {
+        return new StandardCuratorCacheStorage(true);
+    }
+
+    /**
+     * Return a new storage instance that does not retain the data bytes. i.e. ChildData objects
+     * returned by this storage will always return {@code null} for {@link ChildData#getData()}.
+     *
+     * @return storage instance that does not retain data bytes
+     */
+    static CuratorCacheStorage bytesNotCached() {
+        return new StandardCuratorCacheStorage(false);
+    }
+
+    /**
+     * Add an entry to storage and return any previous entry at that path
+     *
+     * @param data entry to add
+     * @return previous entry or {@code empty()}
+     */
+    Optional<ChildData> put(ChildData data);
+
+    /**
+     * Remove the entry from storage and return any previous entry at that path
+     *
+     * @param path path to remove
+     * @return previous entry or {@code empty()}
+     */
+    Optional<ChildData> remove(String path);
+
+    /**
+     * Return an entry from storage
+     *
+     * @param path path to get
+     * @return entry or {@code empty()}
+     */
+    Optional<ChildData> get(String path);
+
+    /**
+     * Return the current number of entries in storage
+     *
+     * @return number of entries
+     */
+    int size();
+
+    /**
+     * Return a stream over the storage entries. Note: for a standard storage instance, the stream
+     * behaves like a stream returned by {@link java.util.concurrent.ConcurrentHashMap#entrySet()}
+     *
+     * @return stream over entries
+     */
+    Stream<ChildData> stream();
+
+    /**
+     * Return a stream over the storage entries that are the immediate children of the given node.
+     *
+     * @return stream over entries
+     */
+    Stream<ChildData> streamImmediateChildren(String fromParent);
+
+    /**
+     * Utility - given a stream of child nodes, build a map. Note: it is assumed that each child
+     * data in the stream has a unique path
+     *
+     * @param stream stream of child nodes with unique paths
+     * @return map
+     */
+    static Map<String, ChildData> toMap(Stream<ChildData> stream)
+    {
+        return stream.map(data -> new AbstractMap.SimpleEntry<>(data.getPath(), data))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    /**
+     * Reset the storage to zero entries
+     */
+    void clear();
+}
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/NodeCacheListenerWrapper.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/NodeCacheListenerWrapper.java
new file mode 100644
index 0000000..ab380e7
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/NodeCacheListenerWrapper.java
@@ -0,0 +1,46 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+
+class NodeCacheListenerWrapper implements CuratorCacheListener
+{
+    private final NodeCacheListener listener;
+
+    NodeCacheListenerWrapper(NodeCacheListener listener)
+    {
+        this.listener = listener;
+    }
+
+    @Override
+    public void event(Type type, ChildData oldData, ChildData data)
+    {
+        try
+        {
+            listener.nodeChanged();
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/PathChildrenCacheListenerWrapper.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/PathChildrenCacheListenerWrapper.java
new file mode 100644
index 0000000..3bb2681
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/PathChildrenCacheListenerWrapper.java
@@ -0,0 +1,81 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+
+class PathChildrenCacheListenerWrapper implements CuratorCacheListener
+{
+    private final PathChildrenCacheListener listener;
+    private final CuratorFramework client;
+
+    PathChildrenCacheListenerWrapper(CuratorFramework client, PathChildrenCacheListener listener)
+    {
+        this.listener = listener;
+        this.client = client;
+    }
+
+    @Override
+    public void event(Type type, ChildData oldData, ChildData data)
+    {
+        switch ( type )
+        {
+            case NODE_CREATED:
+            {
+                sendEvent(data, PathChildrenCacheEvent.Type.CHILD_ADDED);
+                break;
+            }
+
+            case NODE_CHANGED:
+            {
+                sendEvent(data, PathChildrenCacheEvent.Type.CHILD_UPDATED);
+                break;
+            }
+
+            case NODE_DELETED:
+            {
+                sendEvent(oldData, PathChildrenCacheEvent.Type.CHILD_REMOVED);
+                break;
+            }
+        }
+    }
+
+    @Override
+    public void initialized()
+    {
+        sendEvent(null, PathChildrenCacheEvent.Type.INITIALIZED);
+    }
+
+    private void sendEvent(ChildData node, PathChildrenCacheEvent.Type type)
+    {
+        PathChildrenCacheEvent event = new PathChildrenCacheEvent(type, node);
+        try
+        {
+            listener.childEvent(client, event);
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/StandardCuratorCacheStorage.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/StandardCuratorCacheStorage.java
new file mode 100644
index 0000000..4d3ac24
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/StandardCuratorCacheStorage.java
@@ -0,0 +1,87 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.utils.ZKPaths;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Stream;
+
+class StandardCuratorCacheStorage implements CuratorCacheStorage
+{
+    private final Map<String, ChildData> dataMap;
+    private final boolean cacheBytes;
+
+    StandardCuratorCacheStorage(boolean cacheBytes)
+    {
+        this.dataMap = new ConcurrentHashMap<>();
+        this.cacheBytes = cacheBytes;
+    }
+
+    @Override
+    public Optional<ChildData> put(ChildData data)
+    {
+        ChildData localData = cacheBytes ? data : new ChildData(data.getPath(), data.getStat(), null);
+        return Optional.ofNullable(dataMap.put(data.getPath(), localData));
+    }
+
+    @Override
+    public Optional<ChildData> remove(String path)
+    {
+        return Optional.ofNullable(dataMap.remove(path));
+    }
+
+    @Override
+    public Optional<ChildData> get(String path)
+    {
+        return Optional.ofNullable(dataMap.get(path));
+    }
+
+    @Override
+    public int size()
+    {
+        return dataMap.size();
+    }
+
+    @Override
+    public Stream<ChildData> stream()
+    {
+        return dataMap.values().stream();
+    }
+
+    @Override
+    public Stream<ChildData> streamImmediateChildren(String fromParent)
+    {
+        return dataMap.entrySet()
+            .stream()
+            .filter(entry -> {
+                ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(entry.getKey());
+                return pathAndNode.getPath().equals(fromParent);
+            })
+            .map(Map.Entry::getValue);
+    }
+
+    @Override
+    public void clear()
+    {
+        dataMap.clear();
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/TreeCacheListenerWrapper.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/TreeCacheListenerWrapper.java
new file mode 100644
index 0000000..4c224d2
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/cache/TreeCacheListenerWrapper.java
@@ -0,0 +1,81 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
+import org.apache.curator.framework.recipes.cache.TreeCacheListener;
+
+class TreeCacheListenerWrapper implements CuratorCacheListener
+{
+    private final CuratorFramework client;
+    private final TreeCacheListener listener;
+
+    TreeCacheListenerWrapper(CuratorFramework client, TreeCacheListener listener)
+    {
+        this.client = client;
+        this.listener = listener;
+    }
+
+    @Override
+    public void event(Type type, ChildData oldData, ChildData data)
+    {
+        switch ( type )
+        {
+            case NODE_CREATED:
+            {
+                sendEvent(data, TreeCacheEvent.Type.NODE_ADDED);
+                break;
+            }
+
+            case NODE_CHANGED:
+            {
+                sendEvent(data, TreeCacheEvent.Type.NODE_UPDATED);
+                break;
+            }
+
+            case NODE_DELETED:
+            {
+                sendEvent(oldData, TreeCacheEvent.Type.NODE_REMOVED);
+                break;
+            }
+        }
+    }
+
+    @Override
+    public void initialized()
+    {
+        sendEvent(null, TreeCacheEvent.Type.INITIALIZED);
+    }
+
+    private void sendEvent(ChildData node, TreeCacheEvent.Type type)
+    {
+        TreeCacheEvent event = new TreeCacheEvent(type, node);
+        try
+        {
+            listener.childEvent(client, event);
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/recipes/watch/PersistentWatcher.java b/curator-v2/src/main/java/org/apache/curator/v2/recipes/watch/PersistentWatcher.java
new file mode 100644
index 0000000..a0cbe39
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/recipes/watch/PersistentWatcher.java
@@ -0,0 +1,180 @@
+/**
+ * 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.
+ */
+/**
+ * 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.v2.recipes.watch;
+
+import com.google.common.base.Preconditions;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.framework.listen.StandardListenerManager;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.utils.ThreadUtils;
+import org.apache.curator.v2.CuratorFrameworkV2;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.Closeable;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A managed persistent watcher. The watch will be managed such that it stays set through
+ * connection lapses, etc.
+ */
+public class PersistentWatcher implements Closeable
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final AtomicReference<State> state = new AtomicReference<>(State.LATENT);
+    private final StandardListenerManager<Watcher> listeners = StandardListenerManager.standard();
+    private final StandardListenerManager<Runnable> resetListeners = StandardListenerManager.standard();
+    private final ConnectionStateListener connectionStateListener = (client, newState) -> {
+        if ( newState.isConnected() )
+        {
+            reset();
+        }
+    };
+    private final Watcher watcher = event -> listeners.forEach(w -> w.process(event));
+    private final CuratorFrameworkV2 client;
+    private final String basePath;
+    private final boolean recursive;
+
+    private enum State
+    {
+        LATENT,
+        STARTED,
+        CLOSED
+    }
+
+    /**
+     * @param client client
+     * @param basePath path to set the watch on
+     * @param recursive ZooKeeper persistent watches can optionally be recursive
+     */
+    public PersistentWatcher(CuratorFramework client, String basePath, boolean recursive)
+    {
+        this(CuratorFrameworkV2.wrap(client), basePath, recursive);
+    }
+
+    /**
+     * @param client client
+     * @param basePath path to set the watch on
+     * @param recursive ZooKeeper persistent watches can optionally be recursive
+     */
+    public PersistentWatcher(CuratorFrameworkV2 client, String basePath, boolean recursive)
+    {
+        this.client = Objects.requireNonNull(client, "client cannot be null");
+        this.basePath = Objects.requireNonNull(basePath, "basePath cannot be null");
+        this.recursive = recursive;
+    }
+
+    /**
+     * Start watching
+     */
+    public void start()
+    {
+        Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Already started");
+        client.getConnectionStateListenable().addListener(connectionStateListener);
+        reset();
+    }
+
+    /**
+     * Remove the watcher
+     */
+    @Override
+    public void close()
+    {
+        if ( state.compareAndSet(State.STARTED, State.CLOSED) )
+        {
+            listeners.clear();
+            client.getConnectionStateListenable().removeListener(connectionStateListener);
+            try
+            {
+                client.watches().remove(watcher).guaranteed().inBackground().forPath(basePath);
+            }
+            catch ( Exception e )
+            {
+                ThreadUtils.checkInterrupted(e);
+                log.debug(String.format("Could not remove watcher for path: %s", basePath), e);
+            }
+        }
+    }
+
+    /**
+     * Container for setting listeners
+     *
+     * @return listener container
+     */
+    public Listenable<Watcher> getListenable()
+    {
+        return listeners;
+    }
+
+    /**
+     * Listeners are called when the persistent watcher has been successfully registered
+     * or re-registered after a connection disruption
+     *
+     * @return listener container
+     */
+    public StandardListenerManager<Runnable> getResetListenable()
+    {
+        return resetListeners;
+    }
+
+    private void reset()
+    {
+        try
+        {
+            BackgroundCallback callback = (__, event) -> {
+                if ( event.getResultCode() != KeeperException.Code.OK.intValue() )
+                {
+                    reset();
+                }
+                else
+                {
+                    resetListeners.forEach(Runnable::run);
+                }
+            };
+            client.watches().add().withMode(recursive ? AddWatchMode.PERSISTENT_RECURSIVE : AddWatchMode.PERSISTENT).inBackground(callback).usingWatcher(watcher).forPath(basePath);
+        }
+        catch ( Exception e )
+        {
+            log.error("Could not reset persistent watch at path: " + basePath, e);
+        }
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/framework/v2/TestAsyncCuratorFrameworkV2.java b/curator-v2/src/test/java/org/apache/curator/framework/v2/TestAsyncCuratorFrameworkV2.java
deleted file mode 100644
index 4785787..0000000
--- a/curator-v2/src/test/java/org/apache/curator/framework/v2/TestAsyncCuratorFrameworkV2.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.framework.v2;
-
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.compatibility.CuratorTestBase;
-import org.apache.curator.v2.AsyncCuratorFrameworkV2;
-import org.apache.zookeeper.AddWatchMode;
-import org.apache.zookeeper.Watcher;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-import java.util.concurrent.CountDownLatch;
-
-public class TestAsyncCuratorFrameworkV2 extends CuratorTestBase
-{
-    @Test
-    public void testPersistentRecursiveWatch() throws Exception
-    {
-        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
-        {
-            AsyncCuratorFrameworkV2 async = AsyncCuratorFrameworkV2.wrap(client);
-
-            client.start();
-            client.blockUntilConnected();
-
-            CountDownLatch latch = new CountDownLatch(5);
-            Watcher watcher = event -> latch.countDown();
-            async.addWatch().withMode(AddWatchMode.PERSISTENT_RECURSIVE).usingWatcher(watcher).forPath("/test").toCompletableFuture().get();
-
-            client.create().forPath("/test");
-            client.create().forPath("/test/a");
-            client.create().forPath("/test/a/b");
-            client.create().forPath("/test/a/b/c");
-            client.create().forPath("/test/a/b/c/d");
-
-            Assert.assertTrue(timing.awaitLatch(latch));
-        }
-    }
-}
diff --git a/curator-v2/src/test/java/org/apache/curator/framework/v2/TestFrameworkV2.java b/curator-v2/src/test/java/org/apache/curator/framework/v2/TestFrameworkV2.java
deleted file mode 100644
index f00dab3..0000000
--- a/curator-v2/src/test/java/org/apache/curator/framework/v2/TestFrameworkV2.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.framework.v2;
-
-import org.apache.curator.v2.CuratorFrameworkV2;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.compatibility.CuratorTestBase;
-import org.apache.zookeeper.AddWatchMode;
-import org.apache.zookeeper.Watcher;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-import java.util.concurrent.CountDownLatch;
-
-import static org.apache.curator.v2.CuratorFrameworkV2.wrap;
-
-public class TestFrameworkV2 extends CuratorTestBase
-{
-    @Test
-    public void testPersistentRecursiveWatch() throws Exception
-    {
-        try ( CuratorFrameworkV2 client = wrap(CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1))) )
-        {
-            client.start();
-            client.blockUntilConnected();
-
-            CountDownLatch latch = new CountDownLatch(5);
-            Watcher watcher = event -> latch.countDown();
-            client.watches().add().withMode(AddWatchMode.PERSISTENT_RECURSIVE).usingWatcher(watcher).forPath("/test");
-
-            client.create().forPath("/test");
-            client.create().forPath("/test/a");
-            client.create().forPath("/test/a/b");
-            client.create().forPath("/test/a/b/c");
-            client.create().forPath("/test/a/b/c/d");
-
-            Assert.assertTrue(timing.awaitLatch(latch));
-        }
-    }
-}
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCache.java b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCache.java
new file mode 100644
index 0000000..58d52ff
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCache.java
@@ -0,0 +1,248 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.InstanceSpec;
+import org.apache.curator.test.TestingCluster;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.curator.v2.recipes.cache.CuratorCache.Options.DO_NOT_CLEAR_ON_CLOSE;
+import static org.apache.curator.v2.recipes.cache.CuratorCacheListener.builder;
+
+public class TestCuratorCache extends CuratorTestBase
+{
+    @Test
+    public void testServerLoss() throws Exception   // mostly copied from TestPathChildrenCacheInCluster
+    {
+        try (TestingCluster cluster = new TestingCluster(3))
+        {
+            cluster.start();
+
+            try (CuratorFramework client = CuratorFrameworkFactory.newClient(cluster.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+            {
+                client.start();
+                client.create().creatingParentsIfNeeded().forPath("/test");
+
+                try (CuratorCache cache = CuratorCache.build(client, "/test"))
+                {
+                    cache.start();
+
+                    CountDownLatch reconnectLatch = new CountDownLatch(1);
+                    client.getConnectionStateListenable().addListener((__, newState) -> {
+                        if ( newState == ConnectionState.RECONNECTED )
+                        {
+                            reconnectLatch.countDown();
+                        }
+                    });
+                    CountDownLatch latch = new CountDownLatch(3);
+                    cache.listenable().addListener((__, ___, ____) -> latch.countDown());
+
+                    client.create().forPath("/test/one");
+                    client.create().forPath("/test/two");
+                    client.create().forPath("/test/three");
+
+                    Assert.assertTrue(timing.awaitLatch(latch));
+
+                    InstanceSpec connectionInstance = cluster.findConnectionInstance(client.getZookeeperClient().getZooKeeper());
+                    cluster.killServer(connectionInstance);
+
+                    Assert.assertTrue(timing.awaitLatch(reconnectLatch));
+
+                    timing.sleepABit();
+
+                    Assert.assertEquals(cache.storage().stream().count(), 4);
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testUpdateWhenNotCachingData() throws Exception // mostly copied from TestPathChildrenCache
+    {
+        CuratorCacheStorage storage = new StandardCuratorCacheStorage(false);
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            final CountDownLatch updatedLatch = new CountDownLatch(1);
+            final CountDownLatch addedLatch = new CountDownLatch(1);
+            client.create().creatingParentsIfNeeded().forPath("/test");
+            try (CuratorCache cache = CuratorCache.builder(client, "/test").withStorage(storage).build())
+            {
+                cache.listenable().addListener(builder().forChanges((__, ___) -> updatedLatch.countDown()).build());
+                cache.listenable().addListener(builder().forCreates(__ -> addedLatch.countDown()).build());
+                cache.start();
+
+                client.create().forPath("/test/foo", "first".getBytes());
+                Assert.assertTrue(timing.awaitLatch(addedLatch));
+
+                client.setData().forPath("/test/foo", "something new".getBytes());
+                Assert.assertTrue(timing.awaitLatch(updatedLatch));
+            }
+        }
+    }
+
+    @Test
+    public void testAfterInitialized() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            client.create().creatingParentsIfNeeded().forPath("/test");
+            client.create().creatingParentsIfNeeded().forPath("/test/one");
+            client.create().creatingParentsIfNeeded().forPath("/test/one/two");
+            client.create().creatingParentsIfNeeded().forPath("/test/one/two/three");
+            try (CuratorCache cache = CuratorCache.build(client, "/test"))
+            {
+                CountDownLatch initializedLatch = new CountDownLatch(1);
+                AtomicInteger eventCount = new AtomicInteger(0);
+                CuratorCacheListener listener = new CuratorCacheListener()
+                {
+                    @Override
+                    public void event(Type type, ChildData oldData, ChildData data)
+                    {
+                        eventCount.incrementAndGet();
+                    }
+
+                    @Override
+                    public void initialized()
+                    {
+                        initializedLatch.countDown();
+                    }
+                };
+                cache.listenable().addListener(builder().forAll(listener).afterInitialized().build());
+                cache.start();
+                Assert.assertTrue(timing.awaitLatch(initializedLatch));
+
+                Assert.assertEquals(initializedLatch.getCount(), 0);
+                Assert.assertEquals(cache.storage().size(), 4);
+                Assert.assertTrue(cache.storage().get("/test").isPresent());
+                Assert.assertTrue(cache.storage().get("/test/one").isPresent());
+                Assert.assertTrue(cache.storage().get("/test/one/two").isPresent());
+                Assert.assertTrue(cache.storage().get("/test/one/two/three").isPresent());
+            }
+        }
+    }
+
+    @Test
+    public void testListenerBuilder() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            try (CuratorCache cache = CuratorCache.build(client, "/test"))
+            {
+                Semaphore all = new Semaphore(0);
+                Semaphore deletes = new Semaphore(0);
+                Semaphore changes = new Semaphore(0);
+                Semaphore creates = new Semaphore(0);
+                Semaphore createsAndChanges = new Semaphore(0);
+
+                CuratorCacheListener listener = builder().forAll((__, ___, ____) -> all.release()).forDeletes(__ -> deletes.release()).forChanges((__, ___) -> changes.release()).forCreates(__ -> creates.release()).forCreatesAndChanges((__, ___) -> createsAndChanges.release()).build();
+                cache.listenable().addListener(listener);
+                cache.start();
+
+                client.create().forPath("/test");
+                Assert.assertTrue(timing.acquireSemaphore(all, 1));
+                Assert.assertTrue(timing.acquireSemaphore(creates, 1));
+                Assert.assertTrue(timing.acquireSemaphore(createsAndChanges, 1));
+                Assert.assertEquals(changes.availablePermits(), 0);
+                Assert.assertEquals(deletes.availablePermits(), 0);
+
+                client.setData().forPath("/test", "new".getBytes());
+                Assert.assertTrue(timing.acquireSemaphore(all, 1));
+                Assert.assertTrue(timing.acquireSemaphore(changes, 1));
+                Assert.assertTrue(timing.acquireSemaphore(createsAndChanges, 1));
+                Assert.assertEquals(creates.availablePermits(), 0);
+                Assert.assertEquals(deletes.availablePermits(), 0);
+
+                client.delete().forPath("/test");
+                Assert.assertTrue(timing.acquireSemaphore(all, 1));
+                Assert.assertTrue(timing.acquireSemaphore(deletes, 1));
+                Assert.assertEquals(creates.availablePermits(), 0);
+                Assert.assertEquals(changes.availablePermits(), 0);
+                Assert.assertEquals(createsAndChanges.availablePermits(), 0);
+            }
+        }
+    }
+
+    @Test
+    public void testOverrideExecutor() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            CountDownLatch latch = new CountDownLatch(2);
+            Executor executor = proc -> {
+                latch.countDown();
+                proc.run();
+            };
+            try ( CuratorCache cache = CuratorCache.builder(client, "/test").withExecutor(executor).build() )
+            {
+                cache.listenable().addListener((type, oldData, data) -> latch.countDown());
+                cache.start();
+
+                client.create().forPath("/test");
+
+                Assert.assertTrue(timing.awaitLatch(latch));
+            }
+        }
+    }
+
+    @Test
+    public void testClearOnClose() throws Exception
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            CuratorCacheStorage storage;
+            client.start();
+
+            try ( CuratorCache cache = CuratorCache.builder(client, "/test").withOptions(DO_NOT_CLEAR_ON_CLOSE).build() )
+            {
+                cache.start();
+                storage = cache.storage();
+
+                client.create().forPath("/test", "foo".getBytes());
+                client.create().forPath("/test/bar", "bar".getBytes());
+                timing.sleepABit();
+            }
+            Assert.assertEquals(storage.size(), 2);
+
+            try ( CuratorCache cache = CuratorCache.build(client, "/test") )
+            {
+                cache.start();
+                storage = cache.storage();
+
+                timing.sleepABit();
+            }
+            Assert.assertEquals(storage.size(), 0);
+        }
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheConsistency.java b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheConsistency.java
new file mode 100644
index 0000000..4d1e7f6
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheConsistency.java
@@ -0,0 +1,373 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.InstanceSpec;
+import org.apache.curator.test.TestingCluster;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.curator.utils.ZKPaths;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.io.Closeable;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.apache.curator.v2.recipes.cache.CuratorCache.Options.DO_NOT_CLEAR_ON_CLOSE;
+import static org.apache.curator.v2.recipes.cache.CuratorCacheListener.builder;
+
+/**
+ * Randomly create nodes in a tree while a set of CuratorCaches listens. Afterwards, validate
+ * that the caches contain the same values as ZK itself
+ */
+public class TestCuratorCacheConsistency extends CuratorTestBase
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final ThreadLocalRandom random = ThreadLocalRandom.current();
+
+    private static final Duration testLength = Duration.ofSeconds(30);
+    private static final Duration thirdOfTestLength = Duration.ofMillis(testLength.toMillis() / 3);
+    private static final Duration sleepLength = Duration.ofMillis(5);
+    private static final int nodesPerLevel = 10;
+    private static final int clusterSize = 5;
+    private static final int maxServerKills = 2;
+
+    private static final String BASE_PATH = "/test";
+
+    private class Client implements Closeable
+    {
+        private final CuratorFramework client;
+        private final CuratorCache cache;
+        private final int index;
+        private final Map<String, ChildData> listenerDataMap = new HashMap<>();
+
+        Client(int index, String connectionString, AtomicReference<Exception> errorSignal)
+        {
+            this.index = index;
+            client = buildClient(connectionString);
+            cache = CuratorCache.builder(client, BASE_PATH).withOptions(DO_NOT_CLEAR_ON_CLOSE).withExceptionHandler(errorSignal::set).build();
+
+            // listenerDataMap is a local data map that will hold values sent by listeners
+            // this way, the listener code can be tested for validity and consistency
+            CuratorCacheListener listener = builder().forCreates(node -> {
+                ChildData previous = listenerDataMap.put(node.getPath(), node);
+                if ( previous != null )
+                {
+                    errorSignal.set(new Exception(String.format("Client: %d - Create for existing node: %s", index, node.getPath())));
+                }
+            }).forChanges((oldNode, node) -> {
+                ChildData previous = listenerDataMap.put(node.getPath(), node);
+                if ( (previous == null) || !Arrays.equals(previous.getData(), oldNode.getData()) )
+                {
+                    errorSignal.set(new Exception(String.format("Client: %d - Bad old value for change node: %s", index, node.getPath())));
+                }
+            }).forDeletes(node -> {
+                ChildData previous = listenerDataMap.remove(node.getPath());
+                if ( previous == null )
+                {
+                    errorSignal.set(new Exception(String.format("Client: %d - Delete for non-existent node: %s", index, node.getPath())));
+                }
+            }).build();
+            cache.listenable().addListener(listener);
+        }
+
+        void start()
+        {
+            client.start();
+            cache.start();
+        }
+
+        @Override
+        public void close()
+        {
+            cache.close();
+            client.close();
+        }
+    }
+
+    @Test
+    public void testConsistencyAfterSimulation() throws Exception
+    {
+        int clientQty = random.nextInt(10, 20);
+        int maxDepth = random.nextInt(5, 10);
+
+        log.info("clientQty: {}, maxDepth: {}", clientQty, maxDepth);
+
+        List<Client> clients = Collections.emptyList();
+        Map<String, String> actualTree;
+
+        AtomicReference<Exception> errorSignal = new AtomicReference<>();
+        try (TestingCluster cluster = new TestingCluster(clusterSize))
+        {
+            cluster.start();
+
+            initializeBasePath(cluster);
+            try
+            {
+                clients = buildClients(cluster, clientQty, errorSignal);
+                workLoop(cluster, clients, maxDepth, errorSignal);
+
+                log.info("Test complete - sleeping to allow events to complete");
+                timing.sleepABit();
+            }
+            finally
+            {
+                clients.forEach(Client::close);
+            }
+
+            actualTree = buildActual(cluster);
+        }
+
+        log.info("client qty: {}", clientQty);
+
+        Map<Integer, List<String>> errorsList = clients.stream()
+            .map(client ->  findErrors(client, actualTree))
+            .filter(errorsEntry -> !errorsEntry.getValue().isEmpty())
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+        if ( !errorsList.isEmpty() )
+        {
+            log.error("{} clients had errors", errorsList.size());
+            errorsList.forEach((index, errorList) -> {
+                log.error("Client {}", index);
+                errorList.forEach(log::error);
+                log.error("");
+            });
+
+            Assert.fail("Errors found");
+        }
+    }
+
+    // build a data map recursively from the actual values in ZK
+    private Map<String, String> buildActual(TestingCluster cluster)
+    {
+        Map<String, String> actual = new HashMap<>();
+        try (CuratorFramework client = buildClient(cluster.getConnectString()))
+        {
+            client.start();
+            buildActual(client, actual, BASE_PATH);
+        }
+        return actual;
+    }
+
+    private void buildActual(CuratorFramework client, Map<String, String> actual, String fromPath)
+    {
+        try
+        {
+            byte[] bytes = client.getData().forPath(fromPath);
+            actual.put(fromPath, new String(bytes));
+            client.getChildren().forPath(fromPath).forEach(child -> buildActual(client, actual, ZKPaths.makePath(fromPath, child)));
+        }
+        catch ( Exception e )
+        {
+            Assert.fail("", e);
+        }
+    }
+
+    private List<Client> buildClients(TestingCluster cluster, int clientQty, AtomicReference<Exception> errorSignal)
+    {
+        return IntStream.range(0, clientQty)
+            .mapToObj(index -> new Client(index, cluster.getConnectString(), errorSignal))
+            .peek(Client::start)
+            .collect(Collectors.toList());
+    }
+
+    private void initializeBasePath(TestingCluster cluster) throws Exception
+    {
+        try (CuratorFramework client = buildClient(cluster.getConnectString()))
+        {
+            client.start();
+            client.create().forPath(BASE_PATH, "".getBytes());
+        }
+    }
+
+    private void workLoop(TestingCluster cluster, List<Client> clients, int maxDepth, AtomicReference<Exception> errorSignal) throws Exception
+    {
+        Instant start = Instant.now();
+        Instant lastServerKill = Instant.now();
+        int serverKillIndex = 0;
+        while ( true )
+        {
+            Duration elapsed = Duration.between(start, Instant.now());
+            if ( elapsed.compareTo(testLength) >= 0 )
+            {
+                break;
+            }
+
+            Exception errorSignalException = errorSignal.get();
+            if ( errorSignalException != null )
+            {
+                Assert.fail("A client's error handler was called", errorSignalException);
+            }
+
+            Duration elapsedFromLastServerKill = Duration.between(lastServerKill, Instant.now());
+            if ( elapsedFromLastServerKill.compareTo(thirdOfTestLength) >= 0 )
+            {
+                lastServerKill = Instant.now();
+                if ( serverKillIndex < maxServerKills )
+                {
+                    doKillServer(cluster, serverKillIndex++);
+                }
+            }
+
+            int thisDepth = random.nextInt(0, maxDepth);
+            String thisPath = randomPath(thisDepth);
+            CuratorFramework client = randomClient(clients);
+            if ( random.nextBoolean() )
+            {
+                doDelete(client, thisPath);
+            }
+            else
+            {
+                doChange(client, thisPath);
+            }
+
+            Thread.sleep(sleepLength.toMillis());
+        }
+    }
+
+    private void doChange(CuratorFramework client, String thisPath)
+    {
+        try
+        {
+            String thisData = Long.toString(random.nextLong());
+            client.create().orSetData().creatingParentsIfNeeded().forPath(thisPath, thisData.getBytes());
+        }
+        catch ( Exception e )
+        {
+            Assert.fail("Could not create/set: " + thisPath);
+        }
+    }
+
+    private void doDelete(CuratorFramework client, String thisPath)
+    {
+        if ( thisPath.equals(BASE_PATH) )
+        {
+            return;
+        }
+        try
+        {
+            client.delete().quietly().deletingChildrenIfNeeded().forPath(thisPath);
+        }
+        catch ( Exception e )
+        {
+            Assert.fail("Could not delete: " + thisPath);
+        }
+    }
+
+    private void doKillServer(TestingCluster cluster, int serverKillIndex) throws Exception
+    {
+        log.info("Killing server {}", serverKillIndex);
+        InstanceSpec killSpec = new ArrayList<>(cluster.getInstances()).get(serverKillIndex);
+        cluster.killServer(killSpec);
+    }
+
+    private CuratorFramework randomClient(List<Client> clients)
+    {
+        return clients.get(random.nextInt(clients.size())).client;
+    }
+
+    private Map.Entry<Integer, List<String>> findErrors(Client client, Map<String, String> tree)
+    {
+        CuratorCacheStorage storage = client.cache.storage();
+        List<String> errors = new ArrayList<>();
+        if ( tree.size() != storage.size() )
+        {
+            errors.add(String.format("Size mismatch. Expected: %d - Actual: %d", tree.size(), storage.size()));
+        }
+        tree.keySet().forEach(path -> {
+            if ( !storage.get(path).isPresent() )
+            {
+                errors.add(String.format("Path %s in master but not client", path));
+            }
+        });
+        storage.stream().forEach(data -> {
+            String treeValue = tree.get(data.getPath());
+            if ( treeValue != null )
+            {
+                if ( !treeValue.equals(new String(data.getData())) )
+                {
+                    errors.add(String.format("Data at %s is not the same", data.getPath()));
+                }
+
+                ChildData listenersMapData = client.listenerDataMap.get(data.getPath());
+                if ( listenersMapData == null )
+                {
+                    errors.add(String.format("listenersMap missing data at: %s", data.getPath()));
+                }
+                else if ( !treeValue.equals(new String(listenersMapData.getData())) )
+                {
+                    errors.add(String.format("Data at %s in listenersMap is not the same", data.getPath()));
+                }
+            }
+            else
+            {
+                errors.add(String.format("Path %s in client but not master", data.getPath()));
+            }
+        });
+
+        client.listenerDataMap.keySet().forEach(path -> {
+            if ( !storage.get(path).isPresent() )
+            {
+                errors.add(String.format("Path %s in listenersMap but not storage", path));
+            }
+        });
+
+        return new AbstractMap.SimpleEntry<>(client.index, errors);
+    }
+
+    private String randomPath(int depth)
+    {
+        StringBuilder str = new StringBuilder(BASE_PATH);
+        while ( depth-- > 0 )
+        {
+            int levelNodeName = random.nextInt(nodesPerLevel);
+            str.append("/").append(levelNodeName);
+        }
+        return str.toString();
+    }
+
+    @Override
+    protected void createServer()
+    {
+        // do nothing - we'll be using TestingCluster instead
+    }
+
+    private CuratorFramework buildClient(String connectionString)
+    {
+        ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(100, 100);
+        return CuratorFrameworkFactory.newClient(connectionString, timing.session(), timing.connection(), retryPolicy);
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheEventOrdering.java b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheEventOrdering.java
new file mode 100644
index 0000000..4dd5869
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheEventOrdering.java
@@ -0,0 +1,50 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.cache.TestEventOrdering;
+import java.util.concurrent.BlockingQueue;
+
+public class TestCuratorCacheEventOrdering extends TestEventOrdering<CuratorCache>
+{
+    @Override
+    protected int getActualQty(CuratorCache cache)
+    {
+        return cache.storage().size();
+    }
+
+    @Override
+    protected CuratorCache newCache(CuratorFramework client, String path, BlockingQueue<Event> events)
+    {
+        CuratorCache cache = CuratorCache.build(client, path);
+        cache.listenable().addListener((type, oldNode, node) -> {
+            if ( type == CuratorCacheListener.Type.NODE_CREATED )
+            {
+                events.add(new Event(EventType.ADDED, node.getPath()));
+            }
+            else if ( type == CuratorCacheListener.Type.NODE_DELETED )
+            {
+                events.add(new Event(EventType.DELETED, oldNode.getPath()));
+            }
+        });
+        cache.start();
+        return cache;
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheWrappers.java b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheWrappers.java
new file mode 100644
index 0000000..40d7fd8
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestCuratorCacheWrappers.java
@@ -0,0 +1,166 @@
+/**
+ * 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.v2.recipes.cache;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.recipes.cache.BaseTestTreeCache;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
+import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import static org.apache.curator.v2.recipes.cache.CuratorCache.Options.SINGLE_NODE_CACHE;
+import static org.apache.curator.v2.recipes.cache.CuratorCacheListener.builder;
+import static org.apache.curator.v2.recipes.cache.CuratorCacheStorage.toMap;
+
+public class TestCuratorCacheWrappers extends CuratorTestBase
+{
+    @Test
+    public void testPathChildrenCache() throws Exception    // copied from TestPathChildrenCache#testBasics()
+    {
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            client.create().forPath("/test");
+
+            final BlockingQueue<PathChildrenCacheEvent.Type> events = new LinkedBlockingQueue<>();
+            try (CuratorCache cache = CuratorCache.build(client, "/test"))
+            {
+                PathChildrenCacheListener listener = (__, event) -> {
+                    if ( event.getData().getPath().equals("/test/one") )
+                    {
+                        events.offer(event.getType());
+                    }
+                };
+                cache.listenable().addListener(builder().forPathChildrenCache(client, listener).build());
+                cache.start();
+
+                client.create().forPath("/test/one", "hey there".getBytes());
+                Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_ADDED);
+
+                client.setData().forPath("/test/one", "sup!".getBytes());
+                Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_UPDATED);
+                Assert.assertEquals(new String(cache.storage().get("/test/one").orElseThrow(AssertionError::new).getData()), "sup!");
+
+                client.delete().forPath("/test/one");
+                Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
+            }
+        }
+    }
+
+    @Test
+    public void testTreeCache() throws Exception    // copied from TestTreeCache#testBasics()
+    {
+        BaseTestTreeCache treeCacheBase = new BaseTestTreeCache();
+        try (CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            client.create().forPath("/test");
+
+            try (CuratorCache cache = CuratorCache.build(client, "/test"))
+            {
+                cache.listenable().addListener(builder().forTreeCache(client, treeCacheBase.eventListener).build());
+                cache.start();
+
+                treeCacheBase.assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test");
+                treeCacheBase.assertEvent(TreeCacheEvent.Type.INITIALIZED);
+                Assert.assertEquals(toMap(cache.storage().streamImmediateChildren("/test")).keySet(), ImmutableSet.of());
+                Assert.assertEquals(cache.storage().streamImmediateChildren("/t").count(), 0);
+                Assert.assertEquals(cache.storage().streamImmediateChildren("/testing").count(), 0);
+
+                client.create().forPath("/test/one", "hey there".getBytes());
+                treeCacheBase.assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/one");
+                Assert.assertEquals(toMap(cache.storage().streamImmediateChildren("/test")).keySet(), ImmutableSet.of("/test/one"));
+                Assert.assertEquals(new String(cache.storage().get("/test/one").orElseThrow(AssertionError::new).getData()), "hey there");
+                Assert.assertEquals(toMap(cache.storage().streamImmediateChildren("/test/one")).keySet(), ImmutableSet.of());
+                Assert.assertEquals(cache.storage().streamImmediateChildren("/test/o").count(), 0);
+                Assert.assertEquals(cache.storage().streamImmediateChildren("/test/onely").count(), 0);
+
+                client.setData().forPath("/test/one", "sup!".getBytes());
+                treeCacheBase.assertEvent(TreeCacheEvent.Type.NODE_UPDATED, "/test/one");
+                Assert.assertEquals(toMap(cache.storage().streamImmediateChildren("/test")).keySet(), ImmutableSet.of("/test/one"));
+                Assert.assertEquals(new String(cache.storage().get("/test/one").orElseThrow(AssertionError::new).getData()), "sup!");
+
+                client.delete().forPath("/test/one");
+                treeCacheBase.assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/one", "sup!".getBytes());
+                Assert.assertEquals(toMap(cache.storage().streamImmediateChildren("/test")).keySet(), ImmutableSet.of());
+            }
+        }
+    }
+
+    @Test
+    public void testNodeCache() throws Exception    // copied from TestNodeCache#testBasics()
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)) )
+        {
+            client.start();
+            client.create().forPath("/test");
+
+            try (CuratorCache cache = CuratorCache.build(client, "/test/node", SINGLE_NODE_CACHE))
+            {
+                Supplier<ChildData> getRootData = () -> cache.storage().get("/test/node").orElseThrow(() -> new AssertionError("is not present"));
+                cache.start();
+
+                final Semaphore semaphore = new Semaphore(0);
+                cache.listenable().addListener(builder().forNodeCache(semaphore::release).build());
+                try
+                {
+                    getRootData.get();
+                    Assert.fail("Should have thrown");
+                }
+                catch ( AssertionError expected )
+                {
+                    // expected
+                }
+
+                client.create().forPath("/test/node", "a".getBytes());
+                Assert.assertTrue(timing.acquireSemaphore(semaphore));
+                Assert.assertEquals(getRootData.get().getData(), "a".getBytes());
+
+                client.setData().forPath("/test/node", "b".getBytes());
+                Assert.assertTrue(timing.acquireSemaphore(semaphore));
+                Assert.assertEquals(getRootData.get().getData(), "b".getBytes());
+
+                client.delete().forPath("/test/node");
+                Assert.assertTrue(timing.acquireSemaphore(semaphore));
+                try
+                {
+                    getRootData.get();
+                    Assert.fail("Should have thrown");
+                }
+                catch ( AssertionError expected )
+                {
+                    // expected
+                }
+            }
+        }
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestWrappedNodeCache.java b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestWrappedNodeCache.java
new file mode 100644
index 0000000..f95d3b4
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/recipes/cache/TestWrappedNodeCache.java
@@ -0,0 +1,178 @@
+/**
+ * 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.v2.recipes.cache;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.NodeCacheListener;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.compatibility.Timing2;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.curator.utils.Compatibility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Semaphore;
+import java.util.function.Supplier;
+
+import static org.apache.curator.v2.recipes.cache.CuratorCacheListener.builder;
+
+@Test(groups = Zk35MethodInterceptor.zk35Group)
+public class TestWrappedNodeCache extends BaseClassForTests
+{
+    private final Timing2 timing = new Timing2();
+
+    @Test
+    public void testDeleteThenCreate() throws Exception
+    {
+        CuratorCache cache = null;
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            client.create().creatingParentsIfNeeded().forPath("/test/foo", "one".getBytes());
+
+            final Semaphore semaphore = new Semaphore(0);
+            cache = CuratorCache.build(client, "/test/foo");
+            NodeCacheListener listener = semaphore::release;
+            cache.listenable().addListener(builder().forNodeCache(listener).build());
+
+            Supplier<Optional<ChildData>> rootData = getRootDataProc(cache, "/test/foo");
+
+            cache.start();
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+
+            Assert.assertTrue(rootData.get().isPresent());
+            Assert.assertEquals(rootData.get().get().getData(), "one".getBytes());
+
+            client.delete().forPath("/test/foo");
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            client.create().forPath("/test/foo", "two".getBytes());
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+
+            Assert.assertTrue(rootData.get().isPresent());
+            Assert.assertEquals(rootData.get().get().getData(), "two".getBytes());
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
+        }
+    }
+
+    @Test
+    public void testKilledSession() throws Exception
+    {
+        CuratorCache cache = null;
+        CuratorFramework client = null;
+        try
+        {
+            client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
+            client.start();
+            client.create().creatingParentsIfNeeded().forPath("/test/node", "start".getBytes());
+
+            CountDownLatch lostLatch = new CountDownLatch(1);
+            client.getConnectionStateListenable().addListener((__, newState) -> {
+                if ( newState == ConnectionState.LOST )
+                {
+                    lostLatch.countDown();
+                }
+            });
+
+            cache = CuratorCache.build(client,"/test/node");
+
+            Semaphore latch = new Semaphore(0);
+            NodeCacheListener listener = latch::release;
+            cache.listenable().addListener(builder().forNodeCache(listener).build());
+
+            Supplier<Optional<ChildData>> rootData = getRootDataProc(cache, "/test/node");
+
+            cache.start();
+            Assert.assertTrue(timing.acquireSemaphore(latch));
+
+            Compatibility.injectSessionExpiration(client.getZookeeperClient().getZooKeeper());
+            Assert.assertTrue(timing.awaitLatch(lostLatch));
+
+            Assert.assertTrue(rootData.get().isPresent());
+            Assert.assertEquals(rootData.get().get().getData(), "start".getBytes());
+
+            client.setData().forPath("/test/node", "new data".getBytes());
+            Assert.assertTrue(timing.acquireSemaphore(latch));
+            Assert.assertTrue(rootData.get().isPresent());
+            Assert.assertEquals(rootData.get().get().getData(), "new data".getBytes());
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
+        }
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    @Test
+    public void testBasics() throws Exception
+    {
+        CuratorCache cache = null;
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            client.create().forPath("/test");
+
+            cache = CuratorCache.build(client, "/test/node");
+            cache.start();
+
+            Supplier<Optional<ChildData>> rootData = getRootDataProc(cache, "/test/node");
+
+            final Semaphore semaphore = new Semaphore(0);
+            NodeCacheListener listener = semaphore::release;
+            cache.listenable().addListener(builder().forNodeCache(listener).build());
+
+            Assert.assertNull(rootData.get().orElse(null));
+
+            client.create().forPath("/test/node", "a".getBytes());
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            Assert.assertEquals(rootData.get().orElse(null).getData(), "a".getBytes());
+
+            client.setData().forPath("/test/node", "b".getBytes());
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            Assert.assertEquals(rootData.get().orElse(null).getData(), "b".getBytes());
+
+            client.delete().forPath("/test/node");
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            Assert.assertNull(rootData.get().orElse(null));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
+        }
+    }
+
+    private Supplier<Optional<ChildData>> getRootDataProc(CuratorCache cache, String rootPath)
+    {
+        return () -> cache.storage().get(rootPath);
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/recipes/watch/TestPersistentWatcher.java b/curator-v2/src/test/java/org/apache/curator/v2/recipes/watch/TestPersistentWatcher.java
new file mode 100644
index 0000000..1ae177b
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/recipes/watch/TestPersistentWatcher.java
@@ -0,0 +1,104 @@
+/**
+ * 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.v2.recipes.watch;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TestPersistentWatcher extends CuratorTestBase
+{
+    @Test
+    public void testConnectionLostRecursive() throws Exception
+    {
+        internalTest(true);
+    }
+
+    @Test
+    public void testConnectionLost() throws Exception
+    {
+        internalTest(false);
+    }
+
+    private void internalTest(boolean recursive) throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)) )
+        {
+            CountDownLatch lostLatch = new CountDownLatch(1);
+            CountDownLatch reconnectedLatch = new CountDownLatch(1);
+            client.start();
+            client.getConnectionStateListenable().addListener((__, newState) -> {
+                if ( newState == ConnectionState.LOST )
+                {
+                    lostLatch.countDown();
+                }
+                else if ( newState == ConnectionState.RECONNECTED )
+                {
+                    reconnectedLatch.countDown();
+                }
+            });
+
+            try ( PersistentWatcher persistentWatcher = new PersistentWatcher(client, "/top/main", recursive) )
+            {
+                persistentWatcher.start();
+
+                BlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<>();
+                persistentWatcher.getListenable().addListener(events::add);
+
+                client.create().creatingParentsIfNeeded().forPath("/top/main/a");
+                Assert.assertEquals(timing.takeFromQueue(events).getPath(), "/top/main");
+                if ( recursive )
+                {
+                    Assert.assertEquals(timing.takeFromQueue(events).getPath(), "/top/main/a");
+                }
+                else
+                {
+                    Assert.assertEquals(timing.takeFromQueue(events).getPath(), "/top/main");   // child added
+                }
+
+                server.stop();
+                Assert.assertEquals(timing.takeFromQueue(events).getState(), Watcher.Event.KeeperState.Disconnected);
+                Assert.assertTrue(timing.awaitLatch(lostLatch));
+
+                server.restart();
+                Assert.assertTrue(timing.awaitLatch(reconnectedLatch));
+
+                timing.sleepABit();     // time to allow watcher to get reset
+                events.clear();
+
+                if ( recursive )
+                {
+                    client.setData().forPath("/top/main/a", "foo".getBytes());
+                    Assert.assertEquals(timing.takeFromQueue(events).getType(), Watcher.Event.EventType.NodeDataChanged);
+                }
+                client.setData().forPath("/top/main", "bar".getBytes());
+                Assert.assertEquals(timing.takeFromQueue(events).getPath(), "/top/main");
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 731cc84..3211a83 100644
--- a/pom.xml
+++ b/pom.xml
@@ -370,6 +370,12 @@
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
+                <artifactId>curator-v2</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.apache.curator</groupId>
                 <artifactId>curator-recipes</artifactId>
                 <version>${project.version}</version>
                 <type>test-jar</type>


[curator] 01/03: wip - new module that will be used to support features in ZK 3.6+ while maintaining background compatability with previous versions of ZK in the other modules. At some point in the future, this can be merged into the main modules and then removed

Posted by ra...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

randgalt pushed a commit to branch curator-v2-recipe
in repository https://gitbox.apache.org/repos/asf/curator.git

commit 5e8ccf14448d159270a546e2511d0035274da51c
Author: randgalt <ra...@apache.org>
AuthorDate: Sat Nov 2 11:40:44 2019 -0500

    wip - new module that will be used to support features in ZK 3.6+ while maintaining background compatability with previous versions of ZK in the other modules. At some point in the future, this can be merged into the main modules and then removed
---
 .../org/apache/curator/utils/Compatibility.java    |  11 +
 .../curator/utils/DefaultZookeeperFactory.java     |  17 ++
 .../curator/framework/api/CuratorEventType.java    |   7 +-
 .../framework/imps/CuratorFrameworkImpl.java       |   9 +
 .../framework/imps/ReconfigBuilderImpl.java        |   5 +-
 .../framework/imps/RemoveWatchesBuilderImpl.java   |   9 +-
 .../apache/curator/framework/imps/Watching.java    |  10 +-
 .../curator/framework/imps/TestFramework.java      |   5 +-
 .../curator/framework/imps/TestFrameworkEdges.java |   2 +
 .../framework/imps/TestReconfiguration.java        |   2 +-
 .../curator/framework/imps/TestWithCluster.java    |   2 +
 .../state/TestConnectionStateManager.java          |   2 +
 .../framework/recipes/cache/BaseTestTreeCache.java |   4 +-
 .../framework/recipes/cache/TestNodeCache.java     |   2 +
 .../recipes/cache/TestPathChildrenCache.java       |   2 +
 .../cache/TestPathChildrenCacheInCluster.java      |   2 +
 .../framework/recipes/cache/TestTreeCache.java     |   2 +
 .../recipes/leader/ChaosMonkeyCnxnFactory.java     |   8 +-
 .../framework/recipes/leader/TestLeaderLatch.java  |   2 +
 .../locks/TestInterProcessSemaphoreCluster.java    |   2 +
 curator-test/pom.xml                               |  10 +
 .../org/apache/curator/test/BaseClassForTests.java |   4 +-
 .../org/apache/curator/test/Compatibility.java     |  93 +++++++
 .../org/apache/curator/test/TestingCluster.java    |   2 +-
 .../apache/curator/test/TestingQuorumPeerMain.java |   2 +-
 .../apache/curator/test/TestingZooKeeperMain.java  |   6 +-
 .../test/compatibility/Zk35MethodInterceptor.java  |   1 +
 curator-v2/pom.xml                                 | 146 +++++++++++
 .../curator/framework/api/AddWatchBuilder.java     |  19 +-
 .../curator/framework/api/AddWatchBuilder2.java    |  13 +-
 .../apache/curator/framework/api/AddWatchable.java |  28 +-
 .../curator/framework/api/WatchesBuilder.java      |  20 +-
 .../framework/imps/AddWatchBuilderImpl.java        | 172 ++++++++++++
 .../framework/imps/CuratorFrameworkV2Impl.java     | 290 +++++++++++++++++++++
 .../imps/WatcherRemoveCuratorFrameworkV2Impl.java  | 284 ++++++++++++++++++++
 .../curator/framework/imps/WatchesBuilderImpl.java |  26 +-
 .../apache/curator/v2/AsyncCuratorFrameworkV2.java |  69 +++++
 .../org/apache/curator/v2/CuratorFrameworkV2.java  |  33 ++-
 .../v2/WatcherRemoveCuratorFrameworkV2.java        |  10 +-
 .../x/async/api/AsyncCuratorFrameworkDslV2.java    |  14 +-
 .../curator/x/async/api/AsyncWatchBuilder.java     |  24 +-
 .../async/details/AsyncCuratorFrameworkV2Impl.java | 189 ++++++++++++++
 .../x/async/details/AsyncWatchBuilderImpl.java     |  77 ++++++
 .../curator/v2/TestAsyncCuratorFrameworkV2.java    |  57 ++++
 .../org/apache/curator/v2/TestFrameworkV2.java     |  57 ++++
 curator-v2/src/test/resources/log4j.properties     |  27 ++
 .../x/async/details/AsyncCuratorFrameworkImpl.java |  10 +
 pom.xml                                            |  45 +++-
 src/site/confluence/zk-compatibility.confluence    |  41 ++-
 49 files changed, 1767 insertions(+), 107 deletions(-)

diff --git a/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java b/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java
index 1ee2301..f9810fe 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/Compatibility.java
@@ -32,6 +32,7 @@ public class Compatibility
 {
     private static final boolean hasZooKeeperAdmin;
     private static final Method queueEventMethod;
+
     private static final Logger logger = LoggerFactory.getLogger(Compatibility.class);
 
     static
@@ -74,6 +75,16 @@ public class Compatibility
     }
 
     /**
+     * Return true if the ZooKeeperAdmin class is available
+     *
+     * @return true/false
+     */
+    public static boolean hasZooKeeperAdmin()
+    {
+        return hasZooKeeperAdmin;
+    }
+
+    /**
      * For ZooKeeper 3.5.x, use the supported <code>zooKeeper.getTestable().injectSessionExpiration()</code>.
      * For ZooKeeper 3.4.x do the equivalent via reflection
      *
diff --git a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java b/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
index 42279d0..f936518 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
@@ -20,12 +20,29 @@ package org.apache.curator.utils;
 
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 
 public class DefaultZookeeperFactory implements ZookeeperFactory
 {
+    // hide org.apache.zookeeper.admin.ZooKeeperAdmin in a nested class so that Curator continues to work with ZK 3.4.x
+    private static class ZooKeeperAdminMaker implements ZookeeperFactory
+    {
+        static final ZooKeeperAdminMaker instance = new ZooKeeperAdminMaker();
+
+        @Override
+        public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) throws Exception
+        {
+            return new ZooKeeperAdmin(connectString, sessionTimeout, watcher, canBeReadOnly);
+        }
+    }
+
     @Override
     public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) throws Exception
     {
+        if ( Compatibility.hasZooKeeperAdmin() )
+        {
+            return ZooKeeperAdminMaker.instance.newZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly);
+        }
         return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly);
     }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEventType.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEventType.java
index 5dea211..d19f49b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEventType.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEventType.java
@@ -96,5 +96,10 @@ public enum CuratorEventType
     /**
      * Event sent when client is being closed
      */
-    CLOSING
+    CLOSING,
+
+    /**
+     * Corresponds to {@code CuratorFrameworkV2.watches().add()}
+     */
+    ADD_WATCH
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
index e003bf0..bc59e77 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
@@ -620,6 +620,15 @@ public class CuratorFrameworkImpl implements CuratorFramework
         return client.getZooKeeper();
     }
 
+    Object getZooKeeperAdmin() throws Exception
+    {
+        if ( isZk34CompatibilityMode() )
+        {
+            Preconditions.checkState(!isZk34CompatibilityMode(), "getZooKeeperAdmin() is not supported when running in ZooKeeper 3.4 compatibility mode");
+        }
+        return client.getZooKeeper();
+    }
+
     CompressionProvider getCompressionProvider()
     {
         return compressionProvider;
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ReconfigBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ReconfigBuilderImpl.java
index 97be59a..9f129ca 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/ReconfigBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ReconfigBuilderImpl.java
@@ -24,6 +24,7 @@ import org.apache.curator.RetryLoop;
 import org.apache.curator.TimeTrace;
 import org.apache.curator.framework.api.*;
 import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.DataTree;
 import java.util.Arrays;
@@ -268,7 +269,7 @@ public class ReconfigBuilderImpl implements ReconfigBuilder, BackgroundOperation
                     client.processBackgroundOperation(data, event);
                 }
             };
-            client.getZooKeeper().reconfig(joining, leaving, newMembers, fromConfig, callback, backgrounding.getContext());
+            ((ZooKeeperAdmin)client.getZooKeeperAdmin()).reconfigure(joining, leaving, newMembers, fromConfig, callback, backgrounding.getContext());
         }
         catch ( Throwable e )
         {
@@ -287,7 +288,7 @@ public class ReconfigBuilderImpl implements ReconfigBuilder, BackgroundOperation
                     @Override
                     public byte[] call() throws Exception
                     {
-                        return client.getZooKeeper().reconfig(joining, leaving, newMembers, fromConfig, responseStat);
+                        return ((ZooKeeperAdmin)client.getZooKeeperAdmin()).reconfigure(joining, leaving, newMembers, fromConfig, responseStat);
                     }
                 }
             );
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/RemoveWatchesBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/RemoveWatchesBuilderImpl.java
index e14deff..961d5f0 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/RemoveWatchesBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/RemoveWatchesBuilderImpl.java
@@ -201,8 +201,13 @@ public class RemoveWatchesBuilderImpl implements RemoveWatchesBuilder, RemoveWat
         }        
         
         return null;
-    }    
-    
+    }
+
+    protected CuratorFrameworkImpl getClient()
+    {
+        return client;
+    }
+
     private void pathInBackground(final String path)
     {
         OperationAndData.ErrorCallback<String>  errorCallback = null;
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/Watching.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/Watching.java
index daa5dd3..5bad7e7 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/Watching.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/Watching.java
@@ -23,7 +23,7 @@ import org.apache.curator.framework.api.CuratorWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 
-class Watching
+public class Watching
 {
     private final Watcher watcher;
     private final CuratorWatcher curatorWatcher;
@@ -31,7 +31,7 @@ class Watching
     private final CuratorFrameworkImpl client;
     private NamespaceWatcher namespaceWatcher;
 
-    Watching(CuratorFrameworkImpl client, boolean watched)
+    public Watching(CuratorFrameworkImpl client, boolean watched)
     {
         this.client = client;
         this.watcher = null;
@@ -39,7 +39,7 @@ class Watching
         this.watched = watched;
     }
 
-    Watching(CuratorFrameworkImpl client, Watcher watcher)
+    public Watching(CuratorFrameworkImpl client, Watcher watcher)
     {
         this.client = client;
         this.watcher = watcher;
@@ -47,7 +47,7 @@ class Watching
         this.watched = false;
     }
 
-    Watching(CuratorFrameworkImpl client, CuratorWatcher watcher)
+    public Watching(CuratorFrameworkImpl client, CuratorWatcher watcher)
     {
         this.client = client;
         this.watcher = null;
@@ -55,7 +55,7 @@ class Watching
         this.watched = false;
     }
 
-    Watching(CuratorFrameworkImpl client)
+    public Watching(CuratorFrameworkImpl client)
     {
         this.client = client;
         watcher = null;
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
index fe49ad7..0431829 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
@@ -59,9 +59,10 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
 @SuppressWarnings("deprecation")
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestFramework extends BaseClassForTests
 {
-    @BeforeMethod
+    @BeforeMethod(alwaysRun = true)
     @Override
     public void setup() throws Exception
     {
@@ -69,7 +70,7 @@ public class TestFramework extends BaseClassForTests
         super.setup();
     }
 
-    @AfterMethod
+    @AfterMethod(alwaysRun = true)
     @Override
     public void teardown() throws Exception
     {
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
index 7c6d156..7af039b 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java
@@ -41,6 +41,7 @@ import org.apache.curator.test.InstanceSpec;
 import org.apache.curator.test.TestingCluster;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.compatibility.Timing2;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.Compatibility;
 import org.apache.curator.utils.ZKPaths;
@@ -66,6 +67,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestFrameworkEdges extends BaseClassForTests
 {
     private final Logger log = LoggerFactory.getLogger(getClass());
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
index c6ff2bb..a6399de 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
@@ -57,7 +57,7 @@ import java.util.Properties;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 
-@Test(groups = Zk35MethodInterceptor.zk35Group)
+@Test(groups = {Zk35MethodInterceptor.zk35Group, Zk35MethodInterceptor.curatorV2Group})
 public class TestReconfiguration extends CuratorTestBase
 {
     private final Timing2 timing = new Timing2();
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWithCluster.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWithCluster.java
index 7e8ffbb..079c186 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWithCluster.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWithCluster.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.framework.imps;
 
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -33,6 +34,7 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 import java.util.concurrent.CountDownLatch;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestWithCluster
 {
     @Test
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/state/TestConnectionStateManager.java b/curator-framework/src/test/java/org/apache/curator/framework/state/TestConnectionStateManager.java
index c929b41..bb0d569 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/state/TestConnectionStateManager.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/state/TestConnectionStateManager.java
@@ -24,12 +24,14 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.compatibility.Timing2;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestConnectionStateManager extends BaseClassForTests {
 
     @Test
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java
index 175ccdf..246704f 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/BaseTestTreeCache.java
@@ -96,7 +96,7 @@ public class BaseTestTreeCache extends BaseClassForTests
     }
 
     @Override
-    @BeforeMethod
+    @BeforeMethod(alwaysRun = true)
     public void setup() throws Exception
     {
         super.setup();
@@ -111,7 +111,7 @@ public class BaseTestTreeCache extends BaseClassForTests
     }
 
     @Override
-    @AfterMethod
+    @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception
     {
         try
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
index ff416d5..f9b8cce 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestNodeCache.java
@@ -21,6 +21,7 @@ package org.apache.curator.framework.recipes.cache;
 import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.compatibility.Timing2;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -40,6 +41,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestNodeCache extends BaseClassForTests
 {
     @Test
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
index 78fabd5..7ec86b9 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
@@ -32,6 +32,7 @@ import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.Compatibility;
 import org.apache.zookeeper.CreateMode;
@@ -45,6 +46,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import static org.testng.AssertJUnit.assertNotNull;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestPathChildrenCache extends BaseClassForTests
 {
     @Test
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCacheInCluster.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCacheInCluster.java
index cd87125..083d158 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCacheInCluster.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCacheInCluster.java
@@ -20,6 +20,7 @@ package org.apache.curator.framework.recipes.cache;
 
 import com.google.common.collect.Queues;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -34,6 +35,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestPathChildrenCacheInCluster extends BaseClassForTests
 {
     @Test(enabled = false)  // this test is very flakey - it needs to be re-written at some point
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
index 1e97ce2..3e8a3a5 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
@@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent.Type;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.Compatibility;
 import org.apache.zookeeper.CreateMode;
@@ -31,6 +32,7 @@ import org.testng.annotations.Test;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestTreeCache extends BaseTestTreeCache
 {
     @Test
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java
index 4cb342c..07e9a17 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java
@@ -19,11 +19,11 @@
 
 package org.apache.curator.framework.recipes.leader;
 
+import org.apache.curator.test.Compatibility;
 import org.apache.curator.test.TestingZooKeeperMain;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.proto.CreateRequest;
 import org.apache.zookeeper.server.ByteBufferInputStream;
-import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
 import org.apache.zookeeper.server.Request;
 import org.apache.zookeeper.server.ZooKeeperServer;
@@ -92,7 +92,7 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory
                 log.debug("Rejected : " + si.toString());
                 // Still reject request
                 log.debug("Still not ready for " + remaining + "ms");
-                ((NIOServerCnxn)si.cnxn).close();
+                Compatibility.serverCnxnClose(si.cnxn);
                 return;
             }
             // Submit the request to the legacy Zookeeper server
@@ -113,13 +113,13 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory
                         firstError = System.currentTimeMillis();
                         // The znode has been created, close the connection and don't tell it to client
                         log.warn("Closing connection right after " + createRequest.getPath() + " creation");
-                        ((NIOServerCnxn)si.cnxn).close();
+                        Compatibility.serverCnxnClose(si.cnxn);
                     }
                 }
                 catch ( Exception e )
                 {
                     // Should not happen
-                    ((NIOServerCnxn)si.cnxn).close();
+                    Compatibility.serverCnxnClose(si.cnxn);
                 }
             }
         }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
index 3d9e9b7..ddb7f53 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
@@ -38,6 +38,7 @@ import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
 import org.apache.curator.test.compatibility.Timing2;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -60,6 +61,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestLeaderLatch extends BaseClassForTests
 {
     private static final String PATH_NAME = "/one/two/me";
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphoreCluster.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphoreCluster.java
index ed56f15..72ecf59 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphoreCluster.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphoreCluster.java
@@ -30,6 +30,7 @@ import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.InstanceSpec;
 import org.apache.curator.test.TestingCluster;
 import org.apache.curator.test.Timing;
+import org.apache.curator.test.compatibility.Zk35MethodInterceptor;
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -46,6 +47,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+@Test(groups = Zk35MethodInterceptor.curatorV2Group)
 public class TestInterProcessSemaphoreCluster extends BaseClassForTests
 {
     @Test
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 3683b7d..4f0c5a2 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -41,6 +41,16 @@
         </dependency>
 
         <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
         </dependency>
diff --git a/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java b/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java
index 51af821..79c8b9a 100644
--- a/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java
+++ b/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java
@@ -113,7 +113,7 @@ public class BaseClassForTests
         context.getSuite().addListener(methodListener2);
     }
 
-    @BeforeMethod
+    @BeforeMethod(alwaysRun = true)
     public void setup() throws Exception
     {
         if ( INTERNAL_PROPERTY_DONT_LOG_CONNECTION_ISSUES != null )
@@ -142,7 +142,7 @@ public class BaseClassForTests
         }
     }
 
-    @AfterMethod
+    @AfterMethod(alwaysRun = true)
     public void teardown() throws Exception
     {
         System.clearProperty(INTERNAL_PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY);
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
index 5b4b53f..1f8d181 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
@@ -18,10 +18,103 @@
  */
 package org.apache.curator.test;
 
+import org.apache.zookeeper.server.ServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import java.lang.reflect.Method;
+
+@SuppressWarnings("unchecked")
 public class Compatibility
 {
+    private static final Method closeAllWithReasonMethod;
+    private static final Method closeAllMethod;
+    private static final Method closeWithReasonMethod;
+    private static final Method closeMethod;
+    private static final Object disconnectReasonObj;
+
+    static
+    {
+        Object localDisconnectReasonObj;
+        Method localCloseAllWithReasonMethod;
+        Method localCloseAllMethod;
+        Method localCloseWithReasonMethod;
+        Method localCloseMethod;
+        try
+        {
+            Class disconnectReasonClass = Class.forName("org.apache.zookeeper.server.ServerCnxn$DisconnectReason");
+            localDisconnectReasonObj = Enum.valueOf(disconnectReasonClass, "UNKNOWN");
+            localCloseAllWithReasonMethod = ServerCnxnFactory.class.getDeclaredMethod("closeAll", disconnectReasonClass);
+            localCloseWithReasonMethod = ServerCnxn.class.getDeclaredMethod("close", disconnectReasonClass);
+            localCloseAllMethod = null;
+            localCloseMethod = null;
+
+            localCloseAllWithReasonMethod.setAccessible(true);
+            localCloseWithReasonMethod.setAccessible(true);
+        }
+        catch ( Throwable e )
+        {
+            localDisconnectReasonObj = null;
+            localCloseAllWithReasonMethod = null;
+            localCloseWithReasonMethod = null;
+            try
+            {
+                localCloseAllMethod = ServerCnxnFactory.class.getDeclaredMethod("closeAll");
+                localCloseMethod = ServerCnxn.class.getDeclaredMethod("close");
+
+                localCloseAllMethod.setAccessible(true);
+                localCloseMethod.setAccessible(true);
+            }
+            catch ( Throwable ex )
+            {
+                throw new IllegalStateException("Could not reflectively find ServerCnxnFactory/ServerCnxn close methods");
+            }
+        }
+        disconnectReasonObj = localDisconnectReasonObj;
+        closeAllWithReasonMethod = localCloseAllWithReasonMethod;
+        closeAllMethod = localCloseAllMethod;
+        closeMethod = localCloseMethod;
+        closeWithReasonMethod = localCloseWithReasonMethod;
+    }
+
     public static boolean isZK34()
     {
         return false;
     }
+
+    public static void serverCnxnFactoryCloseAll(ServerCnxnFactory factory)
+    {
+        try
+        {
+            if ( closeAllMethod != null )
+            {
+                closeAllMethod.invoke(factory);
+            }
+            else
+            {
+                closeAllWithReasonMethod.invoke(factory, disconnectReasonObj);
+            }
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException("Could not close factory", e);
+        }
+    }
+
+    public static void serverCnxnClose(ServerCnxn cnxn)
+    {
+        try
+        {
+            if ( closeMethod != null )
+            {
+                closeMethod.invoke(cnxn);
+            }
+            else
+            {
+                closeWithReasonMethod.invoke(cnxn, disconnectReasonObj);
+            }
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException("Could not close connection", e);
+        }
+    }
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingCluster.java b/curator-test/src/main/java/org/apache/curator/test/TestingCluster.java
index 3d38fe1..58da2c0 100644
--- a/curator-test/src/main/java/org/apache/curator/test/TestingCluster.java
+++ b/curator-test/src/main/java/org/apache/curator/test/TestingCluster.java
@@ -225,7 +225,7 @@ public class TestingCluster implements Closeable
      */
     public InstanceSpec findConnectionInstance(ZooKeeper client) throws Exception
     {
-        Method              m = client.getClass().getDeclaredMethod("testableRemoteSocketAddress");
+        Method              m = ZooKeeper.class.getDeclaredMethod("testableRemoteSocketAddress");
         m.setAccessible(true);
         InetSocketAddress   address = (InetSocketAddress)m.invoke(client);
         if ( address != null )
diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingQuorumPeerMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingQuorumPeerMain.java
index 3b3ab26..7489527 100644
--- a/curator-test/src/main/java/org/apache/curator/test/TestingQuorumPeerMain.java
+++ b/curator-test/src/main/java/org/apache/curator/test/TestingQuorumPeerMain.java
@@ -39,7 +39,7 @@ class TestingQuorumPeerMain extends QuorumPeerMain implements ZooKeeperMainFace
                 Field               cnxnFactoryField = QuorumPeer.class.getDeclaredField("cnxnFactory");
                 cnxnFactoryField.setAccessible(true);
                 ServerCnxnFactory   cnxnFactory = (ServerCnxnFactory)cnxnFactoryField.get(quorumPeer);
-                cnxnFactory.closeAll();
+                Compatibility.serverCnxnFactoryCloseAll(cnxnFactory);
 
                 Field               ssField = cnxnFactory.getClass().getDeclaredField("ss");
                 ssField.setAccessible(true);
diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java
index 841df77..91f185f 100644
--- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java
+++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java
@@ -81,7 +81,7 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace
         {
             if ( cnxnFactory != null )
             {
-                cnxnFactory.closeAll();
+                Compatibility.serverCnxnFactoryCloseAll(cnxnFactory);
 
                 Field ssField = cnxnFactory.getClass().getDeclaredField("ss");
                 ssField.setAccessible(true);
@@ -262,7 +262,9 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace
     {
         public TestZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config)
         {
-            super(txnLog, config.getTickTime(), config.getMinSessionTimeout(), config.getMaxSessionTimeout(), null);
+            this.setTxnLogFactory(txnLog);
+            this.setMinSessionTimeout(config.getMinSessionTimeout());
+            this.setMaxSessionTimeout(config.getMaxSessionTimeout());
         }
 
         private final AtomicBoolean isRunning = new AtomicBoolean(false);
diff --git a/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java b/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java
index 8072b68..10fafde 100644
--- a/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java
+++ b/curator-test/src/main/java/org/apache/curator/test/compatibility/Zk35MethodInterceptor.java
@@ -29,6 +29,7 @@ import java.util.List;
 public class Zk35MethodInterceptor implements IMethodInterceptor
 {
     public static final String zk35Group = "zk35";
+    public static final String curatorV2Group = "curatorV2";
 
     @Override
     public List<IMethodInstance> intercept(List<IMethodInstance> methods, ITestContext context)
diff --git a/curator-v2/pom.xml b/curator-v2/pom.xml
new file mode 100644
index 0000000..0ff537a
--- /dev/null
+++ b/curator-v2/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.curator</groupId>
+        <artifactId>apache-curator</artifactId>
+        <version>4.2.1-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>curator-v2</artifactId>
+
+    <properties>
+        <zookeeper-new-version>3.6.0-SNAPSHOT</zookeeper-new-version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-x-async</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+            <version>${zookeeper-new-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.sun.jmx</groupId>
+                    <artifactId>jmxri</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.sun.jdmk</groupId>
+                    <artifactId>jmxtools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.jms</groupId>
+                    <artifactId>jms</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-test</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+            </exclusions>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <dependenciesToScan>
+                        <dependency>org.apache.curator:curator-framework</dependency>
+                        <dependency>org.apache.curator:curator-recipes</dependency>
+                    </dependenciesToScan>
+                    <groups>curatorV2</groups>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchBuilder.java
similarity index 68%
copy from curator-test/src/main/java/org/apache/curator/test/Compatibility.java
copy to curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchBuilder.java
index 5b4b53f..ad6d434 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchBuilder.java
@@ -16,12 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.test;
+package org.apache.curator.framework.api;
 
-public class Compatibility
+import org.apache.zookeeper.AddWatchMode;
+
+public interface AddWatchBuilder extends AddWatchBuilder2
 {
-    public static boolean isZK34()
-    {
-        return false;
-    }
-}
+    /**
+     * The mode to use. By default, {@link org.apache.zookeeper.AddWatchMode#PERSISTENT_RECURSIVE} is used
+     *
+     * @param mode mode to use
+     * @return this
+     */
+    AddWatchBuilder2 withMode(AddWatchMode mode);
+}
\ No newline at end of file
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchBuilder2.java
similarity index 82%
copy from curator-test/src/main/java/org/apache/curator/test/Compatibility.java
copy to curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchBuilder2.java
index 5b4b53f..d002c23 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchBuilder2.java
@@ -16,12 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.test;
 
-public class Compatibility
+package org.apache.curator.framework.api;
+
+public interface AddWatchBuilder2 extends
+    Backgroundable<AddWatchable<Pathable<Void>>>,
+    AddWatchable<Pathable<Void>>
 {
-    public static boolean isZK34()
-    {
-        return false;
-    }
-}
+}
\ No newline at end of file
diff --git a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java b/curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchable.java
similarity index 68%
copy from curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
copy to curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchable.java
index 42279d0..1f0646c 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
+++ b/curator-v2/src/main/java/org/apache/curator/framework/api/AddWatchable.java
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.utils;
+
+package org.apache.curator.framework.api;
 
 import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
 
-public class DefaultZookeeperFactory implements ZookeeperFactory
+public interface AddWatchable<T>
 {
-    @Override
-    public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) throws Exception
-    {
-        return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly);
-    }
-}
+    /**
+     * Set a watcher for the operation
+     *
+     * @param watcher the watcher
+     * @return this
+     */
+    T usingWatcher(Watcher watcher);
+
+    /**
+     * Set a watcher for the operation
+     *
+     * @param watcher the watcher
+     * @return this
+     */
+    T usingWatcher(CuratorWatcher watcher);
+}
\ No newline at end of file
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-v2/src/main/java/org/apache/curator/framework/api/WatchesBuilder.java
similarity index 75%
copy from curator-test/src/main/java/org/apache/curator/test/Compatibility.java
copy to curator-v2/src/main/java/org/apache/curator/framework/api/WatchesBuilder.java
index 5b4b53f..3cd5528 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-v2/src/main/java/org/apache/curator/framework/api/WatchesBuilder.java
@@ -16,12 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.test;
 
-public class Compatibility
+package org.apache.curator.framework.api;
+
+/**
+ * Builder to allow watches to be removed 
+ */
+public interface WatchesBuilder extends RemoveWatchesBuilder
 {
-    public static boolean isZK34()
-    {
-        return false;
-    }
-}
+    /**
+     * Start an add watch operation
+     *
+     * @return builder
+     */
+    AddWatchBuilder add();
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/framework/imps/AddWatchBuilderImpl.java b/curator-v2/src/main/java/org/apache/curator/framework/imps/AddWatchBuilderImpl.java
new file mode 100644
index 0000000..3be559b
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/framework/imps/AddWatchBuilderImpl.java
@@ -0,0 +1,172 @@
+/**
+ * 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.RetryLoop;
+import org.apache.curator.drivers.OperationTrace;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.framework.api.CuratorWatcher;
+import org.apache.curator.framework.api.Pathable;
+import org.apache.curator.framework.api.AddWatchBuilder;
+import org.apache.curator.framework.api.AddWatchBuilder2;
+import org.apache.curator.framework.api.AddWatchable;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.Watcher;
+import java.util.concurrent.Executor;
+
+public class AddWatchBuilderImpl implements AddWatchBuilder, Pathable<Void>, BackgroundOperation<String>
+{
+    private final CuratorFrameworkImpl client;
+    private Watching watching = null;
+    private Backgrounding backgrounding = new Backgrounding();
+    private AddWatchMode mode = AddWatchMode.PERSISTENT_RECURSIVE;
+
+    AddWatchBuilderImpl(CuratorFrameworkImpl client)
+    {
+        this.client = client;
+    }
+
+    public AddWatchBuilderImpl(CuratorFrameworkImpl client, Watching watching, Backgrounding backgrounding, AddWatchMode mode)
+    {
+        this.client = client;
+        this.watching = watching;
+        this.backgrounding = backgrounding;
+        this.mode = mode;
+    }
+
+    @Override
+    public AddWatchable<Pathable<Void>> inBackground()
+    {
+        backgrounding = new Backgrounding();
+        return this;
+    }
+
+    @Override
+    public AddWatchBuilder2 withMode(AddWatchMode mode)
+    {
+        this.mode = mode;
+        return this;
+    }
+
+    @Override
+    public Pathable<Void> usingWatcher(Watcher watcher)
+    {
+        watching = new Watching(client, watcher);
+        return this;
+    }
+
+    @Override
+    public Pathable<Void> usingWatcher(CuratorWatcher watcher)
+    {
+        watching = new Watching(client, watcher);
+        return this;
+    }
+
+    @Override
+    public AddWatchable<Pathable<Void>> inBackground(Object context)
+    {
+        backgrounding = new Backgrounding(context);
+        return this;
+    }
+
+    @Override
+    public AddWatchable<Pathable<Void>> inBackground(BackgroundCallback callback)
+    {
+        backgrounding = new Backgrounding(callback);
+        return this;
+    }
+
+    @Override
+    public AddWatchable<Pathable<Void>> inBackground(BackgroundCallback callback, Object context)
+    {
+        backgrounding = new Backgrounding(callback, context);
+        return this;
+    }
+
+    @Override
+    public AddWatchable<Pathable<Void>> inBackground(BackgroundCallback callback, Executor executor)
+    {
+        backgrounding = new Backgrounding(callback, executor);
+        return this;
+    }
+
+    @Override
+    public AddWatchable<Pathable<Void>> inBackground(BackgroundCallback callback, Object context, Executor executor)
+    {
+        backgrounding = new Backgrounding(client, callback, context, executor);
+        return this;
+    }
+
+    @Override
+    public Void forPath(String path) throws Exception
+    {
+        if ( backgrounding.inBackground() )
+        {
+            client.processBackgroundOperation(new OperationAndData<>(this, path, backgrounding.getCallback(), null, backgrounding.getContext(), watching), null);
+        }
+        else
+        {
+            pathInForeground(path);
+        }
+        return null;
+    }
+
+    @Override
+    public void performBackgroundOperation(final OperationAndData<String> data) throws Exception
+    {
+        String path = data.getData();
+        String fixedPath = client.fixForNamespace(path);
+        try
+        {
+            final OperationTrace   trace = client.getZookeeperClient().startAdvancedTracer("AddWatchBuilderImpl-Background");
+            client.getZooKeeper().addWatch
+                (
+                    fixedPath,
+                    watching.getWatcher(path),
+                    mode,
+                    (rc, path1, ctx) -> {
+                        trace.setReturnCode(rc).setWithWatcher(true).setPath(path1).commit();
+                        CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.ADD_WATCH, rc, path1, null, ctx, null, null, null, null, null, null);
+                        client.processBackgroundOperation(data, event);
+                    },
+                    backgrounding.getContext()
+                );
+        }
+        catch ( Throwable e )
+        {
+            backgrounding.checkError(e, watching);
+        }
+    }
+
+    private void pathInForeground(final String path) throws Exception
+    {
+        final String fixedPath = client.fixForNamespace(path);
+        OperationTrace trace = client.getZookeeperClient().startAdvancedTracer("AddWatchBuilderImpl-Foreground");
+        RetryLoop.callWithRetry
+        (
+            client.getZookeeperClient(), () -> {
+                client.getZooKeeper().addWatch(fixedPath, watching.getWatcher(path), mode);
+                return null;
+            });
+        trace.setPath(fixedPath).setWithWatcher(true).commit();
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkV2Impl.java b/curator-v2/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkV2Impl.java
new file mode 100644
index 0000000..338ca6f
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkV2Impl.java
@@ -0,0 +1,290 @@
+/**
+ * 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.CuratorZookeeperClient;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.*;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
+import org.apache.curator.framework.api.transaction.CuratorTransaction;
+import org.apache.curator.framework.api.transaction.TransactionOp;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.framework.schema.SchemaSet;
+import org.apache.curator.framework.state.ConnectionStateErrorPolicy;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.v2.CuratorFrameworkV2;
+import org.apache.curator.v2.WatcherRemoveCuratorFrameworkV2;
+import org.apache.curator.utils.EnsurePath;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+public class CuratorFrameworkV2Impl implements CuratorFrameworkV2
+{
+    private final CuratorFrameworkImpl client;
+
+    public CuratorFrameworkV2Impl(CuratorFramework client)
+    {
+        this.client = reveal(client);
+    }
+
+    private static CuratorFrameworkImpl reveal(CuratorFramework client)
+    {
+        try
+        {
+            return (CuratorFrameworkImpl)Objects.requireNonNull(client, "client cannot be null");
+        }
+        catch ( Exception e )
+        {
+            throw new IllegalArgumentException("Only Curator clients created through CuratorFrameworkFactory are supported: " + client.getClass().getName());
+        }
+    }
+
+    @Override
+    public void start()
+    {
+        client.start();
+    }
+
+    @Override
+    public void close()
+    {
+        client.close();
+    }
+
+    @Override
+    public CuratorFrameworkState getState()
+    {
+        return client.getState();
+    }
+
+    @Override
+    public boolean isStarted()
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public CreateBuilder create()
+    {
+        return client.create();
+    }
+
+    @Override
+    public DeleteBuilder delete()
+    {
+        return client.delete();
+    }
+
+    @Override
+    public ExistsBuilder checkExists()
+    {
+        return client.checkExists();
+    }
+
+    @Override
+    public GetDataBuilder getData()
+    {
+        return client.getData();
+    }
+
+    @Override
+    public SetDataBuilder setData()
+    {
+        return client.setData();
+    }
+
+    @Override
+    public GetChildrenBuilder getChildren()
+    {
+        return client.getChildren();
+    }
+
+    @Override
+    public GetACLBuilder getACL()
+    {
+        return client.getACL();
+    }
+
+    @Override
+    public SetACLBuilder setACL()
+    {
+        return client.setACL();
+    }
+
+    @Override
+    public ReconfigBuilder reconfig()
+    {
+        return client.reconfig();
+    }
+
+    @Override
+    public GetConfigBuilder getConfig()
+    {
+        return client.getConfig();
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    public CuratorTransaction inTransaction()
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public CuratorMultiTransaction transaction()
+    {
+        return client.transaction();
+    }
+
+    @Override
+    public TransactionOp transactionOp()
+    {
+        return client.transactionOp();
+    }
+
+    @Override
+    public void sync(String path, Object backgroundContextObject)
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public void createContainers(String path) throws Exception
+    {
+        client.createContainers(path);
+    }
+
+    @Override
+    public SyncBuilder sync()
+    {
+        return client.sync();
+    }
+
+    @Override
+    public WatchesBuilder watches()
+    {
+        return new WatchesBuilderImpl(client);
+    }
+
+    @Override
+    public Listenable<ConnectionStateListener> getConnectionStateListenable()
+    {
+        return client.getConnectionStateListenable();
+    }
+
+    @Override
+    public Listenable<CuratorListener> getCuratorListenable()
+    {
+        return client.getCuratorListenable();
+    }
+
+    @Override
+    public Listenable<UnhandledErrorListener> getUnhandledErrorListenable()
+    {
+        return client.getUnhandledErrorListenable();
+    }
+
+    @Override
+    public CuratorFramework nonNamespaceView()
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public CuratorFrameworkV2 usingNamespace(String newNamespace)
+    {
+        return CuratorFrameworkV2.wrap(client.usingNamespace(newNamespace));
+    }
+
+    @Override
+    public String getNamespace()
+    {
+        return client.getNamespace();
+    }
+
+    @Override
+    public CuratorZookeeperClient getZookeeperClient()
+    {
+        return client.getZookeeperClient();
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    public EnsurePath newNamespaceAwareEnsurePath(String path)
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public void clearWatcherReferences(Watcher watcher)
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public boolean blockUntilConnected(int maxWaitTime, TimeUnit units) throws InterruptedException
+    {
+        return client.blockUntilConnected(maxWaitTime, units);
+    }
+
+    @Override
+    public void blockUntilConnected() throws InterruptedException
+    {
+        client.blockUntilConnected();
+    }
+
+    @Override
+    public WatcherRemoveCuratorFrameworkV2 newWatcherRemoveCuratorFramework()
+    {
+        return new WatcherRemoveCuratorFrameworkV2Impl(client, client.newWatcherRemoveCuratorFramework());
+    }
+
+    @Override
+    public ConnectionStateErrorPolicy getConnectionStateErrorPolicy()
+    {
+        return client.getConnectionStateErrorPolicy();
+    }
+
+    @Override
+    public QuorumVerifier getCurrentConfig()
+    {
+        return client.getCurrentConfig();
+    }
+
+    @Override
+    public SchemaSet getSchemaSet()
+    {
+        return client.getSchemaSet();
+    }
+
+    @Override
+    public boolean isZk34CompatibilityMode()
+    {
+        return client.isZk34CompatibilityMode();
+    }
+
+    @Override
+    public CompletableFuture<Void> runSafe(Runnable runnable)
+    {
+        return client.runSafe(runnable);
+    }
+}
diff --git a/curator-v2/src/main/java/org/apache/curator/framework/imps/WatcherRemoveCuratorFrameworkV2Impl.java b/curator-v2/src/main/java/org/apache/curator/framework/imps/WatcherRemoveCuratorFrameworkV2Impl.java
new file mode 100644
index 0000000..67daf3e
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/framework/imps/WatcherRemoveCuratorFrameworkV2Impl.java
@@ -0,0 +1,284 @@
+/**
+ * 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.CuratorZookeeperClient;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
+import org.apache.curator.framework.api.*;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
+import org.apache.curator.framework.api.transaction.CuratorTransaction;
+import org.apache.curator.framework.api.transaction.TransactionOp;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.framework.schema.SchemaSet;
+import org.apache.curator.framework.state.ConnectionStateErrorPolicy;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.v2.CuratorFrameworkV2;
+import org.apache.curator.v2.WatcherRemoveCuratorFrameworkV2;
+import org.apache.curator.utils.EnsurePath;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+class WatcherRemoveCuratorFrameworkV2Impl implements WatcherRemoveCuratorFrameworkV2
+{
+    private final CuratorFrameworkImpl client;
+    private final WatcherRemoveCuratorFramework facade;
+
+    WatcherRemoveCuratorFrameworkV2Impl(CuratorFrameworkImpl client, WatcherRemoveCuratorFramework facade)
+    {
+        this.client = client;
+        this.facade = facade;
+    }
+
+    @Override
+    public void removeWatchers()
+    {
+        facade.removeWatchers();
+    }
+
+    @Override
+    public void start()
+    {
+        facade.start();
+    }
+
+    @Override
+    public void close()
+    {
+        facade.close();
+    }
+
+    @Override
+    public CuratorFrameworkState getState()
+    {
+        return facade.getState();
+    }
+
+    @Override
+    public boolean isStarted()
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public CreateBuilder create()
+    {
+        return facade.create();
+    }
+
+    @Override
+    public DeleteBuilder delete()
+    {
+        return facade.delete();
+    }
+
+    @Override
+    public ExistsBuilder checkExists()
+    {
+        return facade.checkExists();
+    }
+
+    @Override
+    public GetDataBuilder getData()
+    {
+        return facade.getData();
+    }
+
+    @Override
+    public SetDataBuilder setData()
+    {
+        return facade.setData();
+    }
+
+    @Override
+    public GetChildrenBuilder getChildren()
+    {
+        return facade.getChildren();
+    }
+
+    @Override
+    public GetACLBuilder getACL()
+    {
+        return facade.getACL();
+    }
+
+    @Override
+    public SetACLBuilder setACL()
+    {
+        return facade.setACL();
+    }
+
+    @Override
+    public ReconfigBuilder reconfig()
+    {
+        return facade.reconfig();
+    }
+
+    @Override
+    public GetConfigBuilder getConfig()
+    {
+        return facade.getConfig();
+    }
+
+    @Override
+    public CuratorTransaction inTransaction()
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public CuratorMultiTransaction transaction()
+    {
+        return facade.transaction();
+    }
+
+    @Override
+    public TransactionOp transactionOp()
+    {
+        return facade.transactionOp();
+    }
+
+    @Override
+    public void sync(String path, Object backgroundContextObject)
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public void createContainers(String path) throws Exception
+    {
+        facade.createContainers(path);
+    }
+
+    @Override
+    public SyncBuilder sync()
+    {
+        return facade.sync();
+    }
+
+    @Override
+    public WatchesBuilder watches()
+    {
+        return new WatchesBuilderImpl(client);
+    }
+
+    @Override
+    public Listenable<ConnectionStateListener> getConnectionStateListenable()
+    {
+        return facade.getConnectionStateListenable();
+    }
+
+    @Override
+    public Listenable<CuratorListener> getCuratorListenable()
+    {
+        return facade.getCuratorListenable();
+    }
+
+    @Override
+    public Listenable<UnhandledErrorListener> getUnhandledErrorListenable()
+    {
+        return facade.getUnhandledErrorListenable();
+    }
+
+    @Override
+    public CuratorFramework nonNamespaceView()
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public CuratorFrameworkV2 usingNamespace(String newNamespace)
+    {
+        return CuratorFrameworkV2.wrap(facade.usingNamespace(newNamespace));
+    }
+
+    @Override
+    public String getNamespace()
+    {
+        return facade.getNamespace();
+    }
+
+    @Override
+    public CuratorZookeeperClient getZookeeperClient()
+    {
+        return facade.getZookeeperClient();
+    }
+
+    @Override
+    public EnsurePath newNamespaceAwareEnsurePath(String path)
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public void clearWatcherReferences(Watcher watcher)
+    {
+        throw new UnsupportedOperationException("deprecated");
+    }
+
+    @Override
+    public boolean blockUntilConnected(int maxWaitTime, TimeUnit units) throws InterruptedException
+    {
+        return facade.blockUntilConnected(maxWaitTime, units);
+    }
+
+    @Override
+    public void blockUntilConnected() throws InterruptedException
+    {
+        facade.blockUntilConnected();
+    }
+
+    @Override
+    public WatcherRemoveCuratorFrameworkV2 newWatcherRemoveCuratorFramework()
+    {
+        return new WatcherRemoveCuratorFrameworkV2Impl(client, facade.newWatcherRemoveCuratorFramework());
+    }
+
+    @Override
+    public ConnectionStateErrorPolicy getConnectionStateErrorPolicy()
+    {
+        return facade.getConnectionStateErrorPolicy();
+    }
+
+    @Override
+    public QuorumVerifier getCurrentConfig()
+    {
+        return facade.getCurrentConfig();
+    }
+
+    @Override
+    public SchemaSet getSchemaSet()
+    {
+        return facade.getSchemaSet();
+    }
+
+    @Override
+    public boolean isZk34CompatibilityMode()
+    {
+        return facade.isZk34CompatibilityMode();
+    }
+
+    @Override
+    public CompletableFuture<Void> runSafe(Runnable runnable)
+    {
+        return facade.runSafe(runnable);
+    }
+}
diff --git a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java b/curator-v2/src/main/java/org/apache/curator/framework/imps/WatchesBuilderImpl.java
similarity index 50%
copy from curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
copy to curator-v2/src/main/java/org/apache/curator/framework/imps/WatchesBuilderImpl.java
index 42279d0..a840ffe 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
+++ b/curator-v2/src/main/java/org/apache/curator/framework/imps/WatchesBuilderImpl.java
@@ -16,16 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.utils;
 
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.api.CuratorWatcher;
+import org.apache.curator.framework.api.AddWatchBuilder;
+import org.apache.curator.framework.api.WatchesBuilder;
 import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.WatcherType;
 
-public class DefaultZookeeperFactory implements ZookeeperFactory
+public class WatchesBuilderImpl extends RemoveWatchesBuilderImpl implements WatchesBuilder
 {
+    public WatchesBuilderImpl(CuratorFrameworkImpl client)
+    {
+        super(client);
+    }
+
+    public WatchesBuilderImpl(CuratorFrameworkImpl client, Watcher watcher, CuratorWatcher curatorWatcher, WatcherType watcherType, boolean guaranteed, boolean local, boolean quietly, Backgrounding backgrounding)
+    {
+        super(client, watcher, curatorWatcher, watcherType, guaranteed, local, quietly, backgrounding);
+    }
+
     @Override
-    public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) throws Exception
+    public AddWatchBuilder add()
     {
-        return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly);
+        return new AddWatchBuilderImpl(getClient());
     }
-}
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/v2/AsyncCuratorFrameworkV2.java b/curator-v2/src/main/java/org/apache/curator/v2/AsyncCuratorFrameworkV2.java
new file mode 100644
index 0000000..745322c
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/v2/AsyncCuratorFrameworkV2.java
@@ -0,0 +1,69 @@
+/**
+ * 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.v2;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.curator.x.async.AsyncCuratorFramework;
+import org.apache.curator.x.async.WatchMode;
+import org.apache.curator.x.async.api.AsyncCuratorFrameworkDslV2;
+import org.apache.curator.x.async.details.AsyncCuratorFrameworkV2Impl;
+import org.apache.zookeeper.WatchedEvent;
+import java.util.function.UnaryOperator;
+
+public interface AsyncCuratorFrameworkV2 extends AsyncCuratorFramework, AsyncCuratorFrameworkDslV2
+{
+    /**
+     * Wrap a CuratorFramework instance to gain access to newer ZooKeeper features
+     *
+     * @param client client instance
+     * @return wrapped client
+     */
+    static AsyncCuratorFrameworkV2 wrap(CuratorFramework client)
+    {
+        return new AsyncCuratorFrameworkV2Impl(AsyncCuratorFramework.wrap(client));
+    }
+
+    /**
+     * Wrap a AsyncCuratorFramework instance to gain access to newer ZooKeeper features
+     *
+     * @param client client instance
+     * @return wrapped client
+     */
+    static AsyncCuratorFrameworkV2 wrap(AsyncCuratorFramework client)
+    {
+        return new AsyncCuratorFrameworkV2Impl(client);
+    }
+
+    @Override
+    AsyncCuratorFrameworkDslV2 with(WatchMode mode);
+
+    @Override
+    AsyncCuratorFrameworkDslV2 with(UnhandledErrorListener listener);
+
+    @Override
+    AsyncCuratorFrameworkDslV2 with(UnaryOperator<CuratorEvent> resultFilter, UnaryOperator<WatchedEvent> watcherFilter);
+
+    @Override
+    AsyncCuratorFrameworkDslV2 with(UnhandledErrorListener listener, UnaryOperator<CuratorEvent> resultFilter, UnaryOperator<WatchedEvent> watcherFilter);
+
+    @Override
+    AsyncCuratorFrameworkDslV2 with(WatchMode mode, UnhandledErrorListener listener, UnaryOperator<CuratorEvent> resultFilter, UnaryOperator<WatchedEvent> watcherFilter);
+}
diff --git a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java b/curator-v2/src/main/java/org/apache/curator/v2/CuratorFrameworkV2.java
similarity index 51%
copy from curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
copy to curator-v2/src/main/java/org/apache/curator/v2/CuratorFrameworkV2.java
index 42279d0..a83bb46 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
+++ b/curator-v2/src/main/java/org/apache/curator/v2/CuratorFrameworkV2.java
@@ -16,16 +16,35 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.utils;
+package org.apache.curator.v2;
 
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.WatchesBuilder;
+import org.apache.curator.framework.imps.CuratorFrameworkV2Impl;
 
-public class DefaultZookeeperFactory implements ZookeeperFactory
+public interface CuratorFrameworkV2 extends CuratorFramework
 {
-    @Override
-    public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) throws Exception
+    /**
+     * Wrap a CuratorFramework instance to gain access to newer ZooKeeper features
+     *
+     * @param client client instance
+     * @return wrapped client
+     */
+    static CuratorFrameworkV2 wrap(CuratorFramework client)
     {
-        return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly);
+        return new CuratorFrameworkV2Impl(client);
     }
+
+    /**
+     * Start a watches builder
+     *
+     * @return builder
+     */
+    WatchesBuilder watches();
+
+    @Override
+    CuratorFrameworkV2 usingNamespace(String newNamespace);
+
+    @Override
+    WatcherRemoveCuratorFrameworkV2 newWatcherRemoveCuratorFramework();
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-v2/src/main/java/org/apache/curator/v2/WatcherRemoveCuratorFrameworkV2.java
similarity index 79%
copy from curator-test/src/main/java/org/apache/curator/test/Compatibility.java
copy to curator-v2/src/main/java/org/apache/curator/v2/WatcherRemoveCuratorFrameworkV2.java
index 5b4b53f..ba0b61c 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-v2/src/main/java/org/apache/curator/v2/WatcherRemoveCuratorFrameworkV2.java
@@ -16,12 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.test;
+package org.apache.curator.v2;
 
-public class Compatibility
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
+
+public interface WatcherRemoveCuratorFrameworkV2 extends WatcherRemoveCuratorFramework, CuratorFrameworkV2
 {
-    public static boolean isZK34()
-    {
-        return false;
-    }
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java b/curator-v2/src/main/java/org/apache/curator/x/async/api/AsyncCuratorFrameworkDslV2.java
similarity index 77%
copy from curator-test/src/main/java/org/apache/curator/test/Compatibility.java
copy to curator-v2/src/main/java/org/apache/curator/x/async/api/AsyncCuratorFrameworkDslV2.java
index 5b4b53f..ae851a8 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Compatibility.java
+++ b/curator-v2/src/main/java/org/apache/curator/x/async/api/AsyncCuratorFrameworkDslV2.java
@@ -16,12 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.test;
+package org.apache.curator.x.async.api;
 
-public class Compatibility
+public interface AsyncCuratorFrameworkDslV2 extends AsyncCuratorFrameworkDsl
 {
-    public static boolean isZK34()
-    {
-        return false;
-    }
+    /**
+     * Start an add watch builder
+     *
+     * @return builder object
+     */
+    AsyncWatchBuilder addWatch();
 }
diff --git a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java b/curator-v2/src/main/java/org/apache/curator/x/async/api/AsyncWatchBuilder.java
similarity index 60%
copy from curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
copy to curator-v2/src/main/java/org/apache/curator/x/async/api/AsyncWatchBuilder.java
index 42279d0..73172c3 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java
+++ b/curator-v2/src/main/java/org/apache/curator/x/async/api/AsyncWatchBuilder.java
@@ -16,16 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.utils;
 
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
+package org.apache.curator.x.async.api;
 
-public class DefaultZookeeperFactory implements ZookeeperFactory
+import org.apache.curator.framework.api.AddWatchable;
+import org.apache.curator.x.async.AsyncStage;
+import org.apache.zookeeper.AddWatchMode;
+
+public interface AsyncWatchBuilder extends AddWatchable<AsyncPathable<AsyncStage<Void>>>
 {
-    @Override
-    public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) throws Exception
-    {
-        return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly);
-    }
-}
+    /**
+     * The mode to use. By default, {@link org.apache.zookeeper.AddWatchMode#PERSISTENT_RECURSIVE} is used
+     *
+     * @param mode mode
+     * @return this
+     */
+    AddWatchable<AsyncPathable<AsyncStage<Void>>> withMode(AddWatchMode mode);
+}
\ No newline at end of file
diff --git a/curator-v2/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkV2Impl.java b/curator-v2/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkV2Impl.java
new file mode 100644
index 0000000..210a79f
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkV2Impl.java
@@ -0,0 +1,189 @@
+/**
+ * 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.CuratorFramework;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.curator.v2.AsyncCuratorFrameworkV2;
+import org.apache.curator.x.async.AsyncCuratorFramework;
+import org.apache.curator.x.async.WatchMode;
+import org.apache.curator.x.async.api.*;
+import org.apache.zookeeper.WatchedEvent;
+import java.util.Objects;
+import java.util.function.UnaryOperator;
+
+public class AsyncCuratorFrameworkV2Impl implements AsyncCuratorFrameworkV2
+{
+    private final AsyncCuratorFrameworkImpl client;
+
+    public AsyncCuratorFrameworkV2Impl(AsyncCuratorFramework client)
+    {
+        this(reveal(client));
+    }
+
+    private AsyncCuratorFrameworkV2Impl(AsyncCuratorFrameworkImpl client)
+    {
+        this.client = client;
+    }
+
+    private static AsyncCuratorFrameworkImpl reveal(Object client)
+    {
+        try
+        {
+            return (AsyncCuratorFrameworkImpl)Objects.requireNonNull(client, "client cannot be null");
+        }
+        catch ( Exception e )
+        {
+            throw new IllegalArgumentException("Only AsyncCuratorFramework clients wrapped via AsyncCuratorFramework.wrap(): " + client.getClass().getName());
+        }
+    }
+
+    @Override
+    public CuratorFramework unwrap()
+    {
+        return client.unwrap();
+    }
+
+    @Override
+    public AsyncCuratorFrameworkDslV2 with(WatchMode mode)
+    {
+        return new AsyncCuratorFrameworkV2Impl(reveal(client.with(mode)));
+    }
+
+    @Override
+    public AsyncCuratorFrameworkDslV2 with(UnhandledErrorListener listener)
+    {
+        return new AsyncCuratorFrameworkV2Impl(reveal(client.with(listener)));
+    }
+
+    @Override
+    public AsyncCuratorFrameworkDslV2 with(UnaryOperator<CuratorEvent> resultFilter, UnaryOperator<WatchedEvent> watcherFilter)
+    {
+        return new AsyncCuratorFrameworkV2Impl(reveal(client.with(resultFilter, watcherFilter)));
+    }
+
+    @Override
+    public AsyncCuratorFrameworkDslV2 with(UnhandledErrorListener listener, UnaryOperator<CuratorEvent> resultFilter, UnaryOperator<WatchedEvent> watcherFilter)
+    {
+        return new AsyncCuratorFrameworkV2Impl(reveal(client.with(listener, resultFilter, watcherFilter)));
+    }
+
+    @Override
+    public AsyncCuratorFrameworkDslV2 with(WatchMode mode, UnhandledErrorListener listener, UnaryOperator<CuratorEvent> resultFilter, UnaryOperator<WatchedEvent> watcherFilter)
+    {
+        return new AsyncCuratorFrameworkV2Impl(reveal(client.with(mode, listener, resultFilter, watcherFilter)));
+    }
+
+    @Override
+    public AsyncWatchBuilder addWatch()
+    {
+        return new AsyncWatchBuilderImpl(client.getClient(), client.getFilters());
+    }
+
+    @Override
+    public WatchableAsyncCuratorFramework watched()
+    {
+        return client.watched();
+    }
+
+    @Override
+    public AsyncCreateBuilder create()
+    {
+        return client.create();
+    }
+
+    @Override
+    public AsyncDeleteBuilder delete()
+    {
+        return client.delete();
+    }
+
+    @Override
+    public AsyncSetDataBuilder setData()
+    {
+        return client.setData();
+    }
+
+    @Override
+    public AsyncGetACLBuilder getACL()
+    {
+        return client.getACL();
+    }
+
+    @Override
+    public AsyncSetACLBuilder setACL()
+    {
+        return client.setACL();
+    }
+
+    @Override
+    public AsyncReconfigBuilder reconfig()
+    {
+        return client.reconfig();
+    }
+
+    @Override
+    public AsyncMultiTransaction transaction()
+    {
+        return client.transaction();
+    }
+
+    @Override
+    public AsyncTransactionOp transactionOp()
+    {
+        return client.transactionOp();
+    }
+
+    @Override
+    public AsyncSyncBuilder sync()
+    {
+        return client.sync();
+    }
+
+    @Override
+    public AsyncRemoveWatchesBuilder removeWatches()
+    {
+        return client.removeWatches();
+    }
+
+    @Override
+    public AsyncExistsBuilder checkExists()
+    {
+        return client.checkExists();
+    }
+
+    @Override
+    public AsyncGetDataBuilder getData()
+    {
+        return client.getData();
+    }
+
+    @Override
+    public AsyncGetChildrenBuilder getChildren()
+    {
+        return client.getChildren();
+    }
+
+    @Override
+    public AsyncGetConfigBuilder getConfig()
+    {
+        return client.getConfig();
+    }
+}
diff --git a/curator-v2/src/main/java/org/apache/curator/x/async/details/AsyncWatchBuilderImpl.java b/curator-v2/src/main/java/org/apache/curator/x/async/details/AsyncWatchBuilderImpl.java
new file mode 100644
index 0000000..f3c17ad
--- /dev/null
+++ b/curator-v2/src/main/java/org/apache/curator/x/async/details/AsyncWatchBuilderImpl.java
@@ -0,0 +1,77 @@
+/**
+ * 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.api.AddWatchable;
+import org.apache.curator.framework.api.CuratorWatcher;
+import org.apache.curator.framework.imps.AddWatchBuilderImpl;
+import org.apache.curator.framework.imps.CuratorFrameworkImpl;
+import org.apache.curator.framework.imps.Watching;
+import org.apache.curator.x.async.AsyncStage;
+import org.apache.curator.x.async.api.AsyncPathable;
+import org.apache.curator.x.async.api.AsyncWatchBuilder;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.Watcher;
+
+import static org.apache.curator.x.async.details.BackgroundProcs.ignoredProc;
+import static org.apache.curator.x.async.details.BackgroundProcs.safeCall;
+
+class AsyncWatchBuilderImpl implements AsyncWatchBuilder, AddWatchable<AsyncPathable<AsyncStage<Void>>>, AsyncPathable<AsyncStage<Void>>
+{
+    private final CuratorFrameworkImpl client;
+    private final Filters filters;
+    private Watching watching = null;
+    private AddWatchMode mode = AddWatchMode.PERSISTENT_RECURSIVE;
+
+    AsyncWatchBuilderImpl(CuratorFrameworkImpl client, Filters filters)
+    {
+        this.client = client;
+        this.filters = filters;
+    }
+
+    @Override
+    public AddWatchable<AsyncPathable<AsyncStage<Void>>> withMode(AddWatchMode mode)
+    {
+        this.mode = mode;
+        return this;
+    }
+
+    @Override
+    public AsyncPathable<AsyncStage<Void>> usingWatcher(Watcher watcher)
+    {
+        watching = new Watching(client, watcher);
+        return this;
+    }
+
+    @Override
+    public AsyncPathable<AsyncStage<Void>> usingWatcher(CuratorWatcher watcher)
+    {
+        watching = new Watching(client, watcher);
+        return this;
+    }
+
+    @Override
+    public AsyncStage<Void> forPath(String path)
+    {
+        BuilderCommon<Void> common = new BuilderCommon<>(filters, ignoredProc);
+        AddWatchBuilderImpl builder = new AddWatchBuilderImpl(client, watching, common.backgrounding, mode);
+        return safeCall(common.internalCallback, () -> builder.forPath(path));
+    }
+}
\ No newline at end of file
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/TestAsyncCuratorFrameworkV2.java b/curator-v2/src/test/java/org/apache/curator/v2/TestAsyncCuratorFrameworkV2.java
new file mode 100644
index 0000000..1e38929
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/TestAsyncCuratorFrameworkV2.java
@@ -0,0 +1,57 @@
+/**
+ * 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.v2;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.curator.v2.AsyncCuratorFrameworkV2;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.concurrent.CountDownLatch;
+
+public class TestAsyncCuratorFrameworkV2 extends CuratorTestBase
+{
+    @Test
+    public void testPersistentRecursiveWatch() throws Exception
+    {
+        try ( CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1)) )
+        {
+            AsyncCuratorFrameworkV2 async = AsyncCuratorFrameworkV2.wrap(client);
+
+            client.start();
+            client.blockUntilConnected();
+
+            CountDownLatch latch = new CountDownLatch(5);
+            Watcher watcher = event -> latch.countDown();
+            async.addWatch().withMode(AddWatchMode.PERSISTENT_RECURSIVE).usingWatcher(watcher).forPath("/test").toCompletableFuture().get();
+
+            client.create().forPath("/test");
+            client.create().forPath("/test/a");
+            client.create().forPath("/test/a/b");
+            client.create().forPath("/test/a/b/c");
+            client.create().forPath("/test/a/b/c/d");
+
+            Assert.assertTrue(timing.awaitLatch(latch));
+        }
+    }
+}
diff --git a/curator-v2/src/test/java/org/apache/curator/v2/TestFrameworkV2.java b/curator-v2/src/test/java/org/apache/curator/v2/TestFrameworkV2.java
new file mode 100644
index 0000000..4ff02d3
--- /dev/null
+++ b/curator-v2/src/test/java/org/apache/curator/v2/TestFrameworkV2.java
@@ -0,0 +1,57 @@
+/**
+ * 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.v2;
+
+import org.apache.curator.v2.CuratorFrameworkV2;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.compatibility.CuratorTestBase;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.curator.v2.CuratorFrameworkV2.wrap;
+
+public class TestFrameworkV2 extends CuratorTestBase
+{
+    @Test
+    public void testPersistentRecursiveWatch() throws Exception
+    {
+        try ( CuratorFrameworkV2 client = wrap(CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1))) )
+        {
+            client.start();
+            client.blockUntilConnected();
+
+            CountDownLatch latch = new CountDownLatch(5);
+            Watcher watcher = event -> latch.countDown();
+            client.watches().add().withMode(AddWatchMode.PERSISTENT_RECURSIVE).usingWatcher(watcher).forPath("/test");
+
+            client.create().forPath("/test");
+            client.create().forPath("/test/a");
+            client.create().forPath("/test/a/b");
+            client.create().forPath("/test/a/b/c");
+            client.create().forPath("/test/a/b/c/d");
+
+            Assert.assertTrue(timing.awaitLatch(latch));
+        }
+    }
+}
diff --git a/curator-v2/src/test/resources/log4j.properties b/curator-v2/src/test/resources/log4j.properties
new file mode 100644
index 0000000..2a85e0d
--- /dev/null
+++ b/curator-v2/src/test/resources/log4j.properties
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+log4j.rootLogger=ERROR, console
+
+log4j.logger.org.apache.curator=DEBUG, console
+log4j.additivity.org.apache.curator=false
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n
diff --git a/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkImpl.java b/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkImpl.java
index 167cf50..f13d311 100644
--- a/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkImpl.java
+++ b/curator-x-async/src/main/java/org/apache/curator/x/async/details/AsyncCuratorFrameworkImpl.java
@@ -221,6 +221,16 @@ public class AsyncCuratorFrameworkImpl implements AsyncCuratorFramework
         return new AsyncGetConfigBuilderImpl(client, filters, getBuilderWatchMode());
     }
 
+    Filters getFilters()
+    {
+        return filters;
+    }
+
+    CuratorFrameworkImpl getClient()
+    {
+        return client;
+    }
+
     private WatchMode getBuilderWatchMode()
     {
         return watched ? watchMode : null;
diff --git a/pom.xml b/pom.xml
index 3a5e152..731cc84 100644
--- a/pom.xml
+++ b/pom.xml
@@ -60,7 +60,7 @@
         <jdk-version>1.${short-jdk-version}</jdk-version>
 
         <!-- versions -->
-        <zookeeper-version>3.5.5</zookeeper-version>
+        <zookeeper-version>3.5.6</zookeeper-version>
         <maven-bundle-plugin-version>4.1.0</maven-bundle-plugin-version>
         <maven-javadoc-plugin-version>3.0.1</maven-javadoc-plugin-version>
         <doxia-module-confluence-version>1.8</doxia-module-confluence-version>
@@ -85,25 +85,27 @@
         <guava-failureaccess-version>1.0.1</guava-failureaccess-version>
         <testng-version>6.14.3</testng-version>
         <swift-version>0.23.1</swift-version>
-        <dropwizard-version>1.3.7</dropwizard-version>
         <maven-shade-plugin-version>3.2.1</maven-shade-plugin-version>
         <slf4j-version>1.7.25</slf4j-version>
         <clirr-maven-plugin-version>2.8</clirr-maven-plugin-version>
+        <dropwizard-version>3.2.5</dropwizard-version>
+        <snappy-version>1.1.7</snappy-version>
 
         <!-- OSGi Properties -->
-        <osgi.export.package />
-        <osgi.import.package />
-        <osgi.private.package />
-        <osgi.dynamic.import />
-        <osgi.require.bundle />
-        <osgi.export.service />
-        <osgi.activator />
+        <osgi.export.package/>
+        <osgi.import.package/>
+        <osgi.private.package/>
+        <osgi.dynamic.import/>
+        <osgi.require.bundle/>
+        <osgi.export.service/>
+        <osgi.activator/>
     </properties>
 
     <scm>
         <url>https://github.com/apache/curator.git</url>
         <connection>scm:git:https://gitbox.apache.org/repos/asf/curator.git</connection>
-        <developerConnection>scm:git:https://gitbox.apache.org/repos/asf/curator.git</developerConnection>
+        <developerConnection>scm:git:https://gitbox.apache.org/repos/asf/curator.git
+        </developerConnection>
         <tag>apache-curator-3.2.0</tag>
     </scm>
 
@@ -318,6 +320,7 @@
         <module>curator-x-discovery-server</module>
         <module>curator-x-async</module>
         <module>curator-test-zk34</module>
+        <module>curator-v2</module>
     </modules>
 
     <dependencyManagement>
@@ -567,6 +570,24 @@
                 <artifactId>dropwizard-logging</artifactId>
                 <version>${dropwizard-version}</version>
             </dependency>
+
+            <dependency>
+                <groupId>io.dropwizard.metrics</groupId>
+                <artifactId>metrics-core</artifactId>
+                <version>${dropwizard-version}</version>
+                <exclusions>
+                    <exclusion>
+                        <groupId>org.slf4j</groupId>
+                        <artifactId>slf4j-api</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+
+            <dependency>
+                <groupId>org.xerial.snappy</groupId>
+                <artifactId>snappy-java</artifactId>
+                <version>${snappy-version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 
@@ -582,7 +603,6 @@
                 <artifactId>maven-javadoc-plugin</artifactId>
                 <version>${maven-javadoc-plugin-version}</version>
                 <configuration>
-                    <aggregate>true</aggregate>
                     <additionalJOptions>
                         <additionalJOption>-J-Xmx1g</additionalJOption>
                     </additionalJOptions>
@@ -872,7 +892,8 @@
                             <relocations>
                                 <relocation>
                                     <pattern>com.google</pattern>
-                                    <shadedPattern>org.apache.curator.shaded.com.google</shadedPattern>
+                                    <shadedPattern>org.apache.curator.shaded.com.google
+                                    </shadedPattern>
                                     <excludes>
                                         <exclude>com.google.common.base.Function</exclude>
                                         <exclude>com.google.common.base.Predicate</exclude>
diff --git a/src/site/confluence/zk-compatibility.confluence b/src/site/confluence/zk-compatibility.confluence
index ed6e32e..cd65784 100644
--- a/src/site/confluence/zk-compatibility.confluence
+++ b/src/site/confluence/zk-compatibility.confluence
@@ -1,15 +1,46 @@
 h1. ZooKeeper Version Compatibility
 
-While ZooKeeper 3.5.x is still considered "beta" by the ZooKeeper development team, the reality is that it is
-used in production by many users. However, ZooKeeper 3.4.x is also used in production. Prior to Apache Curator
-4.0, both versions of ZooKeeper were supported via two versions of Apache Curator. Starting with Curator 4.0
-both versions of ZooKeeper are supported via the same Curator libraries.
+There are multiple active version lines of ZooKeeper used in production by users. Curator can act
+as a client for any of these versions.
 
 h2. ZooKeeper 3.5.x
 
-* Curator 4.0 has a hard dependency on ZooKeeper 3.5.x
+* Curator 4.x has a hard dependency on ZooKeeper 3.5.x
 * If you are using ZooKeeper 3.5.x there's nothing additional to do \- just use Curator 4.0
 
+h2. ZooKeeper 3.6.x
+
+ZooKeeper 3.6.x is the newest version of ZooKeeper which adds new features such as
+Persistent/Recursive watchers. To use Curator
+X.X with ZooKeeper 3.6.x use the dependency
+{{curator-v2}} instead of {{curator-recipes}}. Create a {{CuratorFramework}} instance in the
+normal manner. To get access to new features, wrap the CuratorFramework in a CuratorFrameworkV2
+instances. E.g.
+
+{code}
+CuratorFramework client = CuratorFrameworkFactory...
+CuratorFrameworkV2 clientV2 = CuratorFrameworkV2.wrap(client);
+
+client.watches().add()...
+{code}
+
+_Maven_
+
+{code}
+<dependency>
+    <groupId>org.apache.curator</groupId>
+    <artifactId>curator-v2</artifactId>
+    <version>${curator-version}</version>
+</dependency>
+{code}
+
+_Gradle_
+
+{code}
+compile 'org.apache.curator:curator-v2:$curatorVersion'
+{code}
+
+
 h2. ZooKeeper 3.4.x
 
 Curator 4.0 supports ZooKeeper 3.4.x ensembles in a soft\-compatibility mode. To use this mode