Merge branch 'master' into CURATOR-3.0
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 9a12344..ec2c504 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-client</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
     <packaging>bundle</packaging>
 
     <name>Curator Client</name>
diff --git a/curator-client/src/main/java/org/apache/curator/ConnectionState.java b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
index dc6ac53..d79ec58 100644
--- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
+++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
@@ -18,9 +18,10 @@
  */
 package org.apache.curator;
 
-import org.apache.curator.utils.CloseableUtils;
+import org.apache.curator.connection.ConnectionHandlingPolicy;
 import org.apache.curator.drivers.TracerDriver;
 import org.apache.curator.ensemble.EnsembleProvider;
+import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.DebugUtils;
 import org.apache.curator.utils.ThreadUtils;
 import org.apache.curator.utils.ZookeeperFactory;
@@ -33,8 +34,10 @@
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Queue;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -45,21 +48,24 @@
     private static final Logger log = LoggerFactory.getLogger(ConnectionState.class);
     private final HandleHolder zooKeeper;
     private final AtomicBoolean isConnected = new AtomicBoolean(false);
+    private final AtomicInteger lastNegotiatedSessionTimeoutMs = new AtomicInteger(0);
     private final EnsembleProvider ensembleProvider;
     private final int sessionTimeoutMs;
     private final int connectionTimeoutMs;
     private final AtomicReference<TracerDriver> tracer;
+    private final ConnectionHandlingPolicy connectionHandlingPolicy;
     private final Queue<Exception> backgroundExceptions = new ConcurrentLinkedQueue<Exception>();
     private final Queue<Watcher> parentWatchers = new ConcurrentLinkedQueue<Watcher>();
     private final AtomicLong instanceIndex = new AtomicLong();
     private volatile long connectionStartMs = 0;
 
-    ConnectionState(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher parentWatcher, AtomicReference<TracerDriver> tracer, boolean canBeReadOnly)
+    ConnectionState(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher parentWatcher, AtomicReference<TracerDriver> tracer, boolean canBeReadOnly, ConnectionHandlingPolicy connectionHandlingPolicy)
     {
         this.ensembleProvider = ensembleProvider;
         this.sessionTimeoutMs = sessionTimeoutMs;
         this.connectionTimeoutMs = connectionTimeoutMs;
         this.tracer = tracer;
+        this.connectionHandlingPolicy = connectionHandlingPolicy;
         if ( parentWatcher != null )
         {
             parentWatchers.offer(parentWatcher);
@@ -139,6 +145,11 @@
         return instanceIndex.get();
     }
 
+    int getLastNegotiatedSessionTimeoutMs()
+    {
+        return lastNegotiatedSessionTimeoutMs.get();
+    }
+
     @Override
     public void process(WatchedEvent event)
     {
@@ -155,6 +166,11 @@
             {
                 isConnected.set(newIsConnected);
                 connectionStartMs = System.currentTimeMillis();
+                if ( newIsConnected )
+                {
+                    lastNegotiatedSessionTimeoutMs.set(zooKeeper.getNegotiatedSessionTimeoutMs());
+                    log.debug("Negotiated session timeout: " + lastNegotiatedSessionTimeoutMs.get());
+                }
             }
         }
 
@@ -171,42 +187,7 @@
         return ensembleProvider;
     }
 
-    private synchronized void checkTimeouts() throws Exception
-    {
-        int minTimeout = Math.min(sessionTimeoutMs, connectionTimeoutMs);
-        long elapsed = System.currentTimeMillis() - connectionStartMs;
-        if ( elapsed >= minTimeout )
-        {
-            if ( zooKeeper.hasNewConnectionString() )
-            {
-                handleNewConnectionString();
-            }
-            else
-            {
-                int maxTimeout = Math.max(sessionTimeoutMs, connectionTimeoutMs);
-                if ( elapsed > maxTimeout )
-                {
-                    if ( !Boolean.getBoolean(DebugUtils.PROPERTY_DONT_LOG_CONNECTION_ISSUES) )
-                    {
-                        log.warn(String.format("Connection attempt unsuccessful after %d (greater than max timeout of %d). Resetting connection and trying again with a new connection.", elapsed, maxTimeout));
-                    }
-                    reset();
-                }
-                else
-                {
-                    KeeperException.ConnectionLossException connectionLossException = new CuratorConnectionLossException();
-                    if ( !Boolean.getBoolean(DebugUtils.PROPERTY_DONT_LOG_CONNECTION_ISSUES) )
-                    {
-                        log.error(String.format("Connection timed out for connection string (%s) and timeout (%d) / elapsed (%d)", zooKeeper.getConnectionString(), connectionTimeoutMs, elapsed), connectionLossException);
-                    }
-                    tracer.get().addCount("connections-timed-out", 1);
-                    throw connectionLossException;
-                }
-            }
-        }
-    }
-
-    private synchronized void reset() throws Exception
+    synchronized void reset() throws Exception
     {
         log.debug("reset");
 
@@ -218,6 +199,67 @@
         zooKeeper.getZooKeeper();   // initiate connection
     }
 
+    private synchronized void checkTimeouts() throws Exception
+    {
+        final AtomicReference<String> newConnectionString = new AtomicReference<>();
+        Callable<String> hasNewConnectionString = new Callable<String>()
+        {
+            @Override
+            public String call()
+            {
+                newConnectionString.set(zooKeeper.getNewConnectionString());
+                return newConnectionString.get();
+            }
+        };
+        int lastNegotiatedSessionTimeoutMs = getLastNegotiatedSessionTimeoutMs();
+        int useSessionTimeoutMs = (lastNegotiatedSessionTimeoutMs > 0) ? lastNegotiatedSessionTimeoutMs : sessionTimeoutMs;
+        ConnectionHandlingPolicy.CheckTimeoutsResult result = connectionHandlingPolicy.checkTimeouts(hasNewConnectionString, connectionStartMs, useSessionTimeoutMs, connectionTimeoutMs);
+        switch ( result )
+        {
+            default:
+            case NOP:
+            {
+                break;
+            }
+
+            case NEW_CONNECTION_STRING:
+            {
+                handleNewConnectionString(newConnectionString.get());
+                break;
+            }
+
+            case RESET_CONNECTION:
+            {
+                if ( !Boolean.getBoolean(DebugUtils.PROPERTY_DONT_LOG_CONNECTION_ISSUES) )
+                {
+                    long elapsed = System.currentTimeMillis() - connectionStartMs;
+                    int maxTimeout = Math.max(useSessionTimeoutMs, connectionTimeoutMs);
+                    log.warn(String.format("Connection attempt unsuccessful after %d (greater than max timeout of %d). Resetting connection and trying again with a new connection.", elapsed, maxTimeout));
+                }
+                reset();
+                break;
+            }
+
+            case CONNECTION_TIMEOUT:
+            {
+                KeeperException.ConnectionLossException connectionLossException = new CuratorConnectionLossException();
+                if ( !Boolean.getBoolean(DebugUtils.PROPERTY_DONT_LOG_CONNECTION_ISSUES) )
+                {
+                    long elapsed = System.currentTimeMillis() - connectionStartMs;
+                    log.error(String.format("Connection timed out for connection string (%s) and timeout (%d) / elapsed (%d)", zooKeeper.getConnectionString(), connectionTimeoutMs, elapsed), connectionLossException);
+                }
+                tracer.get().addCount("connections-timed-out", 1);
+                throw connectionLossException;
+            }
+
+            case SESSION_TIMEOUT:
+            {
+                handleExpiredSession();
+                break;
+            }
+        }
+    }
+
     private boolean checkState(Event.KeeperState state, boolean wasConnected)
     {
         boolean isConnected = wasConnected;
@@ -260,22 +302,41 @@
         }
         }
 
-        if ( checkNewConnectionString && zooKeeper.hasNewConnectionString() )
+        if ( checkNewConnectionString )
         {
-            handleNewConnectionString();
+            String newConnectionString = zooKeeper.getNewConnectionString();
+            if ( newConnectionString != null )
+            {
+                handleNewConnectionString(newConnectionString);
+            }
         }
 
         return isConnected;
     }
 
-    private void handleNewConnectionString()
+    private void handleNewConnectionString(String newConnectionString)
     {
-        log.info("Connection string changed");
+        log.info("Connection string changed to: " + newConnectionString);
         tracer.get().addCount("connection-string-changed", 1);
 
         try
         {
-            reset();
+            ZooKeeper zooKeeper = this.zooKeeper.getZooKeeper();
+            if ( zooKeeper == null )
+            {
+                log.warn("Could not update the connection string because getZooKeeper() returned null.");
+            }
+            else
+            {
+                if ( ensembleProvider.updateServerListEnabled() )
+                {
+                    zooKeeper.updateServerList(newConnectionString);
+                }
+                else
+                {
+                    reset();
+                }
+            }
         }
         catch ( Exception e )
         {
diff --git a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java
index c9f5199..ad5c55c 100644
--- a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java
+++ b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java
@@ -16,9 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator;
 
 import com.google.common.base.Preconditions;
+import org.apache.curator.connection.ClassicConnectionHandlingPolicy;
+import org.apache.curator.connection.ConnectionHandlingPolicy;
 import org.apache.curator.drivers.TracerDriver;
 import org.apache.curator.ensemble.EnsembleProvider;
 import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
@@ -44,12 +47,13 @@
 @SuppressWarnings("UnusedDeclaration")
 public class CuratorZookeeperClient implements Closeable
 {
-    private final Logger                            log = LoggerFactory.getLogger(getClass());
-    private final ConnectionState                   state;
-    private final AtomicReference<RetryPolicy>      retryPolicy = new AtomicReference<RetryPolicy>();
-    private final int                               connectionTimeoutMs;
-    private final AtomicBoolean                     started = new AtomicBoolean(false);
-    private final AtomicReference<TracerDriver>     tracer = new AtomicReference<TracerDriver>(new DefaultTracerDriver());
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final ConnectionState state;
+    private final AtomicReference<RetryPolicy> retryPolicy = new AtomicReference<RetryPolicy>();
+    private final int connectionTimeoutMs;
+    private final AtomicBoolean started = new AtomicBoolean(false);
+    private final AtomicReference<TracerDriver> tracer = new AtomicReference<TracerDriver>(new DefaultTracerDriver());
+    private final ConnectionHandlingPolicy connectionHandlingPolicy;
 
     /**
      *
@@ -61,7 +65,7 @@
      */
     public CuratorZookeeperClient(String connectString, int sessionTimeoutMs, int connectionTimeoutMs, Watcher watcher, RetryPolicy retryPolicy)
     {
-        this(new DefaultZookeeperFactory(), new FixedEnsembleProvider(connectString), sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, false);
+        this(new DefaultZookeeperFactory(), new FixedEnsembleProvider(connectString), sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, false, new ClassicConnectionHandlingPolicy());
     }
 
     /**
@@ -73,7 +77,7 @@
      */
     public CuratorZookeeperClient(EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher watcher, RetryPolicy retryPolicy)
     {
-        this(new DefaultZookeeperFactory(), ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, false);
+        this(new DefaultZookeeperFactory(), ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, false, new ClassicConnectionHandlingPolicy());
     }
 
     /**
@@ -90,6 +94,26 @@
      */
     public CuratorZookeeperClient(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher watcher, RetryPolicy retryPolicy, boolean canBeReadOnly)
     {
+        this(zookeeperFactory, ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, canBeReadOnly, new ClassicConnectionHandlingPolicy());
+    }
+
+    /**
+     * @param zookeeperFactory factory for creating {@link ZooKeeper} instances
+     * @param ensembleProvider the ensemble provider
+     * @param sessionTimeoutMs session timeout
+     * @param connectionTimeoutMs connection timeout
+     * @param watcher default watcher or null
+     * @param retryPolicy the retry policy to use
+     * @param canBeReadOnly if true, allow ZooKeeper client to enter
+     *                      read only mode in case of a network partition. See
+     *                      {@link ZooKeeper#ZooKeeper(String, int, Watcher, long, byte[], boolean)}
+     *                      for details
+     * @param connectionHandlingPolicy connection handling policy - use one of the pre-defined policies or write your own
+     * @since 3.0.0
+     */
+    public CuratorZookeeperClient(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher watcher, RetryPolicy retryPolicy, boolean canBeReadOnly, ConnectionHandlingPolicy connectionHandlingPolicy)
+    {
+        this.connectionHandlingPolicy = connectionHandlingPolicy;
         if ( sessionTimeoutMs < connectionTimeoutMs )
         {
             log.warn(String.format("session timeout [%d] is less than connection timeout [%d]", sessionTimeoutMs, connectionTimeoutMs));
@@ -99,7 +123,7 @@
         ensembleProvider = Preconditions.checkNotNull(ensembleProvider, "ensembleProvider cannot be null");
 
         this.connectionTimeoutMs = connectionTimeoutMs;
-        state = new ConnectionState(zookeeperFactory, ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, tracer, canBeReadOnly);
+        state = new ConnectionState(zookeeperFactory, ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, tracer, canBeReadOnly, connectionHandlingPolicy);
         setRetryPolicy(retryPolicy);
     }
 
@@ -160,7 +184,7 @@
         Preconditions.checkState(started.get(), "Client is not started");
 
         log.debug("blockUntilConnectedOrTimedOut() start");
-        TimeTrace       trace = startTracer("blockUntilConnectedOrTimedOut");
+        TimeTrace trace = startTracer("blockUntilConnectedOrTimedOut");
 
         internalBlockUntilConnectedOrTimedOut();
 
@@ -177,14 +201,13 @@
      *
      * @throws IOException errors
      */
-    public void     start() throws Exception
+    public void start() throws Exception
     {
         log.debug("Starting");
 
         if ( !started.compareAndSet(false, true) )
         {
-            IllegalStateException ise = new IllegalStateException("Already started");
-            throw ise;
+            throw new IllegalStateException("Already started");
         }
 
         state.start();
@@ -193,7 +216,7 @@
     /**
      * Close the client
      */
-    public void     close()
+    public void close()
     {
         log.debug("Closing");
 
@@ -214,7 +237,7 @@
      *
      * @param policy new policy
      */
-    public void     setRetryPolicy(RetryPolicy policy)
+    public void setRetryPolicy(RetryPolicy policy)
     {
         Preconditions.checkNotNull(policy, "policy cannot be null");
 
@@ -236,7 +259,7 @@
      * @param name name of the event
      * @return the new tracer ({@link TimeTrace#commit()} must be called)
      */
-    public TimeTrace          startTracer(String name)
+    public TimeTrace startTracer(String name)
     {
         return new TimeTrace(name, tracer.get());
     }
@@ -246,7 +269,7 @@
      *
      * @return tracing driver
      */
-    public TracerDriver       getTracerDriver()
+    public TracerDriver getTracerDriver()
     {
         return tracer.get();
     }
@@ -256,7 +279,7 @@
      *
      * @param tracer new tracing driver
      */
-    public void               setTracerDriver(TracerDriver tracer)
+    public void setTracerDriver(TracerDriver tracer)
     {
         this.tracer.set(tracer);
     }
@@ -267,7 +290,7 @@
      *
      * @return connection string
      */
-    public String             getCurrentConnectionString()
+    public String getCurrentConnectionString()
     {
         return state.getEnsembleProvider().getConnectionString();
     }
@@ -283,6 +306,16 @@
     }
 
     /**
+     * For internal use only - reset the internally managed ZK handle
+     *
+     * @throws Exception errors
+     */
+    public void reset() throws Exception
+    {
+        state.reset();
+    }
+
+    /**
      * Every time a new {@link ZooKeeper} instance is allocated, the "instance index"
      * is incremented.
      *
@@ -293,22 +326,47 @@
         return state.getInstanceIndex();
     }
 
-    void        addParentWatcher(Watcher watcher)
+    /**
+     * Return the configured connection handling policy
+     *
+     * @return ConnectionHandlingPolicy
+     */
+    public ConnectionHandlingPolicy getConnectionHandlingPolicy()
+    {
+        return connectionHandlingPolicy;
+    }
+
+    /**
+     * Return the most recent value of {@link ZooKeeper#getSessionTimeout()} or 0
+     *
+     * @return session timeout or 0
+     */
+    public int getLastNegotiatedSessionTimeoutMs()
+    {
+        return state.getLastNegotiatedSessionTimeoutMs();
+    }
+
+    void addParentWatcher(Watcher watcher)
     {
         state.addParentWatcher(watcher);
     }
 
-    void        removeParentWatcher(Watcher watcher)
+    void removeParentWatcher(Watcher watcher)
     {
         state.removeParentWatcher(watcher);
     }
 
-    void internalBlockUntilConnectedOrTimedOut() throws InterruptedException
+    /**
+     * For internal use only
+     *
+     * @throws InterruptedException interruptions
+     */
+    public void internalBlockUntilConnectedOrTimedOut() throws InterruptedException
     {
-        long            waitTimeMs = connectionTimeoutMs;
+        long waitTimeMs = connectionTimeoutMs;
         while ( !state.isConnected() && (waitTimeMs > 0) )
         {
-            final CountDownLatch            latch = new CountDownLatch(1);
+            final CountDownLatch latch = new CountDownLatch(1);
             Watcher tempWatcher = new Watcher()
             {
                 @Override
@@ -317,9 +375,9 @@
                     latch.countDown();
                 }
             };
-            
+
             state.addParentWatcher(tempWatcher);
-            long        startTimeMs = System.currentTimeMillis();
+            long startTimeMs = System.currentTimeMillis();
             try
             {
                 latch.await(1, TimeUnit.SECONDS);
@@ -328,7 +386,7 @@
             {
                 state.removeParentWatcher(tempWatcher);
             }
-            long        elapsed = Math.max(1, System.currentTimeMillis() - startTimeMs);
+            long elapsed = Math.max(1, System.currentTimeMillis() - startTimeMs);
             waitTimeMs -= elapsed;
         }
     }
diff --git a/curator-client/src/main/java/org/apache/curator/HandleHolder.java b/curator-client/src/main/java/org/apache/curator/HandleHolder.java
index 1f7cd91..98b39ce 100644
--- a/curator-client/src/main/java/org/apache/curator/HandleHolder.java
+++ b/curator-client/src/main/java/org/apache/curator/HandleHolder.java
@@ -39,6 +39,8 @@
         ZooKeeper getZooKeeper() throws Exception;
         
         String getConnectionString();
+
+        int getNegotiatedSessionTimeoutMs();
     }
 
     HandleHolder(ZookeeperFactory zookeeperFactory, Watcher watcher, EnsembleProvider ensembleProvider, int sessionTimeout, boolean canBeReadOnly)
@@ -55,15 +57,20 @@
         return (helper != null) ? helper.getZooKeeper() : null;
     }
 
+    int getNegotiatedSessionTimeoutMs()
+    {
+        return (helper != null) ? helper.getNegotiatedSessionTimeoutMs() : 0;
+    }
+
     String  getConnectionString()
     {
         return (helper != null) ? helper.getConnectionString() : null;
     }
 
-    boolean hasNewConnectionString() 
+    String getNewConnectionString()
     {
         String helperConnectionString = (helper != null) ? helper.getConnectionString() : null;
-        return (helperConnectionString != null) && !ensembleProvider.getConnectionString().equals(helperConnectionString);
+        return ((helperConnectionString != null) && !ensembleProvider.getConnectionString().equals(helperConnectionString)) ? helperConnectionString : null;
     }
 
     void closeAndClear() throws Exception
@@ -107,6 +114,12 @@
                         {
                             return connectionString;
                         }
+
+                        @Override
+                        public int getNegotiatedSessionTimeoutMs()
+                        {
+                            return (zooKeeperHandle != null) ? zooKeeperHandle.getSessionTimeout() : 0;
+                        }
                     };
 
                     return zooKeeperHandle;
@@ -118,6 +131,12 @@
             {
                 return connectionString;
             }
+
+            @Override
+            public int getNegotiatedSessionTimeoutMs()
+            {
+                return (zooKeeperHandle != null) ? zooKeeperHandle.getSessionTimeout() : 0;
+            }
         };
     }
 
diff --git a/curator-client/src/main/java/org/apache/curator/RetryLoop.java b/curator-client/src/main/java/org/apache/curator/RetryLoop.java
index 48d4d5c..e58ce20 100644
--- a/curator-client/src/main/java/org/apache/curator/RetryLoop.java
+++ b/curator-client/src/main/java/org/apache/curator/RetryLoop.java
@@ -20,7 +20,6 @@
 
 import org.apache.curator.drivers.TracerDriver;
 import org.apache.curator.utils.DebugUtils;
-import org.apache.curator.utils.ThreadUtils;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -97,24 +96,7 @@
      */
     public static<T> T      callWithRetry(CuratorZookeeperClient client, Callable<T> proc) throws Exception
     {
-        T               result = null;
-        RetryLoop       retryLoop = client.newRetryLoop();
-        while ( retryLoop.shouldContinue() )
-        {
-            try
-            {
-                client.internalBlockUntilConnectedOrTimedOut();
-                
-                result = proc.call();
-                retryLoop.markComplete();
-            }
-            catch ( Exception e )
-            {
-                ThreadUtils.checkInterrupted(e);
-                retryLoop.takeException(e);
-            }
-        }
-        return result;
+        return client.getConnectionHandlingPolicy().callWithRetry(client, proc);
     }
 
     RetryLoop(RetryPolicy retryPolicy, AtomicReference<TracerDriver> tracer)
@@ -152,7 +134,8 @@
         return (rc == KeeperException.Code.CONNECTIONLOSS.intValue()) ||
             (rc == KeeperException.Code.OPERATIONTIMEOUT.intValue()) ||
             (rc == KeeperException.Code.SESSIONMOVED.intValue()) ||
-            (rc == KeeperException.Code.SESSIONEXPIRED.intValue());
+            (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) ||
+            (rc == KeeperException.Code.NEWCONFIGNOQUORUM.intValue());
     }
 
     /**
diff --git a/curator-client/src/main/java/org/apache/curator/SessionFailRetryLoop.java b/curator-client/src/main/java/org/apache/curator/SessionFailRetryLoop.java
index 7fa1275..65c5c97 100644
--- a/curator-client/src/main/java/org/apache/curator/SessionFailRetryLoop.java
+++ b/curator-client/src/main/java/org/apache/curator/SessionFailRetryLoop.java
@@ -181,7 +181,7 @@
 
     static boolean sessionForThreadHasFailed()
     {
-        return (failedSessionThreads.size() > 0) && failedSessionThreads.contains(Thread.currentThread());
+        return failedSessionThreads.contains(Thread.currentThread());
     }
 
     /**
diff --git a/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java
new file mode 100644
index 0000000..fe24b42
--- /dev/null
+++ b/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.connection;
+
+import org.apache.curator.CuratorZookeeperClient;
+import org.apache.curator.RetryLoop;
+import org.apache.curator.utils.ThreadUtils;
+import java.util.concurrent.Callable;
+
+/**
+ * Emulates the pre 3.0.0 Curator connection handling
+ */
+public class ClassicConnectionHandlingPolicy implements ConnectionHandlingPolicy
+{
+    @Override
+    public int getSimulatedSessionExpirationPercent()
+    {
+        return 0;
+    }
+
+    @Override
+    public <T> T callWithRetry(CuratorZookeeperClient client, Callable<T> proc) throws Exception
+    {
+        T result = null;
+        RetryLoop retryLoop = client.newRetryLoop();
+        while ( retryLoop.shouldContinue() )
+        {
+            try
+            {
+                client.internalBlockUntilConnectedOrTimedOut();
+                result = proc.call();
+                retryLoop.markComplete();
+            }
+            catch ( Exception e )
+            {
+                ThreadUtils.checkInterrupted(e);
+                retryLoop.takeException(e);
+            }
+        }
+
+        return result;
+    }
+
+    @Override
+    public CheckTimeoutsResult checkTimeouts(Callable<String> hasNewConnectionString, long connectionStartMs, int sessionTimeoutMs, int connectionTimeoutMs) throws Exception
+    {
+        CheckTimeoutsResult result = CheckTimeoutsResult.NOP;
+        int minTimeout = Math.min(sessionTimeoutMs, connectionTimeoutMs);
+        long elapsed = System.currentTimeMillis() - connectionStartMs;
+        if ( elapsed >= minTimeout )
+        {
+            if ( hasNewConnectionString.call() != null )
+            {
+                result = CheckTimeoutsResult.NEW_CONNECTION_STRING;
+            }
+            else
+            {
+                int maxTimeout = Math.max(sessionTimeoutMs, connectionTimeoutMs);
+                if ( elapsed > maxTimeout )
+                {
+                    result = CheckTimeoutsResult.RESET_CONNECTION;
+                }
+                else
+                {
+                    result = CheckTimeoutsResult.CONNECTION_TIMEOUT;
+                }
+            }
+        }
+
+        return result;
+    }
+}
diff --git a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java
new file mode 100644
index 0000000..8f6a147
--- /dev/null
+++ b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java
@@ -0,0 +1,111 @@
+/**
+ * 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.connection;
+
+import org.apache.curator.CuratorZookeeperClient;
+import org.apache.curator.RetryLoop;
+import java.util.concurrent.Callable;
+
+/**
+ * Abstracts connection handling so that Curator can emulate it's old, pre 3.0.0
+ * handling and update to newer handling.
+ */
+public interface ConnectionHandlingPolicy
+{
+    /**
+     * <p>
+     *     Prior to 3.0.0, Curator did not try to manage session expiration
+     *     other than the functionality provided by ZooKeeper itself. Starting with
+     *     3.0.0, Curator has the option of attempting to monitor session expiration
+     *     above what is provided by ZooKeeper. The percentage returned by this method
+     *     determines how and if Curator will check for session expiration.
+     * </p>
+     *
+     * <p>
+     *     If this method returns <tt>0</tt>, Curator does not
+     *     do any additional checking for session expiration.
+     * </p>
+     *
+     * <p>
+     *     If a positive number is returned, Curator will check for session expiration
+     *     as follows: when ZooKeeper sends a Disconnect event, Curator will start a timer.
+     *     If re-connection is not achieved before the elapsed time exceeds the negotiated
+     *     session time multiplied by the session expiration percent, Curator will simulate
+     *     a session expiration. Due to timing/network issues, it is <b>not possible</b> for
+     *     a client to match the server's session timeout with complete accuracy. Thus, the need
+     *     for a session expiration percentage.
+     * </p>
+     *
+     * @return a percentage from 0 to 100 (0 implied no extra session checking)
+     */
+    int getSimulatedSessionExpirationPercent();
+
+    /**
+     * Called by {@link RetryLoop#callWithRetry(CuratorZookeeperClient, Callable)} to do the work
+     * of retrying
+     *
+     * @param client client
+     * @param proc the procedure to retry
+     * @return result
+     * @throws Exception errors
+     */
+    <T> T callWithRetry(CuratorZookeeperClient client, Callable<T> proc) throws Exception;
+
+    enum CheckTimeoutsResult
+    {
+        /**
+         * Do nothing
+         */
+        NOP,
+
+        /**
+         * handle a new connection string
+         */
+        NEW_CONNECTION_STRING,
+
+        /**
+         * reset/recreate the internal ZooKeeper connection
+         */
+        RESET_CONNECTION,
+
+        /**
+         * handle a connection timeout
+         */
+        CONNECTION_TIMEOUT,
+
+        /**
+         * handle a session timeout
+         */
+        SESSION_TIMEOUT
+    }
+
+    /**
+     * Check timeouts. NOTE: this method is only called when an attempt to access to the ZooKeeper instances
+     * is made and the connection has not completed.
+     *
+     * @param getNewConnectionString proc to call to check if there is a new connection string. Important: the internal state is cleared after
+     *                               this is called so you MUST handle the new connection string if non null is returned
+     * @param connectionStartMs the epoch/ms time that the connection was first initiated
+     * @param sessionTimeoutMs the configured/negotiated session timeout in milliseconds
+     * @param connectionTimeoutMs the configured connection timeout in milliseconds
+     * @return result
+     * @throws Exception errors
+     */
+    CheckTimeoutsResult checkTimeouts(Callable<String> getNewConnectionString, long connectionStartMs, int sessionTimeoutMs, int connectionTimeoutMs) throws Exception;
+}
diff --git a/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java
new file mode 100644
index 0000000..8f7a438
--- /dev/null
+++ b/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java
@@ -0,0 +1,89 @@
+/**
+ * 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.connection;
+
+import com.google.common.base.Preconditions;
+import org.apache.curator.CuratorZookeeperClient;
+import org.apache.curator.RetryLoop;
+import org.apache.curator.utils.ThreadUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.concurrent.Callable;
+
+/**
+ * Curator's standard connection handling since 3.0.0
+ *
+ * @since 3.0.0
+ */
+public class StandardConnectionHandlingPolicy implements ConnectionHandlingPolicy
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final int expirationPercent;
+
+    public StandardConnectionHandlingPolicy()
+    {
+        this(100);
+    }
+
+    public StandardConnectionHandlingPolicy(int expirationPercent)
+    {
+        Preconditions.checkArgument((expirationPercent > 0) && (expirationPercent <= 100), "expirationPercent must be > 0 and <= 100");
+        this.expirationPercent = expirationPercent;
+    }
+
+    @Override
+    public int getSimulatedSessionExpirationPercent()
+    {
+        return expirationPercent;
+    }
+
+    @Override
+    public <T> T callWithRetry(CuratorZookeeperClient client, Callable<T> proc) throws Exception
+    {
+        client.internalBlockUntilConnectedOrTimedOut();
+
+        T result = null;
+        RetryLoop retryLoop = client.newRetryLoop();
+        while ( retryLoop.shouldContinue() )
+        {
+            try
+            {
+                result = proc.call();
+                retryLoop.markComplete();
+            }
+            catch ( Exception e )
+            {
+                ThreadUtils.checkInterrupted(e);
+                retryLoop.takeException(e);
+            }
+        }
+
+        return result;
+    }
+
+    @Override
+    public CheckTimeoutsResult checkTimeouts(Callable<String> hasNewConnectionString, long connectionStartMs, int sessionTimeoutMs, int connectionTimeoutMs) throws Exception
+    {
+        if ( hasNewConnectionString.call() != null )
+        {
+            return CheckTimeoutsResult.NEW_CONNECTION_STRING;
+        }
+        return CheckTimeoutsResult.NOP;
+    }
+}
diff --git a/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java b/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java
index b118294..4db8348 100644
--- a/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java
+++ b/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java
@@ -51,4 +51,18 @@
      * @throws IOException errors
      */
     public void         close() throws IOException;
+
+    /**
+     * A new connection string event was received
+     *
+     * @param connectionString the new connection string
+     */
+    public void setConnectionString(String connectionString);
+
+    /**
+     * Return true if this ensemble provider supports {@link ZooKeeper#updateServerList(String)}
+     *
+     * @return true/false
+     */
+    public boolean updateServerListEnabled();
 }
diff --git a/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java b/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java
index 9c3ec34..eb4f8bd 100644
--- a/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java
+++ b/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java
@@ -116,19 +116,19 @@
         Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Cannot be started more than once");
 
         service.scheduleWithFixedDelay
-        (
-            new Runnable()
-            {
-                @Override
-                public void run()
+            (
+                new Runnable()
                 {
-                    poll();
-                }
-            },
-            pollingMs,
-            pollingMs,
-            TimeUnit.MILLISECONDS
-        );
+                    @Override
+                    public void run()
+                    {
+                        poll();
+                    }
+                },
+                pollingMs,
+                pollingMs,
+                TimeUnit.MILLISECONDS
+            );
     }
 
     @Override
@@ -145,6 +145,18 @@
         return connectionString.get();
     }
 
+    @Override
+    public void setConnectionString(String connectionString)
+    {
+        log.info("setConnectionString received. Ignoring. " + connectionString);
+    }
+
+    @Override
+    public boolean updateServerListEnabled()
+    {
+        return false;
+    }
+
     @VisibleForTesting
     protected void poll()
     {
diff --git a/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java b/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java
index 411c712..5f486f4 100644
--- a/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java
+++ b/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java
@@ -20,14 +20,17 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.curator.ensemble.EnsembleProvider;
+import org.apache.zookeeper.ZooKeeper;
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * Standard ensemble provider that wraps a fixed connection string
  */
 public class FixedEnsembleProvider implements EnsembleProvider
 {
-    private final String connectionString;
+    private final AtomicReference<String> connectionString = new AtomicReference<>();
+    private final boolean updateServerListEnabled;
 
     /**
      * The connection string to use
@@ -36,7 +39,19 @@
      */
     public FixedEnsembleProvider(String connectionString)
     {
-        this.connectionString = Preconditions.checkNotNull(connectionString, "connectionString cannot be null");
+        this(connectionString, true);
+    }
+
+    /**
+     * The connection string to use
+     *
+     * @param connectionString connection string
+     * @param updateServerListEnabled if true, allow Curator to call {@link ZooKeeper#updateServerList(String)}
+     */
+    public FixedEnsembleProvider(String connectionString, boolean updateServerListEnabled)
+    {
+        this.updateServerListEnabled = updateServerListEnabled;
+        this.connectionString.set(Preconditions.checkNotNull(connectionString, "connectionString cannot be null"));
     }
 
     @Override
@@ -52,8 +67,20 @@
     }
 
     @Override
+    public void setConnectionString(String connectionString)
+    {
+        this.connectionString.set(connectionString);
+    }
+
+    @Override
     public String getConnectionString()
     {
-        return connectionString;
+        return connectionString.get();
+    }
+
+    @Override
+    public boolean updateServerListEnabled()
+    {
+        return updateServerListEnabled;
     }
 }
diff --git a/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java b/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
index b098989..beea726 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
@@ -16,14 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.utils;
 
 public class DebugUtils
 {
-    public static final String          PROPERTY_LOG_EVENTS = "curator-log-events";
-    public static final String          PROPERTY_DONT_LOG_CONNECTION_ISSUES = "curator-dont-log-connection-problems";
-    public static final String          PROPERTY_LOG_ONLY_FIRST_CONNECTION_ISSUE_AS_ERROR_LEVEL = "curator-log-only-first-connection-issue-as-error-level";
-    public static final String          PROPERTY_RETRY_FAILED_TESTS = "curator-retry-failed-tests";
+    public static final String PROPERTY_LOG_EVENTS = "curator-log-events";
+    public static final String PROPERTY_DONT_LOG_CONNECTION_ISSUES = "curator-dont-log-connection-problems";
+    public static final String PROPERTY_LOG_ONLY_FIRST_CONNECTION_ISSUE_AS_ERROR_LEVEL = "curator-log-only-first-connection-issue-as-error-level";
+    public static final String PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND = "curator-remove-watchers-in-foreground";
+    public static final String PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY = "curator-validate-namespace-watcher-map-empty";
 
     private DebugUtils()
     {
diff --git a/curator-client/src/test/java/org/apache/curator/TestEnsurePath.java b/curator-client/src/test/java/org/apache/curator/TestEnsurePath.java
index 4fe9561..59c30ac 100644
--- a/curator-client/src/test/java/org/apache/curator/TestEnsurePath.java
+++ b/curator-client/src/test/java/org/apache/curator/TestEnsurePath.java
@@ -32,6 +32,7 @@
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.curator.connection.ClassicConnectionHandlingPolicy;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.zookeeper.ZooKeeper;
@@ -51,6 +52,7 @@
         CuratorZookeeperClient  curator = mock(CuratorZookeeperClient.class);
         RetryPolicy             retryPolicy = new RetryOneTime(1);
         RetryLoop               retryLoop = new RetryLoop(retryPolicy, null);
+        when(curator.getConnectionHandlingPolicy()).thenReturn(new ClassicConnectionHandlingPolicy());
         when(curator.getZooKeeper()).thenReturn(client);
         when(curator.getRetryPolicy()).thenReturn(retryPolicy);
         when(curator.newRetryLoop()).thenReturn(retryLoop);
@@ -76,6 +78,7 @@
         RetryPolicy             retryPolicy = new RetryOneTime(1);
         RetryLoop               retryLoop = new RetryLoop(retryPolicy, null);
         final CuratorZookeeperClient  curator = mock(CuratorZookeeperClient.class);
+        when(curator.getConnectionHandlingPolicy()).thenReturn(new ClassicConnectionHandlingPolicy());
         when(curator.getZooKeeper()).thenReturn(client);
         when(curator.getRetryPolicy()).thenReturn(retryPolicy);
         when(curator.newRetryLoop()).thenReturn(retryLoop);
diff --git a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
index c17b5bf..b33939f 100644
--- a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
+++ b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator;
 
+import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.retry.RetryOneTime;
@@ -34,7 +35,7 @@
     public void     testRetry() throws Exception
     {
         Timing                          timing = new Timing();
-        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1));
+        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3));
         SessionFailRetryLoop            retryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.RETRY);
         retryLoop.start();
         try
@@ -103,7 +104,7 @@
     public void     testRetryStatic() throws Exception
     {
         Timing                          timing = new Timing();
-        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1));
+        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3));
         SessionFailRetryLoop            retryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.RETRY);
         retryLoop.start();
         try
@@ -174,8 +175,8 @@
     @Test
     public void     testBasic() throws Exception
     {
-        Timing                          timing = new Timing();
-        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1));
+        final Timing                          timing = new Timing();
+        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3));
         SessionFailRetryLoop            retryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.FAIL);
         retryLoop.start();
         try
@@ -198,6 +199,8 @@
                                     Assert.assertNull(client.getZooKeeper().exists("/foo/bar", false));
                                     KillSession.kill(client.getZooKeeper(), server.getConnectString());
 
+                                    timing.sleepABit();
+
                                     client.getZooKeeper();
                                     client.blockUntilConnectedOrTimedOut();
                                     Assert.assertNull(client.getZooKeeper().exists("/foo/bar", false));
@@ -230,7 +233,7 @@
     public void     testBasicStatic() throws Exception
     {
         Timing                          timing = new Timing();
-        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1));
+        final CuratorZookeeperClient    client = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3));
         SessionFailRetryLoop            retryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.FAIL);
         retryLoop.start();
         try
diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index 67505ad..9ec9ccc 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-examples</artifactId>
diff --git a/curator-examples/src/main/java/framework/TransactionExamples.java b/curator-examples/src/main/java/framework/TransactionExamples.java
index f559b5a..7ff8064 100644
--- a/curator-examples/src/main/java/framework/TransactionExamples.java
+++ b/curator-examples/src/main/java/framework/TransactionExamples.java
@@ -19,25 +19,24 @@
 package framework;
 
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.api.transaction.CuratorTransaction;
-import org.apache.curator.framework.api.transaction.CuratorTransactionFinal;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.transaction.CuratorOp;
 import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.TestingServer;
 import java.util.Collection;
 
 public class TransactionExamples
 {
     public static Collection<CuratorTransactionResult>      transaction(CuratorFramework client) throws Exception
     {
-        // this example shows how to use ZooKeeper's new transactions
+        // this example shows how to use ZooKeeper's transactions
 
-        Collection<CuratorTransactionResult>    results = client.inTransaction()
-                .create().forPath("/a/path", "some data".getBytes())
-            .and()
-                .setData().forPath("/another/path", "other data".getBytes())
-            .and()
-                .delete().forPath("/yet/another/path")
-            .and()
-                .commit();  // IMPORTANT! The transaction is not submitted until commit() is called
+        CuratorOp createOp = client.transactionOp().create().forPath("/a/path", "some data".getBytes());
+        CuratorOp setDataOp = client.transactionOp().setData().forPath("/another/path", "other data".getBytes());
+        CuratorOp deleteOp = client.transactionOp().delete().forPath("/yet/another/path");
+
+        Collection<CuratorTransactionResult>    results = client.transaction().forOperations(createOp, setDataOp, deleteOp);
 
         for ( CuratorTransactionResult result : results )
         {
@@ -46,33 +45,4 @@
 
         return results;
     }
-
-    /*
-            These next four methods show how to use Curator's transaction APIs in a more
-            traditional - one-at-a-time - manner
-     */
-
-    public static CuratorTransaction        startTransaction(CuratorFramework client)
-    {
-        // start the transaction builder
-        return client.inTransaction();
-    }
-
-    public static CuratorTransactionFinal   addCreateToTransaction(CuratorTransaction transaction) throws Exception
-    {
-        // add a create operation
-        return transaction.create().forPath("/a/path", "some data".getBytes()).and();
-    }
-
-    public static CuratorTransactionFinal   addDeleteToTransaction(CuratorTransaction transaction) throws Exception
-    {
-        // add a delete operation
-        return transaction.delete().forPath("/another/path").and();
-    }
-
-    public static void                      commitTransaction(CuratorTransactionFinal transaction) throws Exception
-    {
-        // commit the transaction
-        transaction.commit();
-    }
 }
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index 3dbbdbb..dff0892 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-framework</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
     <packaging>bundle</packaging>
 
     <name>Curator Framework</name>
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
index b9d67b9..29c5f06 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
@@ -21,12 +21,17 @@
 
 import org.apache.curator.CuratorZookeeperClient;
 import org.apache.curator.framework.api.*;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransaction;
+import org.apache.curator.framework.api.transaction.CuratorOp;
 import org.apache.curator.framework.api.transaction.CuratorTransaction;
+import org.apache.curator.framework.api.transaction.TransactionOp;
 import org.apache.curator.framework.imps.CuratorFrameworkState;
 import org.apache.curator.framework.listen.Listenable;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.framework.state.ConnectionStateErrorPolicy;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
 
 import java.io.Closeable;
 import java.util.concurrent.TimeUnit;
@@ -122,11 +127,42 @@
     public SetACLBuilder setACL();
 
     /**
+     * Start a reconfig builder
+     *
+     * @return builder object
+     */
+    public ReconfigBuilder reconfig();
+
+    /**
+     * Start a getConfig builder
+     *
+     * @return builder object
+     */
+    public GetConfigBuilder getConfig();
+
+    /**
+     * Start a transaction builder
+     *
+     * @return builder object
+     * @deprecated use {@link #transaction()} instead
+     */
+    public CuratorTransaction inTransaction();
+
+    /**
      * Start a transaction builder
      *
      * @return builder object
      */
-    public CuratorTransaction inTransaction();
+    public CuratorMultiTransaction transaction();
+
+    /**
+     * Allocate an operation that can be used with {@link #transaction()}.
+     * NOTE: {@link CuratorOp} instances created by this builder are
+     * reusable.
+     *
+     * @return operation builder
+     */
+    public TransactionOp transactionOp();
 
     /**
      * Perform a sync on the given path - syncs are always in the background
@@ -156,6 +192,12 @@
     public SyncBuilder sync();
 
     /**
+     * Start a remove watches builder.
+     * @return builder object
+     */
+    public RemoveWatchesBuilder watches();
+
+    /**
      * Returns the listenable interface for the Connect State
      *
      * @return listenable
@@ -225,7 +267,11 @@
      * Call this method on watchers you are no longer interested in.
      *
      * @param watcher the watcher
+     * 
+     * @deprecated As of ZooKeeper 3.5 Curators recipes will handle removing watcher references
+     * when they are no longer used.
      */
+    @Deprecated
     public void clearWatcherReferences(Watcher watcher);
         
     /**
@@ -244,4 +290,27 @@
      * @throws InterruptedException If interrupted while waiting
      */
     public void blockUntilConnected() throws InterruptedException;
+
+    /**
+     * Returns a facade of the current instance that tracks
+     * watchers created and allows a one-shot removal of all watchers
+     * via {@link WatcherRemoveCuratorFramework#removeWatchers()}
+     *
+     * @return facade
+     */
+    public WatcherRemoveCuratorFramework newWatcherRemoveCuratorFramework();
+
+    /**
+     * Return the configured error policy
+     *
+     * @return error policy
+     */
+    public ConnectionStateErrorPolicy getConnectionStateErrorPolicy();
+
+    /**
+     * Current maintains a cached view of the Zookeeper quorum config.
+     *
+     * @return the current config
+     */
+    public QuorumVerifier getCurrentConfig();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index 41ff9cd..fedafa4 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -21,6 +21,9 @@
 
 import com.google.common.collect.ImmutableList;
 import org.apache.curator.RetryPolicy;
+import org.apache.curator.connection.ClassicConnectionHandlingPolicy;
+import org.apache.curator.connection.ConnectionHandlingPolicy;
+import org.apache.curator.connection.StandardConnectionHandlingPolicy;
 import org.apache.curator.ensemble.EnsembleProvider;
 import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
 import org.apache.curator.framework.api.ACLProvider;
@@ -31,9 +34,13 @@
 import org.apache.curator.framework.imps.CuratorTempFrameworkImpl;
 import org.apache.curator.framework.imps.DefaultACLProvider;
 import org.apache.curator.framework.imps.GzipCompressionProvider;
+import org.apache.curator.framework.state.ConnectionStateErrorPolicy;
+import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy;
+import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.utils.DefaultZookeeperFactory;
 import org.apache.curator.utils.ZookeeperFactory;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import java.net.InetAddress;
@@ -134,6 +141,8 @@
         private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER;
         private boolean canBeReadOnly = false;
         private boolean useContainerParentsIfAvailable = true;
+        private ConnectionStateErrorPolicy connectionStateErrorPolicy = new StandardConnectionStateErrorPolicy();
+        private ConnectionHandlingPolicy connectionHandlingPolicy = Boolean.getBoolean("curator-use-classic-connection-handling") ? new ClassicConnectionHandlingPolicy() : new StandardConnectionHandlingPolicy();
 
         /**
          * Apply the current values and build a new CuratorFramework
@@ -361,6 +370,66 @@
             return this;
         }
 
+        /**
+         * Set the error policy to use. The default is {@link StandardConnectionStateErrorPolicy}
+         *
+         * @since 3.0.0
+         * @param connectionStateErrorPolicy new error policy
+         * @return this
+         */
+        public Builder connectionStateErrorPolicy(ConnectionStateErrorPolicy connectionStateErrorPolicy)
+        {
+            this.connectionStateErrorPolicy = connectionStateErrorPolicy;
+            return this;
+        }
+
+        /**
+         * <p>
+         *     Change the connection handling policy. The default policy is {@link StandardConnectionHandlingPolicy}.
+         * </p>
+         * <p>
+         *     <strong>IMPORTANT: </strong> StandardConnectionHandlingPolicy has different behavior than the connection
+         *     policy handling prior to version 3.0.0. You can specify that the connection handling be the method
+         *     prior to 3.0.0 by passing in an instance of {@link ClassicConnectionHandlingPolicy} here or by
+         *     setting the command line value "curator-use-classic-connection-handling" to true (e.g. <tt>-Dcurator-use-classic-connection-handling=true</tt>).
+         * </p>
+         * <p>
+         *     Major differences from the older behavior are:
+         * </p>
+         * <ul>
+         *     <li>
+         *         Session/connection timeouts are no longer managed by the low-level client. They are managed
+         *         by the CuratorFramework instance. There should be no noticeable differences.
+         *     </li>
+         *     <li>
+         *         Prior to 3.0.0, each iteration of the retry policy would allow the connection timeout to elapse
+         *         if the connection hadn't yet succeeded. This meant that the true connection timeout was the configured
+         *         value times the maximum retries in the retry policy. This longstanding issue has been address.
+         *         Now, the connection timeout can elapse only once for a single API call.
+         *     </li>
+         *     <li>
+         *         <strong>MOST IMPORTANTLY!</strong> Prior to 3.0.0, {@link ConnectionState#LOST} did not imply
+         *         a lost session (much to the confusion of users). Now,
+         *         Curator will set the LOST state only when it believes that the ZooKeeper session
+         *         has expired. ZooKeeper connections have a session. When the session expires, clients must take appropriate
+         *         action. In Curator, this is complicated by the fact that Curator internally manages the ZooKeeper
+         *         connection. Now, Curator will set the LOST state when any of the following occurs:
+         *         a) ZooKeeper returns a {@link Watcher.Event.KeeperState#Expired} or {@link KeeperException.Code#SESSIONEXPIRED};
+         *         b) Curator closes the internally managed ZooKeeper instance; c) The session timeout
+         *         elapses during a network partition.
+         *     </li>
+         * </ul>
+         *
+         * @param connectionHandlingPolicy the policy
+         * @return this
+         * @since 3.0.0
+         */
+        public Builder connectionHandlingPolicy(ConnectionHandlingPolicy connectionHandlingPolicy)
+        {
+            this.connectionHandlingPolicy = connectionHandlingPolicy;
+            return this;
+        }
+
         public ACLProvider getAclProvider()
         {
             return aclProvider;
@@ -416,6 +485,16 @@
             return useContainerParentsIfAvailable;
         }
 
+        public ConnectionStateErrorPolicy getConnectionStateErrorPolicy()
+        {
+            return connectionStateErrorPolicy;
+        }
+
+        public ConnectionHandlingPolicy getConnectionHandlingPolicy()
+        {
+            return connectionHandlingPolicy;
+        }
+
         @Deprecated
         public String getAuthScheme()
         {
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/WatcherRemoveCuratorFramework.java
similarity index 71%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/WatcherRemoveCuratorFramework.java
index bc8e6bf..871b53c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/WatcherRemoveCuratorFramework.java
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * A CuratorFramework facade that tracks watchers created and allows a one-shot removal of all watchers
+ */
+public interface WatcherRemoveCuratorFramework extends CuratorFramework
 {
+    /**
+     * Remove all outstanding watchers that have been set
+     */
+    void removeWatchers();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java
old mode 100644
new mode 100755
similarity index 82%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java
index bc8e6bf..0becfb9
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java
@@ -18,8 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+public interface ACLCreateModeStatBackgroundPathAndBytesable<T> extends
+    ACLCreateModeBackgroundPathAndBytesable<T>,
+    Statable<ACLCreateModeBackgroundPathAndBytesable<T>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java
similarity index 69%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java
index bc8e6bf..c60f617 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java
@@ -16,10 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * An non-incremental reconfiguration builder.
+ * This builder has access only to the non-incremental reconfiguration methods withMembers, so that we prevent
+ * mixing concepts that can't be used together.
+ */
+public interface AddStatConfigEnsembleable extends
+    Addable<Statable<ConfigureEnsembleable>>,
+    ConfigureEnsembleable,
+    Statable<ConfigureEnsembleable>
 {
+
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/Addable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Addable.java
new file mode 100644
index 0000000..e908f1e
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Addable.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.curator.framework.api;
+
+import java.util.List;
+
+public interface Addable<T>
+{
+    /**
+     * Sets one or more members that are meant to be part of the ensemble.
+     * The expected format is server.[id]=[hostname]:[peer port]:[election port]:[type];[client port]
+     *
+     * @param server The server to add as a member of the ensemble.
+     * @return this
+     */
+    T adding(String... server);
+
+    /**
+     * Sets one or more members that are meant to be part of the ensemble.
+     * The expected format is server.[id]=[hostname]:[peer port]:[election port]:[type];[client port]
+     *
+     * @param servers The server to add as a member of the ensemble.
+     * @return this
+     */
+    T adding(List<String> servers);
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/AsyncReconfigurable.java
similarity index 78%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/AsyncReconfigurable.java
index bc8e6bf..fc7fd57 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/AsyncReconfigurable.java
@@ -18,8 +18,12 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
-{
+public interface AsyncReconfigurable  {
+
+    /**
+     * Sets the configuration version to use.
+     * @param config The version of the configuration.
+     * @throws Exception
+     */
+    void fromConfig(long config) throws Exception;
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java
similarity index 88%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java
index d63281d..c8b323f 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java
@@ -18,8 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface BackgroundEnsembleable<T> extends
+    Backgroundable<Ensembleable<T>>,
+    Ensembleable<T>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java
similarity index 88%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java
index d63281d..cd88739 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java
@@ -18,8 +18,6 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface BackgroundPathableQuietlyable<T> extends BackgroundPathable<T>, Quietly<BackgroundPathable<T>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ConfigureEnsembleable.java
similarity index 75%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/ConfigureEnsembleable.java
index bc8e6bf..8c739bc 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ConfigureEnsembleable.java
@@ -16,10 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+public interface ConfigureEnsembleable extends
+    Ensembleable<byte[]>
 {
+
+    /**
+     * Sets the configuration version to use.
+     * @param config The version of the configuration.
+     * @throws Exception
+     */
+    Ensembleable<byte[]> fromConfig(long config) throws Exception;
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java
new file mode 100755
index 0000000..2ebe63e
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.api;
+
+import org.apache.zookeeper.CreateMode;
+
+public interface CreateBackgroundModeStatACLable extends
+    BackgroundPathAndBytesable<String>,
+    CreateModable<ACLBackgroundPathAndBytesable<String>>,
+    ACLCreateModeBackgroundPathAndBytesable<String>,
+    Statable<CreateBackgroundModeACLable>
+{
+    /**
+     * Causes any parent nodes to get created if they haven't already been
+     *
+     * @return this
+     */
+    public ACLCreateModePathAndBytesable<String>    creatingParentsIfNeeded();
+
+    /**
+     * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
+     * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper.
+     * If the ZooKeeper version you're using does not support containers, the parent nodes
+     * are created as ordinary PERSISTENT nodes.
+     *
+     * @return this
+     */
+    public ACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded();
+
+    /**
+     * <p>
+     *     Hat-tip to https://github.com/sbridges for pointing this out
+     * </p>
+     *
+     * <p>
+     *     It turns out there is an edge case that exists when creating sequential-ephemeral
+     *     nodes. The creation can succeed on the server, but the server can crash before
+     *     the created node name is returned to the client. However, the ZK session is still
+     *     valid so the ephemeral node is not deleted. Thus, there is no way for the client to
+     *     determine what node was created for them.
+     * </p>
+     *
+     * <p>
+     *     Putting the create builder into protected-ephemeral-sequential mode works around this.
+     *     The name of the node that is created is prefixed with a GUID. If node creation fails
+     *     the normal retry mechanism will occur. On the retry, the parent path is first searched
+     *     for a node that has the GUID in it. If that node is found, it is assumed to be the lost
+     *     node that was successfully created on the first try and is returned to the caller.
+     * </p>
+     *
+     * @return this
+     */
+    public ACLPathAndBytesable<String>              withProtectedEphemeralSequential();
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
index 0db2094..564d11b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
@@ -18,68 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
-import org.apache.zookeeper.CreateMode;
-
 public interface CreateBuilder extends
-    BackgroundPathAndBytesable<String>,
-    CreateModable<ACLBackgroundPathAndBytesable<String>>,
-    ACLCreateModeBackgroundPathAndBytesable<String>,
-    Compressible<CreateBackgroundModeACLable>
+    CreateBuilderMain
 {
-    /**
-     * Causes any parent nodes to get created if they haven't already been
-     *
-     * @return this
-     */
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded();
-
-    /**
-     * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
-     * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper.
-     * If the ZooKeeper version you're using does not support containers, the parent nodes
-     * are created as ordinary PERSISTENT nodes.
-     *
-     * @return this
-     */
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded();
-
-    /**
-     * @deprecated this has been generalized to support all create modes. Instead, use:
-     * <pre>
-     *     client.create().withProtection().withMode(CreateMode.PERSISTENT_SEQUENTIAL)...
-     * </pre>
-     * @return this
-     */
-    @Deprecated
-    public ACLPathAndBytesable<String>              withProtectedEphemeralSequential();
-
-    /**
-     * <p>
-     *     Hat-tip to https://github.com/sbridges for pointing this out
-     * </p>
-     *
-     * <p>
-     *     It turns out there is an edge case that exists when creating sequential-ephemeral
-     *     nodes. The creation can succeed on the server, but the server can crash before
-     *     the created node name is returned to the client. However, the ZK session is still
-     *     valid so the ephemeral node is not deleted. Thus, there is no way for the client to
-     *     determine what node was created for them.
-     * </p>
-     *
-     * <p>
-     *     Even without sequential-ephemeral, however, the create can succeed on the sever
-     *     but the client (for various reasons) will not know it.
-     * </p>
-     *
-     * <p>
-     *     Putting the create builder into protection mode works around this.
-     *     The name of the node that is created is prefixed with a GUID. If node creation fails
-     *     the normal retry mechanism will occur. On the retry, the parent path is first searched
-     *     for a node that has the GUID in it. If that node is found, it is assumed to be the lost
-     *     node that was successfully created on the first try and is returned to the caller.
-     * </p>
-     *
-     * @return this
-     */
-    public ACLCreateModeBackgroundPathAndBytesable<String>    withProtection();
+    CreateBuilderMain orSetData();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilderMain.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilderMain.java
new file mode 100644
index 0000000..e550349
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilderMain.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.api;
+
+import org.apache.zookeeper.CreateMode;
+
+public interface CreateBuilderMain extends
+    BackgroundPathAndBytesable<String>,
+    CreateModable<ACLBackgroundPathAndBytesable<String>>,
+    ACLCreateModeBackgroundPathAndBytesable<String>,
+    Compressible<CreateBackgroundModeStatACLable>,
+    Statable<CreateProtectACLCreateModePathAndBytesable<String>>
+{
+    /**
+     * Causes any parent nodes to get created if they haven't already been
+     *
+     * @return this
+     */
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentsIfNeeded();
+
+    /**
+     * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
+     * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper.
+     * If the ZooKeeper version you're using does not support containers, the parent nodes
+     * are created as ordinary PERSISTENT nodes.
+     *
+     * @return this
+     */
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentContainersIfNeeded();
+
+    /**
+     * @deprecated this has been generalized to support all create modes. Instead, use:
+     * <pre>
+     *     client.create().withProtection().withMode(CreateMode.PERSISTENT_SEQUENTIAL)...
+     * </pre>
+     * @return this
+     */
+    @Deprecated
+    public ACLPathAndBytesable<String>              withProtectedEphemeralSequential();
+
+    /**
+     * <p>
+     *     Hat-tip to https://github.com/sbridges for pointing this out
+     * </p>
+     *
+     * <p>
+     *     It turns out there is an edge case that exists when creating sequential-ephemeral
+     *     nodes. The creation can succeed on the server, but the server can crash before
+     *     the created node name is returned to the client. However, the ZK session is still
+     *     valid so the ephemeral node is not deleted. Thus, there is no way for the client to
+     *     determine what node was created for them.
+     * </p>
+     *
+     * <p>
+     *     Even without sequential-ephemeral, however, the create can succeed on the sever
+     *     but the client (for various reasons) will not know it.
+     * </p>
+     *
+     * <p>
+     *     Putting the create builder into protection mode works around this.
+     *     The name of the node that is created is prefixed with a GUID. If node creation fails
+     *     the normal retry mechanism will occur. On the retry, the parent path is first searched
+     *     for a node that has the GUID in it. If that node is found, it is assumed to be the lost
+     *     node that was successfully created on the first try and is returned to the caller.
+     * </p>
+     *
+     * @return this
+     */
+    public ACLCreateModeStatBackgroundPathAndBytesable<String>    withProtection();
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java
new file mode 100755
index 0000000..9e0c840
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateProtectACLCreateModePathAndBytesable.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.api;
+
+import org.apache.zookeeper.CreateMode;
+
+public interface CreateProtectACLCreateModePathAndBytesable<T> extends
+    ProtectACLCreateModePathAndBytesable<T>
+{
+    /**
+     * Causes any parent nodes to get created if they haven't already been
+     *
+     * @return this
+     */
+    public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded();
+
+    /**
+     * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been.
+     * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper.
+     * If the ZooKeeper version you're using does not support containers, the parent nodes
+     * are created as ordinary PERSISTENT nodes.
+     *
+     * @return this
+     */
+    public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded();
+
+    /**
+     * <p>
+     *     Hat-tip to https://github.com/sbridges for pointing this out
+     * </p>
+     *
+     * <p>
+     *     It turns out there is an edge case that exists when creating sequential-ephemeral
+     *     nodes. The creation can succeed on the server, but the server can crash before
+     *     the created node name is returned to the client. However, the ZK session is still
+     *     valid so the ephemeral node is not deleted. Thus, there is no way for the client to
+     *     determine what node was created for them.
+     * </p>
+     *
+     * <p>
+     *     Even without sequential-ephemeral, however, the create can succeed on the sever
+     *     but the client (for various reasons) will not know it.
+     * </p>
+     *
+     * <p>
+     *     Putting the create builder into protection mode works around this.
+     *     The name of the node that is created is prefixed with a GUID. If node creation fails
+     *     the normal retry mechanism will occur. On the retry, the parent path is first searched
+     *     for a node that has the GUID in it. If that node is found, it is assumed to be the lost
+     *     node that was successfully created on the first try and is returned to the caller.
+     * </p>
+     *
+     * @return this
+     */
+    public ACLCreateModeBackgroundPathAndBytesable<String>    withProtection();
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEvent.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEvent.java
index 2a5408c..673613c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEvent.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CuratorEvent.java
@@ -18,6 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
+import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
+import org.apache.zookeeper.OpResult;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
@@ -80,6 +82,11 @@
     public List<ACL> getACLList();
 
     /**
+     * @return any operation results or null
+     */
+    public List<CuratorTransactionResult> getOpResults();
+
+    /**
      * If {@link #getType()} returns {@link CuratorEventType#WATCHED} this will
      * return the WatchedEvent
      *
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 684d11b..5dea211 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
@@ -69,11 +69,31 @@
     SET_ACL,
 
     /**
+     * Corresponds to {@link CuratorFramework#transaction()}
+     */
+    TRANSACTION,
+
+    /**
+     * Corresponds to {@link CuratorFramework#getConfig()}
+     */
+    GET_CONFIG,
+
+    /**
+     * Corresponds to {@link CuratorFramework#reconfig()}
+     */
+    RECONFIG,
+
+    /**
      * Corresponds to {@link Watchable#usingWatcher(Watcher)} or {@link Watchable#watched()}
      */
     WATCHED,
 
     /**
+     * Corresponds to {@link CuratorFramework#watches()} ()}
+     */
+    REMOVE_WATCHES,
+
+    /**
      * Event sent when client is being closed
      */
     CLOSING
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/DataCallbackable.java
similarity index 68%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/DataCallbackable.java
index bc8e6bf..75ded65 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/DataCallbackable.java
@@ -18,8 +18,15 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
-{
+import org.apache.zookeeper.AsyncCallback.DataCallback;
+
+public interface DataCallbackable<T> {
+
+    /**
+     * Passes a callback and a context object to the config/reconfig command.
+     * @param callback  The async callback to use.
+     * @param ctx       An object that will be passed to the callback.
+     * @return this
+     */
+    T usingDataCallback(DataCallback callback, Object ctx);
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilder.java
index 3a3faf7..84fcd8f 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilder.java
@@ -18,6 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
-public interface DeleteBuilder extends Guaranteeable, ChildrenDeletable
+public interface DeleteBuilder extends
+    Quietly<DeleteBuilderMain>,
+    DeleteBuilderMain
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java
similarity index 88%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java
index d63281d..8c68607 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java
@@ -18,8 +18,6 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface DeleteBuilderMain extends GuaranteeableDeletable, ChildrenDeletable
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Ensembleable.java
similarity index 88%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/Ensembleable.java
index d63281d..c8a82fe 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Ensembleable.java
@@ -18,8 +18,7 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
-{
+public interface Ensembleable<T> {
+
+    T forEnsemble() throws Exception;
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/GetConfigBuilder.java
similarity index 80%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/GetConfigBuilder.java
index bc8e6bf..d137f28 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/GetConfigBuilder.java
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+public interface GetConfigBuilder extends
+    Ensembleable<byte[]>,
+    Backgroundable<Ensembleable<byte[]>>,
+    Watchable<BackgroundEnsembleable<byte[]>>,
+    Statable<WatchBackgroundEnsembleable<byte[]>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/Guaranteeable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Guaranteeable.java
index 481911b..bc033ac 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/Guaranteeable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Guaranteeable.java
@@ -18,23 +18,15 @@
  */
 package org.apache.curator.framework.api;
 
-public interface Guaranteeable extends BackgroundVersionable
+public interface Guaranteeable<T>
 {
     /**
-     * <p>
-     *     Solves this edge case: deleting a node can fail due to connection issues. Further,
-     *     if the node was ephemeral, the node will not get auto-deleted as the session is still valid.
-     *     This can wreak havoc with lock implementations.
-     * </p>
-     *
-     * <p>
-     *     When <code>guaranteed</code> is set, Curator will record failed node deletions and
-     *     attempt to delete them in the background until successful. NOTE: you will still get an
-     *     exception when the deletion fails. But, you can be assured that as long as the
-     *     {@link org.apache.curator.framework.CuratorFramework} instance is open attempts will be made to delete the node.
-     * </p>
+     * Solves edge cases where an operation may succeed on the server but connection failure occurs before a
+     * response can be successfully returned to the client.
+     * 
+     * @see org.apache.curator.framework.api.GuaranteeableDeletable 
      *  
      * @return this
      */
-    public ChildrenDeletable guaranteed();
+    public T guaranteed();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/GuaranteeableDeletable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/GuaranteeableDeletable.java
new file mode 100644
index 0000000..7f8139c
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/GuaranteeableDeletable.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.api;
+
+/**
+ * <p>
+ *     Solves this edge case: deleting a node can fail due to connection issues. Further,
+ *     if the node was ephemeral, the node will not get auto-deleted as the session is still valid.
+ *     This can wreak havoc with lock implementations.
+ * </p>
+ *
+ * <p>
+ *     When <code>guaranteed</code> is set, Curator will record failed node deletions and
+ *     attempt to delete them in the background until successful. NOTE: you will still get an
+ *     exception when the deletion fails. But, you can be assured that as long as the
+ *     {@link org.apache.curator.framework.CuratorFramework} instance is open attempts will be made to delete the node.
+ * </p>
+ *  
+ * @return this
+ */
+public interface GuaranteeableDeletable extends Guaranteeable<ChildrenDeletable>, BackgroundVersionable
+{
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/JoinStatConfigEnsembleable.java
similarity index 69%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/JoinStatConfigEnsembleable.java
index bc8e6bf..c20387c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/JoinStatConfigEnsembleable.java
@@ -16,10 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * An incremental reconfiguration builder.
+ * This builder has access only to the incremental reconfiguration methods joining and leaving, so that we prevent
+ * mixing concepts that can't be used together.
+ */
+public interface JoinStatConfigEnsembleable extends
+    Joinable<LeaveStatConfigEnsembleable>,
+    ConfigureEnsembleable,
+    Statable<ConfigureEnsembleable>
 {
+
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/Joinable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Joinable.java
new file mode 100644
index 0000000..5cebe4d
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Joinable.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.curator.framework.api;
+
+import java.util.List;
+
+public interface Joinable<T>
+{
+    /**
+     * Adds one or more servers to joining the ensemble.
+     * The expected format is server.[id]=[hostname]:[peer port]:[election port]:[type];[client port]
+     *
+     * @param server The server joining.
+     * @return this
+     */
+    T joining(String... server);
+
+    /**
+     * Adds one or more servers to joining the ensemble.
+     * The expected format is server.[id]=[hostname]:[peer port]:[election port]:[type];[client port]
+     *
+     * @param servers The servers joining.
+     * @return this
+     */
+    T joining(List<String> servers);
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/LeaveStatConfigEnsembleable.java
similarity index 69%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/LeaveStatConfigEnsembleable.java
index bc8e6bf..b80bd00 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/LeaveStatConfigEnsembleable.java
@@ -16,10 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * An incremental reconfiguration builder.
+ * This builder has access only to the incremental reconfiguration methods joining and leaving, so that we prevent
+ * mixing concepts that can't be used together.
+ */
+public interface LeaveStatConfigEnsembleable extends
+    Leaveable<JoinStatConfigEnsembleable>,
+    ConfigureEnsembleable,
+    Statable<ConfigureEnsembleable>
 {
+
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Leaveable.java
similarity index 68%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/Leaveable.java
index bc8e6bf..6ec3542 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Leaveable.java
@@ -16,10 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+import java.util.List;
+
+public interface Leaveable<T>
 {
+    /**
+     * Sets one or more servers to leaving the ensemble.
+     *
+     * @param server The server ids
+     * @return this
+     */
+    T leaving(String... server);
+
+    /**
+     * Sets one or more servers to leaving the ensemble.
+     *
+     * @param servers The server ids
+     * @return this
+     */
+    T leaving(List<String> servers);
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/Membersable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Membersable.java
new file mode 100644
index 0000000..e1f8d9e
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Membersable.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.curator.framework.api;
+
+import java.util.List;
+
+public interface Membersable<T>
+{
+    /**
+     * Sets one or more members that are meant to be the ensemble.
+     * The expected format is server.[id]=[hostname]:[peer port]:[election port]:[type];[client port]
+     *
+     * @param server The server joining.
+     * @return this
+     */
+    T withNewMembers(String... server);
+
+    /**
+     * Sets one or more members that are meant to be the ensemble.
+     * The expected format is server.[id]=[hostname]:[peer port]:[election port]:[type];[client port]
+     *
+     * @param servers The servers joining.
+     * @return this
+     */
+    T withNewMembers(List<String> servers);
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateModalPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java
similarity index 84%
rename from curator-framework/src/main/java/org/apache/curator/framework/api/CreateModalPathAndBytesable.java
rename to curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java
index 94bfe7e..07638d5 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateModalPathAndBytesable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java
@@ -18,8 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
-public interface CreateModalPathAndBytesable<T> extends
-    CreateModable<PathAndBytesable<T>>,
-    PathAndBytesable<T>
+public interface ProtectACLCreateModeStatPathAndBytesable<T> extends
+    ProtectACLCreateModePathAndBytesable<T>,
+    Statable<ACLBackgroundPathAndBytesable<T>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/Quietly.java
similarity index 88%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/Quietly.java
index d63281d..ad3762f 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/Quietly.java
@@ -18,8 +18,7 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface Quietly<T>
 {
+    public T quietly();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilder.java
similarity index 88%
rename from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
rename to curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilder.java
index d63281d..d8a2cc2 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilder.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface ReconfigBuilder extends
+    ReconfigBuilderMain,
+    Backgroundable<ReconfigBuilderMain>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilderMain.java
similarity index 82%
rename from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
rename to curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilderMain.java
index bc8e6bf..b9d1be3 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilderMain.java
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+public interface ReconfigBuilderMain extends
+    Joinable<LeaveStatConfigEnsembleable>,
+    Leaveable<JoinStatConfigEnsembleable>,
+    Membersable<StatConfigureEnsembleable>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesBuilder.java
similarity index 60%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesBuilder.java
index bc8e6bf..6cc0b05 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesBuilder.java
@@ -1,25 +1,47 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.framework.api;
-
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
-{
-}
+/**

+ * Licensed to the Apache Software Foundation (ASF) under one

+ * or more contributor license agreements.  See the NOTICE file

+ * distributed with this work for additional information

+ * regarding copyright ownership.  The ASF licenses this file

+ * to you under the Apache License, Version 2.0 (the

+ * "License"); you may not use this file except in compliance

+ * with the License.  You may obtain a copy of the License at

+ *

+ *   http://www.apache.org/licenses/LICENSE-2.0

+ *

+ * Unless required by applicable law or agreed to in writing,

+ * software distributed under the License is distributed on an

+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

+ * KIND, either express or implied.  See the License for the

+ * specific language governing permissions and limitations

+ * under the License.

+ */

+package org.apache.curator.framework.api;

+

+import org.apache.zookeeper.Watcher;

+

+/**

+ * Builder to allow watches to be removed 

+ */

+public interface RemoveWatchesBuilder

+{

+    /**

+     * Specify the watcher to be removed

+     * @param watcher

+     * @return

+     */

+    public RemoveWatchesType remove(Watcher watcher);

+    

+    /**

+     * Specify the watcher to be removed

+     * @param watcher

+     * @return

+     */

+    public RemoveWatchesType remove(CuratorWatcher watcher);

+    

+    /**

+     * Specify that all watches should be removed

+     * @return

+     */

+    public RemoveWatchesType removeAll();

+}

diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesLocal.java
similarity index 62%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesLocal.java
index bc8e6bf..4a67470 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesLocal.java
@@ -1,25 +1,35 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.framework.api;
-
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
-{
-}
+/**

+ * Licensed to the Apache Software Foundation (ASF) under one

+ * or more contributor license agreements.  See the NOTICE file

+ * distributed with this work for additional information

+ * regarding copyright ownership.  The ASF licenses this file

+ * to you under the Apache License, Version 2.0 (the

+ * "License"); you may not use this file except in compliance

+ * with the License.  You may obtain a copy of the License at

+ *

+ *   http://www.apache.org/licenses/LICENSE-2.0

+ *

+ * Unless required by applicable law or agreed to in writing,

+ * software distributed under the License is distributed on an

+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

+ * KIND, either express or implied.  See the License for the

+ * specific language governing permissions and limitations

+ * under the License.

+ */

+package org.apache.curator.framework.api;

+

+/**

+ * Builder to allow the specification of whether it is acceptable to remove client side watch information

+ * in the case where ZK cannot be contacted. 

+ */

+public interface RemoveWatchesLocal extends BackgroundPathableQuietlyable<Void>

+{

+   

+    /**

+     * Specify if the client should just remove client side watches if a connection to ZK

+     * is not available. Note that the standard Curator retry loop will not be used in t

+     * @return

+     */

+    public BackgroundPathableQuietlyable<Void> locally();

+    

+}

diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesType.java
similarity index 62%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesType.java
index bc8e6bf..84d8093 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/RemoveWatchesType.java
@@ -1,25 +1,37 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.framework.api;
-
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
-{
-}
+/**

+ * Licensed to the Apache Software Foundation (ASF) under one

+ * or more contributor license agreements.  See the NOTICE file

+ * distributed with this work for additional information

+ * regarding copyright ownership.  The ASF licenses this file

+ * to you under the Apache License, Version 2.0 (the

+ * "License"); you may not use this file except in compliance

+ * with the License.  You may obtain a copy of the License at

+ *

+ *   http://www.apache.org/licenses/LICENSE-2.0

+ *

+ * Unless required by applicable law or agreed to in writing,

+ * software distributed under the License is distributed on an

+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

+ * KIND, either express or implied.  See the License for the

+ * specific language governing permissions and limitations

+ * under the License.

+ */

+package org.apache.curator.framework.api;

+

+import org.apache.zookeeper.Watcher.WatcherType;

+

+/**

+ * Builder to allow the specification of whether it is acceptable to remove client side watch information

+ * in the case where ZK cannot be contacted. 

+ */

+public interface RemoveWatchesType extends RemoveWatchesLocal, Guaranteeable<BackgroundPathableQuietlyable<Void>>

+{

+   

+    /**

+     * Specify the type of watcher to be removed.

+     * @param watcherType

+     * @return

+     */

+    public RemoveWatchesLocal ofType(WatcherType watcherType);

+    

+}

diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/StatConfigureEnsembleable.java
similarity index 87%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/StatConfigureEnsembleable.java
index d63281d..8b61ab9 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/StatConfigureEnsembleable.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface StatConfigureEnsembleable extends
+    Statable<ConfigureEnsembleable>,
+    ConfigureEnsembleable
 {
-}
+}
\ No newline at end of file
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java b/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java
index b463af2..3721d4b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java
@@ -24,9 +24,7 @@
 public interface UnhandledErrorListener
 {
     /**
-     * Called when an exception is caught in a background thread, handler, etc. Before this
-     * listener is called, the error will have been logged and a {@link ConnectionState#LOST}
-     * event will have been queued for any {@link ConnectionStateListener}s.
+     * Called when an exception is caught in a background thread, handler, etc.
      *
      * @param message Source message
      * @param e exception
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java
similarity index 86%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java
index d63281d..a0e5a13 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java
@@ -18,8 +18,8 @@
  */
 package org.apache.curator.framework.api;
 
-public interface ACLBackgroundPathable<T> extends
-    ACLable<BackgroundPathable<T>>,
-    BackgroundPathable<T>
+public interface WatchBackgroundEnsembleable<T> extends
+    Watchable<BackgroundEnsembleable<T>>,
+    BackgroundEnsembleable<T>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransaction.java
similarity index 76%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransaction.java
index bc8e6bf..07bf191 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransaction.java
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.api.transaction;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+import org.apache.curator.framework.api.Backgroundable;
+
+public interface CuratorMultiTransaction extends
+    Backgroundable<CuratorMultiTransactionMain>,
+    CuratorMultiTransactionMain
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransactionMain.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransactionMain.java
new file mode 100644
index 0000000..2425f5b
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransactionMain.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.api.transaction;
+
+import org.apache.curator.framework.CuratorFramework;
+import java.util.List;
+
+public interface CuratorMultiTransactionMain
+{
+    /**
+     * Commit the given operations as a single transaction. Create the
+     * operation instances via {@link CuratorFramework#transactionOp()}
+     *
+     * @param operations operations that make up the transaction.
+     * @return result details for foreground operations or <code>null</code> for background operations
+     * @throws Exception errors
+     */
+    List<CuratorTransactionResult> forOperations(CuratorOp... operations) throws Exception;
+
+    /**
+     * Commit the given operations as a single transaction. Create the
+     * operation instances via {@link CuratorFramework#transactionOp()}
+     *
+     * @param operations operations that make up the transaction.
+     * @return result details for foreground operations or <code>null</code> for background operations
+     * @throws Exception errors
+     */
+    List<CuratorTransactionResult> forOperations(List<CuratorOp> operations) throws Exception;
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorOp.java
similarity index 77%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorOp.java
index bc8e6bf..23bc76c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorOp.java
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.api.transaction;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+import org.apache.zookeeper.Op;
+
+/**
+ * Internal representation of a transaction operation
+ */
+public interface CuratorOp
 {
+    Op get();
+
+    TypeAndPath getTypeAndPath();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransaction.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransaction.java
index 3901abf..5d60b5c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransaction.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransaction.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.framework.api.transaction;
 
+import org.apache.curator.framework.CuratorFramework;
 import org.apache.zookeeper.ZooKeeper;
 
 /**
@@ -49,6 +50,8 @@
  *     <b>Important:</b> the operations are not submitted until
  *     {@link CuratorTransactionFinal#commit()} is called.
  * </p>
+ *
+ * @deprecated Use {@link CuratorFramework#transaction()}
  */
 public interface CuratorTransaction
 {
@@ -57,26 +60,26 @@
      *
      * @return builder object
      */
-    public TransactionCreateBuilder create();
+    public TransactionCreateBuilder<CuratorTransactionBridge> create();
 
     /**
      * Start a delete builder in the transaction
      *
      * @return builder object
      */
-    public TransactionDeleteBuilder delete();
+    public TransactionDeleteBuilder<CuratorTransactionBridge> delete();
 
     /**
      * Start a setData builder in the transaction
      *
      * @return builder object
      */
-    public TransactionSetDataBuilder setData();
+    public TransactionSetDataBuilder<CuratorTransactionBridge> setData();
 
     /**
      * Start a check builder in the transaction
-     *ChildData
+     *
      * @return builder object
      */
-    public TransactionCheckBuilder check();
+    public TransactionCheckBuilder<CuratorTransactionBridge> check();
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransactionResult.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransactionResult.java
index 03bbca2..8d8dc2d 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransactionResult.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransactionResult.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.api.transaction;
 
 import com.google.common.base.Predicate;
@@ -27,9 +28,9 @@
 public class CuratorTransactionResult
 {
     private final OperationType type;
-    private final String        forPath;
-    private final String        resultPath;
-    private final Stat          resultStat;
+    private final String forPath;
+    private final String resultPath;
+    private final Stat resultStat;
 
     /**
      * Utility that can be passed to Google Guava to find a particular result. E.g.
@@ -41,7 +42,7 @@
      * @param forPath path
      * @return predicate
      */
-    public static Predicate<CuratorTransactionResult>       ofTypeAndPath(final OperationType type, final String forPath)
+    public static Predicate<CuratorTransactionResult> ofTypeAndPath(final OperationType type, final String forPath)
     {
         return new Predicate<CuratorTransactionResult>()
         {
@@ -73,7 +74,7 @@
 
     /**
      * Returns the path that was passed to the operation when added
-     * 
+     *
      * @return operation input path
      */
     public String getForPath()
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/OperationType.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/OperationType.java
index 56dcd33..c0aec68 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/OperationType.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/OperationType.java
@@ -24,22 +24,22 @@
 public enum OperationType
 {
     /**
-     * {@link CuratorTransaction#create()}
+     * {@link TransactionOp#create()}
      */
     CREATE,
 
     /**
-     * {@link CuratorTransaction#delete()}
+     * {@link TransactionOp#delete()}
      */
     DELETE,
 
     /**
-     * {@link CuratorTransaction#setData()}
+     * {@link TransactionOp#setData()}
      */
     SET_DATA,
 
     /**
-     * {@link CuratorTransaction#check()}
+     * {@link TransactionOp#check()}
      */
     CHECK
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCheckBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCheckBuilder.java
index 2bc13d1..6de675c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCheckBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCheckBuilder.java
@@ -21,8 +21,8 @@
 import org.apache.curator.framework.api.Pathable;
 import org.apache.curator.framework.api.Versionable;
 
-public interface TransactionCheckBuilder extends
-    Pathable<CuratorTransactionBridge>,
-    Versionable<Pathable<CuratorTransactionBridge>>
+public interface TransactionCheckBuilder<T> extends
+    Pathable<T>,
+    Versionable<Pathable<T>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
index f2ac146..f7b2850 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCreateBuilder.java
@@ -24,10 +24,11 @@
 import org.apache.curator.framework.api.CreateModable;
 import org.apache.curator.framework.api.PathAndBytesable;
 
-public interface TransactionCreateBuilder extends
-    PathAndBytesable<CuratorTransactionBridge>,
-    CreateModable<ACLPathAndBytesable<CuratorTransactionBridge>>,
-    ACLPathAndBytesable<CuratorTransactionBridge>,
-    ACLCreateModePathAndBytesable<CuratorTransactionBridge>,
-    Compressible<ACLCreateModePathAndBytesable<CuratorTransactionBridge>>  {
+public interface TransactionCreateBuilder<T> extends
+    PathAndBytesable<T>,
+    CreateModable<ACLPathAndBytesable<T>>,
+    ACLPathAndBytesable<T>,
+    ACLCreateModePathAndBytesable<T>,
+    Compressible<ACLCreateModePathAndBytesable<T>>
+{
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionDeleteBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionDeleteBuilder.java
index e165394..d977290 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionDeleteBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionDeleteBuilder.java
@@ -21,8 +21,8 @@
 import org.apache.curator.framework.api.Pathable;
 import org.apache.curator.framework.api.Versionable;
 
-public interface TransactionDeleteBuilder extends
-    Pathable<CuratorTransactionBridge>,
-    Versionable<Pathable<CuratorTransactionBridge>>
+public interface TransactionDeleteBuilder<T> extends
+    Pathable<T>,
+    Versionable<Pathable<T>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionOp.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionOp.java
new file mode 100644
index 0000000..84808a1
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionOp.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.api.transaction;
+
+import org.apache.curator.framework.CuratorFramework;
+
+/**
+ * Builds operations that can be committed as a transaction
+ * via {@link CuratorFramework#transaction()}
+ */
+public interface TransactionOp
+{
+    /**
+     * Start a create builder in the transaction
+     *
+     * @return builder object
+     */
+    TransactionCreateBuilder<CuratorOp> create();
+
+    /**
+     * Start a delete builder in the transaction
+     *
+     * @return builder object
+     */
+    TransactionDeleteBuilder<CuratorOp> delete();
+
+    /**
+     * Start a setData builder in the transaction
+     *
+     * @return builder object
+     */
+    TransactionSetDataBuilder<CuratorOp> setData();
+
+    /**
+     * Start a check builder in the transaction
+     *
+     * @return builder object
+     */
+    TransactionCheckBuilder<CuratorOp> check();
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
index cc9e01c..42a0820 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionSetDataBuilder.java
@@ -23,10 +23,10 @@
 import org.apache.curator.framework.api.VersionPathAndBytesable;
 import org.apache.curator.framework.api.Versionable;
 
-public interface TransactionSetDataBuilder extends
-    PathAndBytesable<CuratorTransactionBridge>,
-    Versionable<PathAndBytesable<CuratorTransactionBridge>>,
-    VersionPathAndBytesable<CuratorTransactionBridge>,
-    Compressible<VersionPathAndBytesable<CuratorTransactionBridge>>
+public interface TransactionSetDataBuilder<T> extends
+    PathAndBytesable<T>,
+    Versionable<PathAndBytesable<T>>,
+    VersionPathAndBytesable<T>,
+    Compressible<VersionPathAndBytesable<T>>
 {
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TypeAndPath.java
similarity index 65%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TypeAndPath.java
index bc8e6bf..b1cea95 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TypeAndPath.java
@@ -16,10 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.api.transaction;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+public class TypeAndPath
 {
+    private final OperationType type;
+    private final String forPath;
+
+    public TypeAndPath(OperationType type, String forPath)
+    {
+        this.type = type;
+        this.forPath = forPath;
+    }
+
+    public OperationType getType()
+    {
+        return type;
+    }
+
+    public String getForPath()
+    {
+        return forPath;
+    }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/BackgroundSyncImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/BackgroundSyncImpl.java
index af8e458..f0994e3 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/BackgroundSyncImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/BackgroundSyncImpl.java
@@ -46,7 +46,7 @@
                 public void processResult(int rc, String path, Object ctx)
                 {
                     trace.commit();
-                    CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.SYNC, rc, path, null, ctx, null, null, null, null, null);
+                    CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.SYNC, rc, path, null, ctx, null, null, null, null, null, null);
                     client.processBackgroundOperation(operationAndData, event);
                 }
             },
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java
new file mode 100644
index 0000000..63ba665
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.state.ConnectionState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ClassicInternalConnectionHandler implements InternalConnectionHandler
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Override
+    public void checkNewConnection(CuratorFrameworkImpl client)
+    {
+        // NOP
+    }
+
+    @Override
+    public void suspendConnection(CuratorFrameworkImpl client)
+    {
+        if ( client.setToSuspended() )
+        {
+            doSyncForSuspendedConnection(client, client.getZookeeperClient().getInstanceIndex());
+        }
+    }
+
+    private void doSyncForSuspendedConnection(final CuratorFrameworkImpl client, final long instanceIndex)
+    {
+        // we appear to have disconnected, force a new ZK event and see if we can connect to another server
+        final BackgroundOperation<String> operation = new BackgroundSyncImpl(client, null);
+        OperationAndData.ErrorCallback<String> errorCallback = new OperationAndData.ErrorCallback<String>()
+        {
+            @Override
+            public void retriesExhausted(OperationAndData<String> operationAndData)
+            {
+                // if instanceIndex != newInstanceIndex, the ZooKeeper instance was reset/reallocated
+                // so the pending background sync is no longer valid.
+                // if instanceIndex is -1, this is the second try to sync - punt and mark the connection lost
+                if ( (instanceIndex < 0) || (instanceIndex == client.getZookeeperClient().getInstanceIndex()) )
+                {
+                    client.addStateChange(ConnectionState.LOST);
+                }
+                else
+                {
+                    log.debug("suspendConnection() failure ignored as the ZooKeeper instance was reset. Retrying.");
+                    // send -1 to signal that if it happens again, punt and mark the connection lost
+                    doSyncForSuspendedConnection(client, -1);
+                }
+            }
+        };
+        client.performBackgroundOperation(new OperationAndData<String>(operation, "/", null, errorCallback, null));
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index e11d74f..2eca53c 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
@@ -25,7 +25,6 @@
 import org.apache.curator.RetryLoop;
 import org.apache.curator.TimeTrace;
 import org.apache.curator.framework.api.*;
-import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionCreateBuilder;
 import org.apache.curator.utils.ThreadUtils;
@@ -35,6 +34,8 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -50,8 +51,10 @@
     private boolean createParentsAsContainers;
     private boolean doProtected;
     private boolean compress;
+    private boolean setDataIfExists;
     private String protectedId;
     private ACLing acling;
+    private Stat storingStat;
 
     @VisibleForTesting
     boolean failNextCreateForTesting = false;
@@ -69,42 +72,51 @@
         createParentsAsContainers = false;
         compress = false;
         doProtected = false;
+        setDataIfExists = false;
         protectedId = null;
+        storingStat = null;
     }
 
-    TransactionCreateBuilder asTransactionCreateBuilder(final CuratorTransactionImpl curatorTransaction, final CuratorMultiTransactionRecord transaction)
+    @Override
+    public CreateBuilderMain orSetData()
     {
-        return new TransactionCreateBuilder()
+        setDataIfExists = true;
+        return this;
+    }
+
+    <T> TransactionCreateBuilder<T> asTransactionCreateBuilder(final T context, final CuratorMultiTransactionRecord transaction)
+    {
+        return new TransactionCreateBuilder<T>()
         {
             @Override
-            public PathAndBytesable<CuratorTransactionBridge> withACL(List<ACL> aclList)
+            public PathAndBytesable<T> withACL(List<ACL> aclList)
             {
                 CreateBuilderImpl.this.withACL(aclList);
                 return this;
             }
 
             @Override
-            public ACLPathAndBytesable<CuratorTransactionBridge> withMode(CreateMode mode)
+            public ACLPathAndBytesable<T> withMode(CreateMode mode)
             {
                 CreateBuilderImpl.this.withMode(mode);
                 return this;
             }
 
             @Override
-            public ACLCreateModePathAndBytesable<CuratorTransactionBridge> compressed()
+            public ACLCreateModePathAndBytesable<T> compressed()
             {
                 CreateBuilderImpl.this.compressed();
                 return this;
             }
 
             @Override
-            public CuratorTransactionBridge forPath(String path) throws Exception
+            public T forPath(String path) throws Exception
             {
                 return forPath(path, client.getDefaultData());
             }
 
             @Override
-            public CuratorTransactionBridge forPath(String path, byte[] data) throws Exception
+            public T forPath(String path, byte[] data) throws Exception
             {
                 if ( compress )
                 {
@@ -113,18 +125,24 @@
 
                 String fixedPath = client.fixForNamespace(path);
                 transaction.add(Op.create(fixedPath, data, acling.getAclList(path), createMode), OperationType.CREATE, path);
-                return curatorTransaction;
+                return context;
             }
         };
     }
 
     @Override
-    public CreateBackgroundModeACLable compressed()
+    public CreateBackgroundModeStatACLable compressed()
     {
         compress = true;
-        return new CreateBackgroundModeACLable()
+        return new CreateBackgroundModeStatACLable()
         {
             @Override
+            public CreateBackgroundModeACLable storingStatIn(Stat stat) {
+                storingStat = stat;
+                return asCreateBackgroundModeACLable();
+            }
+            
+            @Override
             public ACLCreateModePathAndBytesable<String> creatingParentsIfNeeded()
             {
                 createParentsIfNeeded = true;
@@ -269,7 +287,7 @@
     }
 
     @Override
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded()
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentContainersIfNeeded()
     {
         setCreateParentsAsContainers();
         return creatingParentsIfNeeded();
@@ -284,10 +302,10 @@
     }
 
     @Override
-    public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded()
+    public ProtectACLCreateModeStatPathAndBytesable<String> creatingParentsIfNeeded()
     {
         createParentsIfNeeded = true;
-        return new ProtectACLCreateModePathAndBytesable<String>()
+        return new ProtectACLCreateModeStatPathAndBytesable<String>()
         {
             @Override
             public ACLCreateModeBackgroundPathAndBytesable<String> withProtection()
@@ -354,14 +372,20 @@
             {
                 return CreateBuilderImpl.this.forPath(path);
             }
+
+            @Override
+            public ACLBackgroundPathAndBytesable<String> storingStatIn(Stat stat) {
+                storingStat = stat;
+                return CreateBuilderImpl.this;
+            }
         };
     }
 
     @Override
-    public ACLCreateModeBackgroundPathAndBytesable<String> withProtection()
+    public ACLCreateModeStatBackgroundPathAndBytesable<String> withProtection()
     {
         setProtected();
-        return this;
+        return asACLCreateModeStatBackgroundPathAndBytesable();
     }
 
     @Override
@@ -502,7 +526,10 @@
     public void performBackgroundOperation(final OperationAndData<PathAndBytes> operationAndData) throws Exception
     {
         final TimeTrace trace = client.getZookeeperClient().startTracer("CreateBuilderImpl-Background");
-        client.getZooKeeper().create
+        
+        if(storingStat == null)
+        {
+            client.getZooKeeper().create
             (
                 operationAndData.getData().getPath(),
                 operationAndData.getData().getData(),
@@ -519,14 +546,133 @@
                         {
                             backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
                         }
+                        else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
+                        {
+                            backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
+                        }
                         else
                         {
-                            sendBackgroundResponse(rc, path, ctx, name, operationAndData);
+                            sendBackgroundResponse(rc, path, ctx, name, null, operationAndData);
                         }
                     }
                 },
                 backgrounding.getContext()
             );
+        }
+        else
+        {
+            client.getZooKeeper().create
+            (
+                operationAndData.getData().getPath(),
+                operationAndData.getData().getData(),
+                acling.getAclList(operationAndData.getData().getPath()),
+                createMode,
+                new AsyncCallback.Create2Callback() {
+                    
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
+                        trace.commit();
+                        
+                        //Need to manually populate the provided stat here.
+                        storingStat.setAversion(stat.getAversion());
+                        storingStat.setCtime(stat.getCtime());
+                        storingStat.setCversion(stat.getCversion());
+                        storingStat.setCzxid(stat.getCzxid());
+                        storingStat.setDataLength(stat.getDataLength());
+                        storingStat.setEphemeralOwner(stat.getEphemeralOwner());
+                        storingStat.setMtime(stat.getMtime());
+                        storingStat.setMzxid(stat.getMzxid());
+                        storingStat.setNumChildren(stat.getNumChildren());
+                        storingStat.setPzxid(stat.getPzxid());
+                        storingStat.setVersion(stat.getVersion());
+
+                        if ( (rc == KeeperException.Code.NONODE.intValue()) && createParentsIfNeeded )
+                        {
+                            backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
+                        }
+                        else
+                        {
+                            sendBackgroundResponse(rc, path, ctx, name, stat, operationAndData);
+                        }
+                    }
+                },
+                backgrounding.getContext()
+            );
+        }
+    }
+    
+    @Override
+    public CreateProtectACLCreateModePathAndBytesable<String> storingStatIn(Stat stat) {
+        storingStat = stat;
+        
+        return new CreateProtectACLCreateModePathAndBytesable<String>() {
+
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+                return CreateBuilderImpl.this.withACL(aclList);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground() {
+                return CreateBuilderImpl.this.inBackground();
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(Object context) {
+                return CreateBuilderImpl.this.inBackground(context);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback) {
+                return CreateBuilderImpl.this.inBackground(callback);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context) {
+                return CreateBuilderImpl.this.inBackground(callback, context);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, executor);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context,
+                    Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, context, executor);
+            }
+
+            @Override
+            public String forPath(String path, byte[] data) throws Exception {
+                return CreateBuilderImpl.this.forPath(path, data);
+            }
+
+            @Override
+            public String forPath(String path) throws Exception {
+                return CreateBuilderImpl.this.forPath(path);
+            }
+
+            @Override
+            public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
+                return CreateBuilderImpl.this.withMode(mode);
+            }
+
+            @Override
+            public ACLCreateModeBackgroundPathAndBytesable<String> withProtection() {
+                return CreateBuilderImpl.this.withProtection();
+            }
+
+            @Override
+            public ProtectACLCreateModePathAndBytesable<String> creatingParentsIfNeeded() {
+                return CreateBuilderImpl.this.creatingParentsIfNeeded();
+            }
+
+            @Override
+            public ProtectACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded() {
+                return CreateBuilderImpl.this.creatingParentContainersIfNeeded();
+            }  
+        };
     }
 
     private static String getProtectedPrefix(String protectedId)
@@ -556,12 +702,48 @@
         client.queueOperation(parentOperation);
     }
 
-    private void sendBackgroundResponse(int rc, String path, Object ctx, String name, OperationAndData<PathAndBytes> operationAndData)
+    private void backgroundSetData(final CuratorFrameworkImpl client, final OperationAndData<PathAndBytes> mainOperationAndData, final String path, final Backgrounding backgrounding)
+    {
+        final AsyncCallback.StatCallback statCallback = new AsyncCallback.StatCallback()
+        {
+            @Override
+            public void processResult(int rc, String path, Object ctx, Stat stat)
+            {
+                if ( rc == KeeperException.Code.NONODE.intValue() )
+                {
+                    client.queueOperation(mainOperationAndData);    // try to create it again
+                }
+                else
+                {
+                    sendBackgroundResponse(rc, path, ctx, path, stat, mainOperationAndData);
+                }
+            }
+        };
+        BackgroundOperation<PathAndBytes> operation = new BackgroundOperation<PathAndBytes>()
+        {
+            @Override
+            public void performBackgroundOperation(OperationAndData<PathAndBytes> op) throws Exception
+            {
+                try
+                {
+                    client.getZooKeeper().setData(path, mainOperationAndData.getData().getData(), -1, statCallback, backgrounding.getContext());
+                }
+                catch ( KeeperException e )
+                {
+                    // ignore
+                }
+                client.queueOperation(mainOperationAndData);
+            }
+        };
+        client.queueOperation(new OperationAndData<>(operation, null, null, null, null));
+    }
+
+    private void sendBackgroundResponse(int rc, String path, Object ctx, String name, Stat stat, OperationAndData<PathAndBytes> operationAndData)
     {
         path = client.unfixForNamespace(path);
         name = client.unfixForNamespace(name);
 
-        CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.CREATE, rc, path, name, ctx, null, null, null, null, null);
+        CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.CREATE, rc, path, name, ctx, stat, null, null, null, null, null);
         client.processBackgroundOperation(operationAndData, event);
     }
 
@@ -620,6 +802,141 @@
             }
         };
     }
+    
+    private CreateBackgroundModeACLable asCreateBackgroundModeACLable()
+    {
+        return new CreateBackgroundModeACLable() {
+            
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+                return CreateBuilderImpl.this.withACL(aclList);
+            }
+            
+            @Override
+            public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
+                return CreateBuilderImpl.this.withMode(mode);
+            }
+            
+            @Override
+            public String forPath(String path) throws Exception {
+                return CreateBuilderImpl.this.forPath(path);
+            }
+            
+            @Override
+            public String forPath(String path, byte[] data) throws Exception {
+                return CreateBuilderImpl.this.forPath(path, data);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, context, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context) {
+                return CreateBuilderImpl.this.inBackground(callback, context);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback) {
+                return CreateBuilderImpl.this.inBackground(callback);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(Object context) {
+                return CreateBuilderImpl.this.inBackground(context);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground() {
+                return CreateBuilderImpl.this.inBackground();
+            }
+            
+            @Override
+            public ACLPathAndBytesable<String> withProtectedEphemeralSequential() {
+                return CreateBuilderImpl.this.withProtectedEphemeralSequential();
+            }
+            
+            @Override
+            public ACLCreateModePathAndBytesable<String> creatingParentsIfNeeded() {
+                createParentsIfNeeded = true;
+                return asACLCreateModePathAndBytesable();
+            }
+            
+            @Override
+            public ACLCreateModePathAndBytesable<String> creatingParentContainersIfNeeded() {
+                setCreateParentsAsContainers();
+                return asACLCreateModePathAndBytesable();
+            }
+        };
+    }
+    
+    private ACLCreateModeStatBackgroundPathAndBytesable<String> asACLCreateModeStatBackgroundPathAndBytesable()
+    {
+        return new ACLCreateModeStatBackgroundPathAndBytesable<String>()
+        {
+            @Override
+            public BackgroundPathAndBytesable<String> withACL(List<ACL> aclList) {
+                return CreateBuilderImpl.this.withACL(aclList);
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground() {
+                return CreateBuilderImpl.this.inBackground();
+            }
+
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, context, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Executor executor) {
+                return CreateBuilderImpl.this.inBackground(callback, executor);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback, Object context) {
+                return CreateBuilderImpl.this.inBackground(callback, context);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(BackgroundCallback callback) {
+                return CreateBuilderImpl.this.inBackground(callback);
+            }
+            
+            @Override
+            public PathAndBytesable<String> inBackground(Object context) {
+                return CreateBuilderImpl.this.inBackground(context);
+            }
+
+            @Override
+            public String forPath(String path) throws Exception {
+                return CreateBuilderImpl.this.forPath(path);
+            }
+            
+            @Override
+            public String forPath(String path, byte[] data) throws Exception {
+                return CreateBuilderImpl.this.forPath(path, data);
+            }
+
+            @Override
+            public ACLBackgroundPathAndBytesable<String> withMode(CreateMode mode) {
+                return CreateBuilderImpl.this.withMode(mode);
+            }
+
+            @Override
+            public ACLCreateModeBackgroundPathAndBytesable<String> storingStatIn(Stat stat) {
+                storingStat = stat;
+                return CreateBuilderImpl.this;
+            }            
+        };
+    }
 
     @VisibleForTesting
     volatile boolean debugForceFindProtectedNode = false;
@@ -658,14 +975,14 @@
                     }
                     catch ( KeeperException.ConnectionLossException e )
                     {
-                        sendBackgroundResponse(KeeperException.Code.CONNECTIONLOSS.intValue(), path, backgrounding.getContext(), null, this);
+                        sendBackgroundResponse(KeeperException.Code.CONNECTIONLOSS.intValue(), path, backgrounding.getContext(), null, null, this);
                         callSuper = false;
                     }
                     if ( createdPath != null )
                     {
                         try
                         {
-                            sendBackgroundResponse(KeeperException.Code.OK.intValue(), createdPath, backgrounding.getContext(), createdPath, this);
+                            sendBackgroundResponse(KeeperException.Code.OK.intValue(), createdPath, backgrounding.getContext(), createdPath, null, this);
                         }
                         catch ( Exception e )
                         {
@@ -718,14 +1035,26 @@
                         {
                             try
                             {
-                                createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode);
+                                createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat);
                             }
                             catch ( KeeperException.NoNodeException e )
                             {
                                 if ( createParentsIfNeeded )
                                 {
                                     ZKPaths.mkdirs(client.getZooKeeper(), path, false, client.getAclProvider(), createParentsAsContainers);
-                                    createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode);
+                                    createdPath = client.getZooKeeper().create(path, data, acling.getAclList(path), createMode, storingStat);
+                                }
+                                else
+                                {
+                                    throw e;
+                                }
+                            }
+                            catch ( KeeperException.NodeExistsException e )
+                            {
+                                if ( setDataIfExists )
+                                {
+                                    client.getZooKeeper().setData(path, data, -1);
+                                    createdPath = path;
                                 }
                                 else
                                 {
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorEventImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorEventImpl.java
index 929fe6d..4aa125f 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorEventImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorEventImpl.java
@@ -16,11 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.imps;
 
 import com.google.common.collect.ImmutableList;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
@@ -29,16 +31,17 @@
 
 class CuratorEventImpl implements CuratorEvent
 {
-    private final CuratorEventType  type;
-    private final int               resultCode;
-    private final String            path;
-    private final String            name;
-    private final List<String>      children;
-    private final Object            context;
-    private final Stat              stat;
-    private final byte[]            data;
-    private final WatchedEvent      watchedEvent;
-    private final List<ACL>         aclList;
+    private final CuratorEventType type;
+    private final int resultCode;
+    private final String path;
+    private final String name;
+    private final List<String> children;
+    private final Object context;
+    private final Stat stat;
+    private final byte[] data;
+    private final WatchedEvent watchedEvent;
+    private final List<ACL> aclList;
+    private final List<CuratorTransactionResult> opResults;
 
     @Override
     public CuratorEventType getType()
@@ -101,6 +104,12 @@
     }
 
     @Override
+    public List<CuratorTransactionResult> getOpResults()
+    {
+        return opResults;
+    }
+
+    @Override
     public String toString()
     {
         return "CuratorEventImpl{" +
@@ -114,20 +123,22 @@
             ", data=" + Arrays.toString(data) +
             ", watchedEvent=" + watchedEvent +
             ", aclList=" + aclList +
+            ", opResults=" + opResults +
             '}';
     }
 
-    CuratorEventImpl(CuratorFrameworkImpl client, CuratorEventType type, int resultCode, String path, String name, Object context, Stat stat, byte[] data, List<String> children, WatchedEvent watchedEvent, List<ACL> aclList)
+    CuratorEventImpl(CuratorFrameworkImpl client, CuratorEventType type, int resultCode, String path, String name, Object context, Stat stat, byte[] data, List<String> children, WatchedEvent watchedEvent, List<ACL> aclList, List<CuratorTransactionResult> opResults)
     {
         this.type = type;
         this.resultCode = resultCode;
+        this.opResults = (opResults != null) ? ImmutableList.copyOf(opResults) : null;
         this.path = client.unfixForNamespace(path);
         this.name = name;
         this.context = context;
         this.stat = stat;
         this.data = data;
         this.children = children;
-        this.watchedEvent = (watchedEvent != null) ? new NamespaceWatchedEvent(client, watchedEvent) : watchedEvent;
+        this.watchedEvent = (watchedEvent != null) ? new NamespaceWatchedEvent(client, watchedEvent) : null;
         this.aclList = (aclList != null) ? ImmutableList.copyOf(aclList) : null;
     }
 }
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 442579d..2b8bbbc 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
@@ -31,11 +31,15 @@
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+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.listen.ListenerContainer;
 import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateErrorPolicy;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.framework.state.ConnectionStateManager;
 import org.apache.curator.utils.DebugUtils;
@@ -47,6 +51,7 @@
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.util.Arrays;
@@ -59,6 +64,7 @@
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
 public class CuratorFrameworkImpl implements CuratorFramework
@@ -75,11 +81,16 @@
     private final List<AuthInfo> authInfos;
     private final byte[] defaultData;
     private final FailedDeleteManager failedDeleteManager;
+    private final FailedRemoveWatchManager failedRemoveWatcherManager;
     private final CompressionProvider compressionProvider;
     private final ACLProvider aclProvider;
     private final NamespaceFacadeCache namespaceFacadeCache;
     private final NamespaceWatcherMap namespaceWatcherMap = new NamespaceWatcherMap(this);
     private final boolean useContainerParentsIfAvailable;
+    private final ConnectionStateErrorPolicy connectionStateErrorPolicy;
+    private final AtomicLong currentInstanceIndex = new AtomicLong(-1);
+    private final InternalConnectionHandler internalConnectionHandler;
+    private final EnsembleTracker ensembleTracker;
 
     private volatile ExecutorService executorService;
     private final AtomicBoolean logAsErrorConnectionErrors = new AtomicBoolean(false);
@@ -100,34 +111,50 @@
     public CuratorFrameworkImpl(CuratorFrameworkFactory.Builder builder)
     {
         ZookeeperFactory localZookeeperFactory = makeZookeeperFactory(builder.getZookeeperFactory());
-        this.client = new CuratorZookeeperClient(localZookeeperFactory, builder.getEnsembleProvider(), builder.getSessionTimeoutMs(), builder.getConnectionTimeoutMs(), new Watcher()
-        {
-            @Override
-            public void process(WatchedEvent watchedEvent)
-            {
-                CuratorEvent event = new CuratorEventImpl(CuratorFrameworkImpl.this, CuratorEventType.WATCHED, watchedEvent.getState().getIntValue(), unfixForNamespace(watchedEvent.getPath()), null, null, null, null, null, watchedEvent, null);
-                processEvent(event);
-            }
-        }, builder.getRetryPolicy(), builder.canBeReadOnly());
+        this.client = new CuratorZookeeperClient
+            (
+                localZookeeperFactory,
+                builder.getEnsembleProvider(),
+                builder.getSessionTimeoutMs(),
+                builder.getConnectionTimeoutMs(),
+                new Watcher()
+                {
+                    @Override
+                    public void process(WatchedEvent watchedEvent)
+                    {
+                        CuratorEvent event = new CuratorEventImpl(CuratorFrameworkImpl.this, CuratorEventType.WATCHED, watchedEvent.getState().getIntValue(), unfixForNamespace(watchedEvent.getPath()), null, null, null, null, null, watchedEvent, null, null);
+                        processEvent(event);
+                    }
+                },
+                builder.getRetryPolicy(),
+                builder.canBeReadOnly(),
+                builder.getConnectionHandlingPolicy()
+            );
 
+        boolean isClassic = (builder.getConnectionHandlingPolicy().getSimulatedSessionExpirationPercent() == 0);
+        internalConnectionHandler = isClassic ? new ClassicInternalConnectionHandler() : new StandardInternalConnectionHandler();
         listeners = new ListenerContainer<CuratorListener>();
         unhandledErrorListeners = new ListenerContainer<UnhandledErrorListener>();
         backgroundOperations = new DelayQueue<OperationAndData<?>>();
         namespace = new NamespaceImpl(this, builder.getNamespace());
         threadFactory = getThreadFactory(builder);
         maxCloseWaitMs = builder.getMaxCloseWaitMs();
-        connectionStateManager = new ConnectionStateManager(this, builder.getThreadFactory());
+        connectionStateManager = new ConnectionStateManager(this, builder.getThreadFactory(), builder.getSessionTimeoutMs(), builder.getConnectionHandlingPolicy().getSimulatedSessionExpirationPercent());
         compressionProvider = builder.getCompressionProvider();
         aclProvider = builder.getAclProvider();
         state = new AtomicReference<CuratorFrameworkState>(CuratorFrameworkState.LATENT);
         useContainerParentsIfAvailable = builder.useContainerParentsIfAvailable();
+        connectionStateErrorPolicy = Preconditions.checkNotNull(builder.getConnectionStateErrorPolicy(), "errorPolicy cannot be null");
 
         byte[] builderDefaultData = builder.getDefaultData();
         defaultData = (builderDefaultData != null) ? Arrays.copyOf(builderDefaultData, builderDefaultData.length) : new byte[0];
         authInfos = buildAuths(builder);
 
         failedDeleteManager = new FailedDeleteManager(this);
+        failedRemoveWatcherManager = new FailedRemoveWatchManager(this);
         namespaceFacadeCache = new NamespaceFacadeCache(this);
+
+        ensembleTracker = new EnsembleTracker(this, builder.getEnsembleProvider());
     }
 
     private List<AuthInfo> buildAuths(CuratorFrameworkFactory.Builder builder)
@@ -140,6 +167,18 @@
         return builder1.build();
     }
 
+    @Override
+    public WatcherRemoveCuratorFramework newWatcherRemoveCuratorFramework()
+    {
+        return new WatcherRemovalFacade(this);
+    }
+
+    @Override
+    public QuorumVerifier getCurrentConfig()
+    {
+        return (ensembleTracker != null) ? ensembleTracker.getCurrentConfig() : null;
+    }
+
     private ZookeeperFactory makeZookeeperFactory(final ZookeeperFactory actualZookeeperFactory)
     {
         return new ZookeeperFactory()
@@ -179,6 +218,7 @@
         connectionStateManager = parent.connectionStateManager;
         defaultData = parent.defaultData;
         failedDeleteManager = parent.failedDeleteManager;
+        failedRemoveWatcherManager = parent.failedRemoveWatcherManager;
         compressionProvider = parent.compressionProvider;
         aclProvider = parent.aclProvider;
         namespaceFacadeCache = parent.namespaceFacadeCache;
@@ -186,6 +226,9 @@
         state = parent.state;
         authInfos = parent.authInfos;
         useContainerParentsIfAvailable = parent.useContainerParentsIfAvailable;
+        connectionStateErrorPolicy = parent.connectionStateErrorPolicy;
+        internalConnectionHandler = parent.internalConnectionHandler;
+        ensembleTracker = null;
     }
 
     @Override
@@ -230,6 +273,12 @@
     }
 
     @Override
+    public ConnectionStateErrorPolicy getConnectionStateErrorPolicy()
+    {
+        return connectionStateErrorPolicy;
+    }
+
+    @Override
     public void start()
     {
         log.info("Starting");
@@ -268,6 +317,8 @@
                     return null;
                 }
             });
+
+            ensembleTracker.start();
         }
         catch ( Exception e )
         {
@@ -287,7 +338,7 @@
                 @Override
                 public Void apply(CuratorListener listener)
                 {
-                    CuratorEvent event = new CuratorEventImpl(CuratorFrameworkImpl.this, CuratorEventType.CLOSING, 0, null, null, null, null, null, null, null, null);
+                    CuratorEvent event = new CuratorEventImpl(CuratorFrameworkImpl.this, CuratorEventType.CLOSING, 0, null, null, null, null, null, null, null, null, null);
                     try
                     {
                         listener.eventReceived(CuratorFrameworkImpl.this, event);
@@ -315,6 +366,7 @@
                 }
             }
 
+            ensembleTracker.close();
             listeners.clear();
             unhandledErrorListeners.clear();
             connectionStateManager.close();
@@ -410,6 +462,18 @@
     }
 
     @Override
+    public ReconfigBuilder reconfig()
+    {
+        return new ReconfigBuilderImpl(this);
+    }
+
+    @Override
+    public GetConfigBuilder getConfig()
+    {
+        return new GetConfigBuilderImpl(this);
+    }
+
+    @Override
     public CuratorTransaction inTransaction()
     {
         Preconditions.checkState(getState() == CuratorFrameworkState.STARTED, "instance must be started before calling this method");
@@ -418,6 +482,22 @@
     }
 
     @Override
+    public CuratorMultiTransaction transaction()
+    {
+        Preconditions.checkState(getState() == CuratorFrameworkState.STARTED, "instance must be started before calling this method");
+
+        return new CuratorMultiTransactionImpl(this);
+    }
+
+    @Override
+    public TransactionOp transactionOp()
+    {
+        Preconditions.checkState(getState() == CuratorFrameworkState.STARTED, "instance must be started before calling this method");
+
+        return new TransactionOpImpl(this);
+    }
+
+    @Override
     public Listenable<ConnectionStateListener> getConnectionStateListenable()
     {
         return connectionStateManager.getListenable();
@@ -451,6 +531,12 @@
         return new SyncBuilderImpl(this);
     }
 
+    @Override
+    public RemoveWatchesBuilder watches()
+    {
+        return new RemoveWatchesBuilderImpl(this);
+    }
+
     protected void internalSync(CuratorFrameworkImpl impl, String path, Object context)
     {
         BackgroundOperation<String> operation = new BackgroundSyncImpl(impl, context);
@@ -479,6 +565,11 @@
         return failedDeleteManager;
     }
 
+    FailedRemoveWatchManager getFailedRemoveWatcherManager()
+    {
+        return failedRemoveWatcherManager;
+    }
+
     RetryLoop newRetryLoop()
     {
         return client.newRetryLoop();
@@ -618,7 +709,7 @@
     {
         if ( state == Watcher.Event.KeeperState.Disconnected )
         {
-            suspendConnection();
+            internalConnectionHandler.suspendConnection(this);
         }
         else if ( state == Watcher.Event.KeeperState.Expired )
         {
@@ -626,14 +717,26 @@
         }
         else if ( state == Watcher.Event.KeeperState.SyncConnected )
         {
+            internalConnectionHandler.checkNewConnection(this);
             connectionStateManager.addStateChange(ConnectionState.RECONNECTED);
         }
         else if ( state == Watcher.Event.KeeperState.ConnectedReadOnly )
         {
+            internalConnectionHandler.checkNewConnection(this);
             connectionStateManager.addStateChange(ConnectionState.READ_ONLY);
         }
     }
 
+    void checkInstanceIndex()
+    {
+        long instanceIndex = client.getInstanceIndex();
+        long newInstanceIndex = currentInstanceIndex.getAndSet(instanceIndex);
+        if ( (newInstanceIndex >= 0) && (instanceIndex != newInstanceIndex) )   // currentInstanceIndex is initially -1 - ignore this
+        {
+            connectionStateManager.addStateChange(ConnectionState.LOST);
+        }
+    }
+
     Watcher.Event.KeeperState codeToState(KeeperException.Code code)
     {
         switch ( code )
@@ -664,41 +767,24 @@
         return Watcher.Event.KeeperState.fromInt(-1);
     }
 
-    private void suspendConnection()
+    WatcherRemovalManager getWatcherRemovalManager()
     {
-        if ( !connectionStateManager.setToSuspended() )
-        {
-            return;
-        }
-
-        doSyncForSuspendedConnection(client.getInstanceIndex());
+        return null;
     }
 
-    private void doSyncForSuspendedConnection(final long instanceIndex)
+    boolean setToSuspended()
     {
-        // we appear to have disconnected, force a new ZK event and see if we can connect to another server
-        final BackgroundOperation<String> operation = new BackgroundSyncImpl(this, null);
-        OperationAndData.ErrorCallback<String> errorCallback = new OperationAndData.ErrorCallback<String>()
-        {
-            @Override
-            public void retriesExhausted(OperationAndData<String> operationAndData)
-            {
-                // if instanceIndex != newInstanceIndex, the ZooKeeper instance was reset/reallocated
-                // so the pending background sync is no longer valid.
-                // if instanceIndex is -1, this is the second try to sync - punt and mark the connection lost
-                if ( (instanceIndex < 0) || (instanceIndex == client.getInstanceIndex()) )
-                {
-                    connectionStateManager.addStateChange(ConnectionState.LOST);
-                }
-                else
-                {
-                    log.debug("suspendConnection() failure ignored as the ZooKeeper instance was reset. Retrying.");
-                    // send -1 to signal that if it happens again, punt and mark the connection lost
-                    doSyncForSuspendedConnection(-1);
-                }
-            }
-        };
-        performBackgroundOperation(new OperationAndData<String>(operation, "/", null, errorCallback, null));
+        return connectionStateManager.setToSuspended();
+    }
+
+    void addStateChange(ConnectionState newConnectionState)
+    {
+        connectionStateManager.addStateChange(newConnectionState);
+    }
+
+    EnsembleTracker getEnsembleTracker()
+    {
+        return ensembleTracker;
     }
 
     @SuppressWarnings({"ThrowableResultOfMethodCallIgnored"})
@@ -822,11 +908,11 @@
         }
     }
 
-    private void performBackgroundOperation(OperationAndData<?> operationAndData)
+    void performBackgroundOperation(OperationAndData<?> operationAndData)
     {
         try
         {
-            if ( client.isConnected() )
+            if ( !operationAndData.isConnectionRequired() || client.isConnected() )
             {
                 operationAndData.callPerformBackgroundOperation();
             }
@@ -853,7 +939,7 @@
             if ( e instanceof CuratorConnectionLossException )
             {
                 WatchedEvent watchedEvent = new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.Disconnected, null);
-                CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.WATCHED, KeeperException.Code.CONNECTIONLOSS.intValue(), null, null, operationAndData.getContext(), null, null, null, watchedEvent, null);
+                CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.WATCHED, KeeperException.Code.CONNECTIONLOSS.intValue(), null, null, operationAndData.getContext(), null, null, null, watchedEvent, null, null);
                 if ( checkBackgroundRetry(operationAndData, event) )
                 {
                     queueOperation(operationAndData);
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java
new file mode 100644
index 0000000..577b0d6
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionImpl.java
@@ -0,0 +1,163 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.curator.RetryLoop;
+import org.apache.curator.TimeTrace;
+import org.apache.curator.framework.CuratorFramework;
+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.transaction.CuratorMultiTransaction;
+import org.apache.curator.framework.api.transaction.CuratorMultiTransactionMain;
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.OpResult;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executor;
+
+public class CuratorMultiTransactionImpl implements
+    CuratorMultiTransaction,
+    CuratorMultiTransactionMain,
+    BackgroundOperation<CuratorMultiTransactionRecord>
+{
+    private final CuratorFrameworkImpl client;
+    private Backgrounding backgrounding = new Backgrounding();
+
+    public CuratorMultiTransactionImpl(CuratorFrameworkImpl client)
+    {
+        this.client = client;
+    }
+
+    @Override
+    public CuratorMultiTransactionMain inBackground()
+    {
+        backgrounding = new Backgrounding(true);
+        return this;
+    }
+
+    @Override
+    public CuratorMultiTransactionMain inBackground(Object context)
+    {
+        backgrounding = new Backgrounding(context);
+        return this;
+    }
+
+    @Override
+    public CuratorMultiTransactionMain inBackground(BackgroundCallback callback)
+    {
+        backgrounding = new Backgrounding(callback);
+        return this;
+    }
+
+    @Override
+    public CuratorMultiTransactionMain inBackground(BackgroundCallback callback, Object context)
+    {
+        backgrounding = new Backgrounding(callback, context);
+        return this;
+    }
+
+    @Override
+    public CuratorMultiTransactionMain inBackground(BackgroundCallback callback, Executor executor)
+    {
+        backgrounding = new Backgrounding(callback, executor);
+        return this;
+    }
+
+    @Override
+    public CuratorMultiTransactionMain inBackground(BackgroundCallback callback, Object context, Executor executor)
+    {
+        backgrounding = new Backgrounding(client, callback, context, executor);
+        return this;
+    }
+
+    @Override
+    public List<CuratorTransactionResult> forOperations(CuratorOp... operations) throws Exception
+    {
+        List<CuratorOp> ops = (operations != null) ? Arrays.asList(operations) : Lists.<CuratorOp>newArrayList();
+        return forOperations(ops);
+    }
+
+    @Override
+    public List<CuratorTransactionResult> forOperations(List<CuratorOp> operations) throws Exception
+    {
+        operations = Preconditions.checkNotNull(operations, "operations cannot be null");
+        Preconditions.checkArgument(!operations.isEmpty(), "operations list cannot be empty");
+
+        CuratorMultiTransactionRecord record = new CuratorMultiTransactionRecord();
+        for ( CuratorOp curatorOp : operations )
+        {
+            record.add(curatorOp.get(), curatorOp.getTypeAndPath().getType(), curatorOp.getTypeAndPath().getForPath());
+        }
+
+        if ( backgrounding.inBackground() )
+        {
+            client.processBackgroundOperation(new OperationAndData<>(this, record, backgrounding.getCallback(), null, backgrounding.getContext()), null);
+            return null;
+        }
+        else
+        {
+            return forOperationsInForeground(record);
+        }
+    }
+
+    @Override
+    public void performBackgroundOperation(final OperationAndData<CuratorMultiTransactionRecord> operationAndData) throws Exception
+    {
+        final TimeTrace trace = client.getZookeeperClient().startTracer("CuratorMultiTransactionImpl-Background");
+        AsyncCallback.MultiCallback callback = new AsyncCallback.MultiCallback()
+        {
+            @Override
+            public void processResult(int rc, String path, Object ctx, List<OpResult> opResults)
+            {
+                trace.commit();
+                List<CuratorTransactionResult> curatorResults = (opResults != null) ? CuratorTransactionImpl.wrapResults(client, opResults, operationAndData.getData()) : null;
+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.TRANSACTION, rc, path, null, ctx, null, null, null, null, null, curatorResults);
+                client.processBackgroundOperation(operationAndData, event);
+            }
+        };
+        client.getZooKeeper().multi(operationAndData.getData(), callback, backgrounding.getContext());
+    }
+
+    private List<CuratorTransactionResult> forOperationsInForeground(final CuratorMultiTransactionRecord record) throws Exception
+    {
+        TimeTrace trace = client.getZookeeperClient().startTracer("CuratorMultiTransactionImpl-Foreground");
+        List<OpResult> responseData = RetryLoop.callWithRetry
+        (
+            client.getZookeeperClient(),
+            new Callable<List<OpResult>>()
+            {
+                @Override
+                public List<OpResult> call() throws Exception
+                {
+                    return client.getZooKeeper().multi(record);
+                }
+            }
+        );
+        trace.commit();
+
+        return CuratorTransactionImpl.wrapResults(client, responseData, record);
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionRecord.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionRecord.java
index 1500d6d..0611df6 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionRecord.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorMultiTransactionRecord.java
@@ -20,6 +20,7 @@
 
 import com.google.common.collect.Lists;
 import org.apache.curator.framework.api.transaction.OperationType;
+import org.apache.curator.framework.api.transaction.TypeAndPath;
 import org.apache.zookeeper.MultiTransactionRecord;
 import org.apache.zookeeper.Op;
 import java.util.List;
@@ -28,30 +29,18 @@
 {
     private final List<TypeAndPath>     metadata = Lists.newArrayList();
 
-    static class TypeAndPath
-    {
-        final OperationType type;
-        final String forPath;
-
-        TypeAndPath(OperationType type, String forPath)
-        {
-            this.type = type;
-            this.forPath = forPath;
-        }
-    }
-
     @Override
     public final void add(Op op)
     {
         throw new UnsupportedOperationException();
     }
-    
+
     void add(Op op, OperationType type, String forPath)
     {
         super.add(op);
         metadata.add(new TypeAndPath(type, forPath));
     }
-    
+
     TypeAndPath     getMetadata(int index)
     {
         return metadata.get(index);
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorTransactionImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorTransactionImpl.java
index 13ffe82..20ec274 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorTransactionImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorTransactionImpl.java
@@ -16,21 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.imps;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import org.apache.curator.RetryLoop;
 import org.apache.curator.framework.api.Pathable;
-import org.apache.curator.framework.api.transaction.CuratorTransaction;
-import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
-import org.apache.curator.framework.api.transaction.CuratorTransactionFinal;
-import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
-import org.apache.curator.framework.api.transaction.OperationType;
-import org.apache.curator.framework.api.transaction.TransactionCheckBuilder;
-import org.apache.curator.framework.api.transaction.TransactionCreateBuilder;
-import org.apache.curator.framework.api.transaction.TransactionDeleteBuilder;
-import org.apache.curator.framework.api.transaction.TransactionSetDataBuilder;
+import org.apache.curator.framework.api.transaction.*;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.OpResult;
@@ -43,10 +36,10 @@
 
 class CuratorTransactionImpl implements CuratorTransaction, CuratorTransactionBridge, CuratorTransactionFinal
 {
-    private final CuratorFrameworkImpl              client;
-    private final CuratorMultiTransactionRecord     transaction;
+    private final CuratorFrameworkImpl client;
+    private final CuratorMultiTransactionRecord transaction;
 
-    private boolean         isCommitted = false;
+    private boolean isCommitted = false;
 
     CuratorTransactionImpl(CuratorFrameworkImpl client)
     {
@@ -61,49 +54,58 @@
     }
 
     @Override
-    public TransactionCreateBuilder create()
+    public TransactionCreateBuilder<CuratorTransactionBridge> create()
     {
         Preconditions.checkState(!isCommitted, "transaction already committed");
 
-        return new CreateBuilderImpl(client).asTransactionCreateBuilder(this, transaction);
+        CuratorTransactionBridge asBridge = this;
+        return new CreateBuilderImpl(client).asTransactionCreateBuilder(asBridge, transaction);
     }
 
     @Override
-    public TransactionDeleteBuilder delete()
+    public TransactionDeleteBuilder<CuratorTransactionBridge> delete()
     {
         Preconditions.checkState(!isCommitted, "transaction already committed");
 
-        return new DeleteBuilderImpl(client).asTransactionDeleteBuilder(this, transaction);
+        CuratorTransactionBridge asBridge = this;
+        return new DeleteBuilderImpl(client).asTransactionDeleteBuilder(asBridge, transaction);
     }
 
     @Override
-    public TransactionSetDataBuilder setData()
+    public TransactionSetDataBuilder<CuratorTransactionBridge> setData()
     {
         Preconditions.checkState(!isCommitted, "transaction already committed");
 
-        return new SetDataBuilderImpl(client).asTransactionSetDataBuilder(this, transaction);
+        CuratorTransactionBridge asBridge = this;
+        return new SetDataBuilderImpl(client).asTransactionSetDataBuilder(asBridge, transaction);
     }
 
     @Override
-    public TransactionCheckBuilder check()
+    public TransactionCheckBuilder<CuratorTransactionBridge> check()
     {
         Preconditions.checkState(!isCommitted, "transaction already committed");
 
-        return new TransactionCheckBuilder()
+        CuratorTransactionBridge asBridge = this;
+        return makeTransactionCheckBuilder(client, asBridge, transaction);
+    }
+
+    static <T> TransactionCheckBuilder<T> makeTransactionCheckBuilder(final CuratorFrameworkImpl client, final T context, final CuratorMultiTransactionRecord transaction)
+    {
+        return new TransactionCheckBuilder<T>()
         {
-            private int         version = -1;
+            private int version = -1;
 
             @Override
-            public CuratorTransactionBridge forPath(String path) throws Exception
+            public T forPath(String path) throws Exception
             {
-                String      fixedPath = client.fixForNamespace(path);
+                String fixedPath = client.fixForNamespace(path);
                 transaction.add(Op.check(fixedPath, version), OperationType.CHECK, path);
 
-                return CuratorTransactionImpl.this;
+                return context;
             }
 
             @Override
-            public Pathable<CuratorTransactionBridge> withVersion(int version)
+            public Pathable<T> withVersion(int version)
             {
                 this.version = version;
                 return this;
@@ -118,65 +120,44 @@
         isCommitted = true;
 
         final AtomicBoolean firstTime = new AtomicBoolean(true);
-        List<OpResult>      resultList = RetryLoop.callWithRetry
-        (
-            client.getZookeeperClient(),
-            new Callable<List<OpResult>>()
-            {
-                @Override
-                public List<OpResult> call() throws Exception
+        List<OpResult> resultList = RetryLoop.callWithRetry
+            (
+                client.getZookeeperClient(),
+                new Callable<List<OpResult>>()
                 {
-                    return doOperation(firstTime);
+                    @Override
+                    public List<OpResult> call() throws Exception
+                    {
+                        return doOperation(firstTime);
+                    }
                 }
-            }
-        );
-        
+            );
+
         if ( resultList.size() != transaction.metadataSize() )
         {
             throw new IllegalStateException(String.format("Result size (%d) doesn't match input size (%d)", resultList.size(), transaction.metadataSize()));
         }
 
-        ImmutableList.Builder<CuratorTransactionResult>     builder = ImmutableList.builder();
+        return wrapResults(client, resultList, transaction);
+    }
+
+    static List<CuratorTransactionResult> wrapResults(CuratorFrameworkImpl client, List<OpResult> resultList, CuratorMultiTransactionRecord transaction)
+    {
+        ImmutableList.Builder<CuratorTransactionResult> builder = ImmutableList.builder();
         for ( int i = 0; i < resultList.size(); ++i )
         {
-            OpResult                                    opResult = resultList.get(i);
-            CuratorMultiTransactionRecord.TypeAndPath   metadata = transaction.getMetadata(i);
-            CuratorTransactionResult                    curatorResult = makeCuratorResult(opResult, metadata);
+            OpResult opResult = resultList.get(i);
+            TypeAndPath metadata = transaction.getMetadata(i);
+            CuratorTransactionResult curatorResult = makeCuratorResult(client, opResult, metadata);
             builder.add(curatorResult);
         }
 
         return builder.build();
     }
 
-    private List<OpResult> doOperation(AtomicBoolean firstTime) throws Exception
+    static CuratorTransactionResult makeCuratorResult(CuratorFrameworkImpl client, OpResult opResult, TypeAndPath metadata)
     {
-        boolean         localFirstTime = firstTime.getAndSet(false);
-        if ( !localFirstTime )
-        {
-
-        }
-
-        List<OpResult>  opResults = client.getZooKeeper().multi(transaction);
-        if ( opResults.size() > 0 )
-        {
-            OpResult        firstResult = opResults.get(0);
-            if ( firstResult.getType() == ZooDefs.OpCode.error )
-            {
-                OpResult.ErrorResult        error = (OpResult.ErrorResult)firstResult;
-                KeeperException.Code        code = KeeperException.Code.get(error.getErr());
-                if ( code == null )
-                {
-                    code = KeeperException.Code.UNIMPLEMENTED;
-                }
-                throw KeeperException.create(code);
-            }
-        }
-        return opResults;
-    }
-
-    private CuratorTransactionResult makeCuratorResult(OpResult opResult, CuratorMultiTransactionRecord.TypeAndPath metadata)
-    {
-        String                                      resultPath = null;
+        String resultPath = null;
         Stat resultStat = null;
         switch ( opResult.getType() )
         {
@@ -188,19 +169,45 @@
 
             case ZooDefs.OpCode.create:
             {
-                OpResult.CreateResult       createResult = (OpResult.CreateResult)opResult;
+                OpResult.CreateResult createResult = (OpResult.CreateResult)opResult;
                 resultPath = client.unfixForNamespace(createResult.getPath());
                 break;
             }
 
             case ZooDefs.OpCode.setData:
             {
-                OpResult.SetDataResult      setDataResult = (OpResult.SetDataResult)opResult;
+                OpResult.SetDataResult setDataResult = (OpResult.SetDataResult)opResult;
                 resultStat = setDataResult.getStat();
                 break;
             }
         }
 
-        return new CuratorTransactionResult(metadata.type, metadata.forPath, resultPath, resultStat);
+        return new CuratorTransactionResult(metadata.getType(), metadata.getForPath(), resultPath, resultStat);
+    }
+
+    private List<OpResult> doOperation(AtomicBoolean firstTime) throws Exception
+    {
+        boolean localFirstTime = firstTime.getAndSet(false);
+        if ( !localFirstTime )
+        {
+            // TODO
+        }
+
+        List<OpResult> opResults = client.getZooKeeper().multi(transaction);
+        if ( opResults.size() > 0 )
+        {
+            OpResult firstResult = opResults.get(0);
+            if ( firstResult.getType() == ZooDefs.OpCode.error )
+            {
+                OpResult.ErrorResult error = (OpResult.ErrorResult)firstResult;
+                KeeperException.Code code = KeeperException.Code.get(error.getErr());
+                if ( code == null )
+                {
+                    code = KeeperException.Code.UNIMPLEMENTED;
+                }
+                throw KeeperException.create(code);
+            }
+        }
+        return opResults;
     }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/DeleteBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/DeleteBuilderImpl.java
index c3247a1..ab72308 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/DeleteBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/DeleteBuilderImpl.java
@@ -27,8 +27,8 @@
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.framework.api.DeleteBuilder;
+import org.apache.curator.framework.api.DeleteBuilderMain;
 import org.apache.curator.framework.api.Pathable;
-import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionDeleteBuilder;
 import org.apache.curator.utils.ThreadUtils;
@@ -46,6 +46,7 @@
     private Backgrounding backgrounding;
     private boolean deletingChildrenIfNeeded;
     private boolean guaranteed;
+    private boolean quietly;
 
     DeleteBuilderImpl(CuratorFrameworkImpl client)
     {
@@ -54,22 +55,23 @@
         backgrounding = new Backgrounding();
         deletingChildrenIfNeeded = false;
         guaranteed = false;
+        quietly = false;
     }
 
-    TransactionDeleteBuilder asTransactionDeleteBuilder(final CuratorTransactionImpl curatorTransaction, final CuratorMultiTransactionRecord transaction)
+    <T> TransactionDeleteBuilder<T> asTransactionDeleteBuilder(final T context, final CuratorMultiTransactionRecord transaction)
     {
-        return new TransactionDeleteBuilder()
+        return new TransactionDeleteBuilder<T>()
         {
             @Override
-            public CuratorTransactionBridge forPath(String path) throws Exception
+            public T forPath(String path) throws Exception
             {
                 String fixedPath = client.fixForNamespace(path);
                 transaction.add(Op.delete(fixedPath, version), OperationType.DELETE, path);
-                return curatorTransaction;
+                return context;
             }
 
             @Override
-            public Pathable<CuratorTransactionBridge> withVersion(int version)
+            public Pathable<T> withVersion(int version)
             {
                 DeleteBuilderImpl.this.withVersion(version);
                 return this;
@@ -78,6 +80,13 @@
     }
 
     @Override
+    public DeleteBuilderMain quietly()
+    {
+        quietly = true;
+        return this;
+    }
+
+    @Override
     public ChildrenDeletable guaranteed()
     {
         guaranteed = true;
@@ -160,7 +169,11 @@
                         }
                         else
                         {
-                            CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.DELETE, rc, path, null, ctx, null, null, null, null, null);
+                            if ( (rc == KeeperException.Code.NONODE.intValue()) && quietly )
+                            {
+                                rc = KeeperException.Code.OK.intValue();
+                            }
+                            CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.DELETE, rc, path, null, ctx, null, null, null, null, null, null);
                             client.processBackgroundOperation(operationAndData, event);
                         }
                     }
@@ -207,7 +220,7 @@
                     @Override
                     public void retriesExhausted(OperationAndData<String> operationAndData)
                     {
-                        client.getFailedDeleteManager().addFailedDelete(unfixedPath);
+                        client.getFailedDeleteManager().addFailedOperation(unfixedPath);
                     }
                 };
             }
@@ -242,6 +255,13 @@
                             {
                                 client.getZooKeeper().delete(path, version);
                             }
+                            catch ( KeeperException.NoNodeException e )
+                            {
+                                if ( !quietly )
+                                {
+                                    throw e;
+                                }
+                            }
                             catch ( KeeperException.NotEmptyException e )
                             {
                                 if ( deletingChildrenIfNeeded )
@@ -264,7 +284,7 @@
             //Only retry a guaranteed delete if it's a retryable error
             if( (RetryLoop.isRetryException(e) || (e instanceof InterruptedException)) && guaranteed )
             {
-                client.getFailedDeleteManager().addFailedDelete(unfixedPath);
+                client.getFailedDeleteManager().addFailedOperation(unfixedPath);
             }
             throw e;
         }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java
new file mode 100644
index 0000000..bc59512
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.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.framework.imps;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.curator.ensemble.EnsembleProvider;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
+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.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+@VisibleForTesting
+public class EnsembleTracker implements Closeable, CuratorWatcher
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final WatcherRemoveCuratorFramework client;
+    private final EnsembleProvider ensembleProvider;
+    private final AtomicReference<State> state = new AtomicReference<>(State.LATENT);
+    private final AtomicReference<QuorumMaj> currentConfig = new AtomicReference<>(new QuorumMaj(Maps.<Long, QuorumPeer.QuorumServer>newHashMap()));
+    private final ConnectionStateListener connectionStateListener = new ConnectionStateListener()
+    {
+        @Override
+        public void stateChanged(CuratorFramework client, ConnectionState newState)
+        {
+            if ( (newState == ConnectionState.CONNECTED) || (newState == ConnectionState.RECONNECTED) )
+            {
+                try
+                {
+                    reset();
+                }
+                catch ( Exception e )
+                {
+                    log.error("Trying to reset after reconnection", e);
+                }
+            }
+        }
+    };
+
+    private enum State
+    {
+        LATENT,
+        STARTED,
+        CLOSED
+    }
+
+    EnsembleTracker(CuratorFramework client, EnsembleProvider ensembleProvider)
+    {
+        this.client = client.newWatcherRemoveCuratorFramework();
+        this.ensembleProvider = ensembleProvider;
+    }
+
+    public void start() throws Exception
+    {
+        Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Cannot be started more than once");
+        client.getConnectionStateListenable().addListener(connectionStateListener);
+        reset();
+    }
+
+    @Override
+    public void close()
+    {
+        if ( state.compareAndSet(State.STARTED, State.CLOSED) )
+        {
+            client.removeWatchers();
+            client.getConnectionStateListenable().removeListener(connectionStateListener);
+        }
+    }
+
+    @Override
+    public void process(WatchedEvent event) throws Exception
+    {
+        if ( event.getType() == Watcher.Event.EventType.NodeDataChanged )
+        {
+            reset();
+        }
+    }
+
+    /**
+     * Return the current quorum config
+     *
+     * @return config
+     */
+    public QuorumVerifier getCurrentConfig()
+    {
+        return currentConfig.get();
+    }
+
+    private void reset() throws Exception
+    {
+        if ( client.getState() == CuratorFrameworkState.STARTED )
+        {
+            BackgroundCallback backgroundCallback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    if ( (event.getType() == CuratorEventType.GET_CONFIG) && (event.getResultCode() == KeeperException.Code.OK.intValue()) )
+                    {
+                        processConfigData(event.getData());
+                    }
+                }
+            };
+            client.getConfig().usingWatcher(this).inBackground(backgroundCallback).forEnsemble();
+        }
+    }
+
+    @VisibleForTesting
+    public static String configToConnectionString(QuorumVerifier data) throws Exception
+    {
+        StringBuilder sb = new StringBuilder();
+        for ( QuorumPeer.QuorumServer server : data.getAllMembers().values() )
+        {
+            if ( sb.length() != 0 )
+            {
+                sb.append(",");
+            }
+            sb.append(server.clientAddr.getAddress().getHostAddress()).append(":").append(server.clientAddr.getPort());
+        }
+
+        return sb.toString();
+    }
+
+    private void processConfigData(byte[] data) throws Exception
+    {
+        log.info("New config event received: " + Arrays.toString(data));
+
+        Properties properties = new Properties();
+        properties.load(new ByteArrayInputStream(data));
+        QuorumMaj newConfig = new QuorumMaj(properties);
+        currentConfig.set(newConfig);
+
+        String connectionString = configToConnectionString(newConfig);
+        if ( connectionString.trim().length() > 0 )
+        {
+            ensembleProvider.setConnectionString(connectionString);
+        }
+        else
+        {
+            log.debug("Ignoring new config as it is empty");
+        }
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
index d4a059d..a6316ac 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
@@ -131,7 +131,7 @@
             public void processResult(int rc, String path, Object ctx, Stat stat)
             {
                 trace.commit();
-                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.EXISTS, rc, path, null, ctx, stat, null, null, null, null);
+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.EXISTS, rc, path, null, ctx, stat, null, null, null, null, null);
                 client.processBackgroundOperation(operationAndData, event);
             }
         };
@@ -141,7 +141,7 @@
         }
         else
         {
-            client.getZooKeeper().exists(operationAndData.getData(), watching.getWatcher(), callback, backgrounding.getContext());
+            client.getZooKeeper().exists(operationAndData.getData(), watching.getWatcher(client, operationAndData.getData()), callback, backgrounding.getContext());
         }
     }
 
@@ -223,7 +223,7 @@
                     }
                     else
                     {
-                        returnStat = client.getZooKeeper().exists(path, watching.getWatcher());
+                        returnStat = client.getZooKeeper().exists(path, watching.getWatcher(client, path));
                     }
                     return returnStat;
                 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExtractingCuratorOp.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExtractingCuratorOp.java
new file mode 100644
index 0000000..7a5db69
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExtractingCuratorOp.java
@@ -0,0 +1,54 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.api.transaction.TypeAndPath;
+import org.apache.zookeeper.Op;
+
+class ExtractingCuratorOp implements CuratorOp
+{
+    private final CuratorMultiTransactionRecord record = new CuratorMultiTransactionRecord();
+
+    CuratorMultiTransactionRecord getRecord()
+    {
+        return record;
+    }
+
+    @Override
+    public TypeAndPath getTypeAndPath()
+    {
+        validate();
+        return record.getMetadata(0);
+    }
+
+    @Override
+    public Op get()
+    {
+        validate();
+        return record.iterator().next();
+    }
+
+    private void validate()
+    {
+        Preconditions.checkArgument(record.size() > 0, "No operation has been added");
+        Preconditions.checkArgument(record.size() == 1, "Multiple operations added");
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedDeleteManager.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedDeleteManager.java
index 4e6fcf3..934ae40 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedDeleteManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedDeleteManager.java
@@ -19,46 +19,18 @@
 package org.apache.curator.framework.imps;
 
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.utils.ThreadUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-class FailedDeleteManager
+class FailedDeleteManager extends FailedOperationManager<String>
 {
-    private final Logger log = LoggerFactory.getLogger(getClass());
-    private final CuratorFramework client;
-    
-    volatile FailedDeleteManagerListener debugListener = null;
-    
-    interface FailedDeleteManagerListener
-    {
-       public void pathAddedForDelete(String path);
-    }
-
     FailedDeleteManager(CuratorFramework client)
     {
-        this.client = client;
+        super(client);
     }
 
-    void addFailedDelete(String path)
+    @Override
+    protected void executeGuaranteedOperationInBackground(String path)
+            throws Exception
     {
-        if ( debugListener != null )
-        {
-            debugListener.pathAddedForDelete(path);
-        }
-
-        if ( client.getState() == CuratorFrameworkState.STARTED )
-        {
-            log.debug("Path being added to guaranteed delete set: " + path);
-            try
-            {
-                client.delete().guaranteed().inBackground().forPath(path);
-            }
-            catch ( Exception e )
-            {
-                ThreadUtils.checkInterrupted(e);
-                addFailedDelete(path);
-            }
-        }
+        client.delete().guaranteed().inBackground().forPath(path);
     }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedOperationManager.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedOperationManager.java
new file mode 100644
index 0000000..c09e2ec
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedOperationManager.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.utils.ThreadUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+abstract class FailedOperationManager<T>
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    protected final CuratorFramework client;
+    
+    @VisibleForTesting
+    volatile FailedOperationManagerListener<T> debugListener = null;
+    
+    interface FailedOperationManagerListener<T>
+    {
+       public void pathAddedForGuaranteedOperation(T detail);
+    }
+
+    FailedOperationManager(CuratorFramework client)
+    {
+        this.client = client;
+    }
+
+    void addFailedOperation(T details)
+    {
+        if ( debugListener != null )
+        {
+            debugListener.pathAddedForGuaranteedOperation(details);
+        }
+        
+        
+        if ( client.getState() == CuratorFrameworkState.STARTED )
+        {
+            log.debug("Details being added to guaranteed operation set: " + details);
+            try
+            {
+                executeGuaranteedOperationInBackground(details);
+            }
+            catch ( Exception e )
+            {
+                ThreadUtils.checkInterrupted(e);
+                addFailedOperation(details);
+            }
+        }
+    }
+    
+    protected abstract void executeGuaranteedOperationInBackground(T details) throws Exception;
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedRemoveWatchManager.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedRemoveWatchManager.java
new file mode 100644
index 0000000..f635660
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/FailedRemoveWatchManager.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.zookeeper.Watcher;
+
+class FailedRemoveWatchManager extends FailedOperationManager<FailedRemoveWatchManager.FailedRemoveWatchDetails>
+{
+    FailedRemoveWatchManager(CuratorFramework client)
+    {
+        super(client);
+    }
+
+    @Override
+    protected void executeGuaranteedOperationInBackground(FailedRemoveWatchDetails details)
+            throws Exception
+    {
+        if(details.watcher == null)
+        {
+            client.watches().removeAll().guaranteed().inBackground().forPath(details.path);
+        }
+        else
+        {
+            client.watches().remove(details.watcher).guaranteed().inBackground().forPath(details.path);
+        }
+    }
+    
+    static class FailedRemoveWatchDetails
+    {
+        public final String path;
+        public final Watcher watcher;
+        
+        public FailedRemoveWatchDetails(String path, Watcher watcher)
+        {
+            this.path = path;
+            this.watcher = watcher;
+        }
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/FindAndDeleteProtectedNodeInBackground.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/FindAndDeleteProtectedNodeInBackground.java
index 5b0f47c..608a005 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/FindAndDeleteProtectedNodeInBackground.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/FindAndDeleteProtectedNodeInBackground.java
@@ -100,7 +100,7 @@
 
                 if ( rc != KeeperException.Code.OK.intValue() )
                 {
-                    CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.CHILDREN, rc, path, null, o, stat, null, strings, null, null);
+                    CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.CHILDREN, rc, path, null, o, stat, null, strings, null, null, null);
                     client.processBackgroundOperation(operationAndData, event);
                 }
             }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetACLBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetACLBuilderImpl.java
index 250c2c8..f65c933 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetACLBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetACLBuilderImpl.java
@@ -104,7 +104,7 @@
             public void processResult(int rc, String path, Object ctx, List<ACL> acl, Stat stat)
             {
                 trace.commit();
-                CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.GET_ACL, rc, path, null, ctx, stat, null, null, null, acl);
+                CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.GET_ACL, rc, path, null, ctx, stat, null, null, null, acl, null);
                 client.processBackgroundOperation(operationAndData, event);
             }
         };
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetChildrenBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetChildrenBuilderImpl.java
index 16f6d4b..7929ba3 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetChildrenBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetChildrenBuilderImpl.java
@@ -162,7 +162,7 @@
                 {
                     strings = Lists.newArrayList();
                 }
-                CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.CHILDREN, rc, path, null, o, stat, null, strings, null, null);
+                CuratorEventImpl event = new CuratorEventImpl(client, CuratorEventType.CHILDREN, rc, path, null, o, stat, null, strings, null, null, null);
                 client.processBackgroundOperation(operationAndData, event);
             }
         };
@@ -172,7 +172,7 @@
         }
         else
         {
-            client.getZooKeeper().getChildren(operationAndData.getData(), watching.getWatcher(), callback, backgrounding.getContext());
+            client.getZooKeeper().getChildren(operationAndData.getData(), watching.getWatcher(client, operationAndData.getData()), callback, backgrounding.getContext());
         }
     }
 
@@ -211,7 +211,7 @@
                     }
                     else
                     {
-                        children = client.getZooKeeper().getChildren(path, watching.getWatcher(), responseStat);
+                        children = client.getZooKeeper().getChildren(path, watching.getWatcher(client, path), responseStat);
                     }
                     return children;
                 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetConfigBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetConfigBuilderImpl.java
new file mode 100644
index 0000000..09cb0ab
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetConfigBuilderImpl.java
@@ -0,0 +1,295 @@
+/**
+ * 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.TimeTrace;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.BackgroundEnsembleable;
+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.Ensembleable;
+import org.apache.curator.framework.api.GetConfigBuilder;
+import org.apache.curator.framework.api.WatchBackgroundEnsembleable;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.Stat;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executor;
+
+public class GetConfigBuilderImpl implements GetConfigBuilder, BackgroundOperation<Void>
+{
+    private final CuratorFrameworkImpl client;
+
+    private Backgrounding backgrounding;
+    private Watching watching;
+    private Stat stat;
+
+    public GetConfigBuilderImpl(CuratorFrameworkImpl client)
+    {
+        this.client = client;
+        backgrounding = new Backgrounding();
+        watching = new Watching();
+    }
+
+    @Override
+    public WatchBackgroundEnsembleable<byte[]> storingStatIn(Stat stat)
+    {
+        this.stat = stat;
+        return new WatchBackgroundEnsembleable<byte[]>()
+        {
+            @Override
+            public Ensembleable<byte[]> inBackground()
+            {
+                return GetConfigBuilderImpl.this.inBackground();
+            }
+
+            @Override
+            public Ensembleable<byte[]> inBackground(Object context)
+            {
+                return GetConfigBuilderImpl.this.inBackground(context);
+            }
+
+            @Override
+            public Ensembleable<byte[]> inBackground(BackgroundCallback callback)
+            {
+                return GetConfigBuilderImpl.this.inBackground(callback);
+            }
+
+            @Override
+            public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Object context)
+            {
+                return GetConfigBuilderImpl.this.inBackground(callback, context);
+            }
+
+            @Override
+            public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Executor executor)
+            {
+                return GetConfigBuilderImpl.this.inBackground(callback, executor);
+            }
+
+            @Override
+            public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Object context, Executor executor)
+            {
+                return GetConfigBuilderImpl.this.inBackground(callback, context, executor);
+            }
+
+            @Override
+            public byte[] forEnsemble() throws Exception
+            {
+                return GetConfigBuilderImpl.this.forEnsemble();
+            }
+
+            @Override
+            public BackgroundEnsembleable<byte[]> watched()
+            {
+                return GetConfigBuilderImpl.this.watched();
+            }
+
+            @Override
+            public BackgroundEnsembleable<byte[]> usingWatcher(Watcher watcher)
+            {
+                return GetConfigBuilderImpl.this.usingWatcher(watcher);
+            }
+
+            @Override
+            public BackgroundEnsembleable<byte[]> usingWatcher(CuratorWatcher watcher)
+            {
+                return GetConfigBuilderImpl.this.usingWatcher(watcher);
+            }
+        };
+    }
+
+    @Override
+    public BackgroundEnsembleable<byte[]> watched()
+    {
+        watching = new Watching(true);
+        return new InternalBackgroundEnsembleable();
+    }
+
+    @Override
+    public BackgroundEnsembleable<byte[]> usingWatcher(Watcher watcher)
+    {
+        watching = new Watching(client, watcher);
+        return new InternalBackgroundEnsembleable();
+    }
+
+    @Override
+    public BackgroundEnsembleable<byte[]> usingWatcher(CuratorWatcher watcher)
+    {
+        watching = new Watching(client, watcher);
+        return new InternalBackgroundEnsembleable();
+    }
+
+    @Override
+    public Ensembleable<byte[]> inBackground()
+    {
+        backgrounding = new Backgrounding(true);
+        return this;
+    }
+
+    @Override
+    public Ensembleable<byte[]> inBackground(Object context)
+    {
+        backgrounding = new Backgrounding(context);
+        return this;
+    }
+
+    @Override
+    public Ensembleable<byte[]> inBackground(BackgroundCallback callback)
+    {
+        backgrounding = new Backgrounding(callback);
+        return this;
+    }
+
+    @Override
+    public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Object context)
+    {
+        backgrounding = new Backgrounding(callback, context);
+        return this;
+    }
+
+    @Override
+    public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Executor executor)
+    {
+        backgrounding = new Backgrounding(callback, executor);
+        return this;
+    }
+
+    @Override
+    public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Object context, Executor executor)
+    {
+        backgrounding = new Backgrounding(client, callback, context, executor);
+        return this;
+    }
+
+    @Override
+    public byte[] forEnsemble() throws Exception
+    {
+        if ( backgrounding.inBackground() )
+        {
+            client.processBackgroundOperation(new OperationAndData<Void>(this, null, backgrounding.getCallback(), null, backgrounding.getContext()), null);
+            return null;
+        }
+        else
+        {
+            return configInForeground();
+        }
+    }
+
+    @Override
+    public void performBackgroundOperation(final OperationAndData<Void> operationAndData) throws Exception
+    {
+        final TimeTrace trace = client.getZookeeperClient().startTracer("GetDataBuilderImpl-Background");
+        AsyncCallback.DataCallback callback = new AsyncCallback.DataCallback()
+        {
+            @Override
+            public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat)
+            {
+                trace.commit();
+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.GET_CONFIG, rc, path, null, ctx, stat, data, null, null, null, null);
+                client.processBackgroundOperation(operationAndData, event);
+            }
+        };
+        if ( watching.isWatched() )
+        {
+            client.getZooKeeper().getConfig(true, callback, backgrounding.getContext());
+        }
+        else
+        {
+            client.getZooKeeper().getConfig(watching.getWatcher(client, ZooDefs.CONFIG_NODE), callback, backgrounding.getContext());
+        }
+    }
+
+    private byte[] configInForeground() throws Exception
+    {
+        TimeTrace trace = client.getZookeeperClient().startTracer("GetConfigBuilderImpl-Foreground");
+        try
+        {
+            return RetryLoop.callWithRetry
+            (
+                client.getZookeeperClient(),
+                new Callable<byte[]>()
+                {
+                    @Override
+                    public byte[] call() throws Exception
+                    {
+                        if ( watching.isWatched() )
+                        {
+                            return client.getZooKeeper().getConfig(true, stat);
+                        }
+                        return client.getZooKeeper().getConfig(watching.getWatcher(client, ZooDefs.CONFIG_NODE), stat);
+                    }
+                }
+            );
+        }
+        finally
+        {
+            trace.commit();
+        }
+    }
+
+    private class InternalBackgroundEnsembleable implements BackgroundEnsembleable<byte[]>
+    {
+        @Override
+        public Ensembleable<byte[]> inBackground()
+        {
+            return GetConfigBuilderImpl.this.inBackground();
+        }
+
+        @Override
+        public Ensembleable<byte[]> inBackground(Object context)
+        {
+            return GetConfigBuilderImpl.this.inBackground(context);
+        }
+
+        @Override
+        public Ensembleable<byte[]> inBackground(BackgroundCallback callback)
+        {
+            return GetConfigBuilderImpl.this.inBackground(callback);
+        }
+
+        @Override
+        public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Object context)
+        {
+            return GetConfigBuilderImpl.this.inBackground(callback, context);
+        }
+
+        @Override
+        public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Executor executor)
+        {
+            return GetConfigBuilderImpl.this.inBackground(callback, executor);
+        }
+
+        @Override
+        public Ensembleable<byte[]> inBackground(BackgroundCallback callback, Object context, Executor executor)
+        {
+            return GetConfigBuilderImpl.this.inBackground(callback, context, executor);
+        }
+
+        @Override
+        public byte[] forEnsemble() throws Exception
+        {
+            return GetConfigBuilderImpl.this.forEnsemble();
+        }
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetDataBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetDataBuilderImpl.java
index 5a8d16c..7a731d3 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/GetDataBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/GetDataBuilderImpl.java
@@ -252,7 +252,7 @@
                         rc = KeeperException.Code.DATAINCONSISTENCY.intValue();
                     }
                 }
-                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.GET_DATA, rc, path, null, ctx, stat, data, null, null, null);
+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.GET_DATA, rc, path, null, ctx, stat, data, null, null, null, null);
                 client.processBackgroundOperation(operationAndData, event);
             }
         };
@@ -262,7 +262,7 @@
         }
         else
         {
-            client.getZooKeeper().getData(operationAndData.getData(), watching.getWatcher(), callback, backgrounding.getContext());
+            client.getZooKeeper().getData(operationAndData.getData(), watching.getWatcher(client, operationAndData.getData()), callback, backgrounding.getContext());
         }
     }
 
@@ -301,7 +301,7 @@
                     }
                     else
                     {
-                        responseData = client.getZooKeeper().getData(path, watching.getWatcher(), responseStat);
+                        responseData = client.getZooKeeper().getData(path, watching.getWatcher(client, path), responseStat);
                     }
                     return responseData;
                 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/InternalConnectionHandler.java
similarity index 80%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/imps/InternalConnectionHandler.java
index bc8e6bf..65669c3 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/InternalConnectionHandler.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.imps;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+interface InternalConnectionHandler
 {
+    void checkNewConnection(CuratorFrameworkImpl client);
+
+    void suspendConnection(CuratorFrameworkImpl client);
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceWatcherMap.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceWatcherMap.java
index e5aecb2..c864f44 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceWatcherMap.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceWatcherMap.java
@@ -19,6 +19,7 @@
 package org.apache.curator.framework.imps;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import org.apache.curator.framework.api.CuratorWatcher;
 import org.apache.zookeeper.Watcher;
@@ -28,10 +29,10 @@
 
 class NamespaceWatcherMap implements Closeable
 {
-    private final ConcurrentMap<Object, NamespaceWatcher> map = CacheBuilder.newBuilder()
+    private final Cache<Object, NamespaceWatcher> cache = CacheBuilder.newBuilder()
         .weakValues()
-        .<Object, NamespaceWatcher>build()
-        .asMap();
+        .<Object, NamespaceWatcher>build();
+    private final ConcurrentMap<Object, NamespaceWatcher> map = cache.asMap();
     private final CuratorFrameworkImpl client;
 
     NamespaceWatcherMap(CuratorFrameworkImpl client)
@@ -42,6 +43,11 @@
     @Override
     public void close()
     {
+        clear();
+    }
+
+    void clear()
+    {
         map.clear();
     }
 
@@ -71,9 +77,16 @@
         return map.remove(key);
     }
 
+    boolean removeWatcher(Object watcher)
+    {
+        //noinspection SuspiciousMethodCalls
+        return map.values().remove(watcher);
+    }
+
     @VisibleForTesting
     boolean isEmpty()
     {
+        cache.cleanUp();
         return map.isEmpty();
     }
 
@@ -92,4 +105,10 @@
         NamespaceWatcher        existingNamespaceWatcher = map.putIfAbsent(watcher, newNamespaceWatcher);
         return (existingNamespaceWatcher != null) ? existingNamespaceWatcher : newNamespaceWatcher;
     }
+
+    @Override
+    public String toString()
+    {
+        return map.toString();
+    }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/OperationAndData.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/OperationAndData.java
index 279eece..5f7b985 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/OperationAndData.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/OperationAndData.java
@@ -40,19 +40,21 @@
     private final AtomicLong sleepUntilTimeMs = new AtomicLong(0);
     private final AtomicLong ordinal = new AtomicLong();
     private final Object context;
+    private final boolean connectionRequired;
 
     interface ErrorCallback<T>
     {
         void retriesExhausted(OperationAndData<T> operationAndData);
     }
-
-    OperationAndData(BackgroundOperation<T> operation, T data, BackgroundCallback callback, ErrorCallback<T> errorCallback, Object context)
+    
+    OperationAndData(BackgroundOperation<T> operation, T data, BackgroundCallback callback, ErrorCallback<T> errorCallback, Object context, boolean connectionRequired)
     {
         this.operation = operation;
         this.data = data;
         this.callback = callback;
         this.errorCallback = errorCallback;
         this.context = context;
+        this.connectionRequired = connectionRequired;
         reset();
     }
 
@@ -62,10 +64,20 @@
         ordinal.set(nextOrdinal.getAndIncrement());
     }
 
+    OperationAndData(BackgroundOperation<T> operation, T data, BackgroundCallback callback, ErrorCallback<T> errorCallback, Object context)
+    {
+        this(operation, data, callback, errorCallback, context, true);
+    }
+
     Object getContext()
     {
         return context;
     }
+    
+    boolean isConnectionRequired()
+    {
+        return connectionRequired;
+    }
 
     void callPerformBackgroundOperation() throws Exception
     {
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
new file mode 100644
index 0000000..e786883
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ReconfigBuilderImpl.java
@@ -0,0 +1,272 @@
+/**
+ * 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 com.google.common.collect.ImmutableList;
+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.data.Stat;
+import org.apache.zookeeper.server.DataTree;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executor;
+
+public class ReconfigBuilderImpl implements ReconfigBuilder, BackgroundOperation<Void>
+{
+    private final CuratorFrameworkImpl client;
+
+    private Backgrounding backgrounding = new Backgrounding();
+    private Stat responseStat;
+    private long fromConfig = -1;
+    private List<String> newMembers;
+    private List<String> joining;
+    private List<String> leaving;
+
+    public ReconfigBuilderImpl(CuratorFrameworkImpl client)
+    {
+        this.client = client;
+    }
+
+    private byte[] forEnsemble() throws Exception
+    {
+        if ( backgrounding.inBackground() )
+        {
+            client.processBackgroundOperation(new OperationAndData<>(this, null, backgrounding.getCallback(), null, backgrounding.getContext()), null);
+            return new byte[0];
+        }
+        else
+        {
+            return ensembleInForeground();
+        }
+    }
+
+    @Override
+    public ReconfigBuilderMain inBackground()
+    {
+        backgrounding = new Backgrounding(true);
+        return this;
+    }
+
+    @Override
+    public ReconfigBuilderMain inBackground(Object context)
+    {
+        backgrounding = new Backgrounding(context);
+        return this;
+    }
+
+    @Override
+    public ReconfigBuilderMain inBackground(BackgroundCallback callback)
+    {
+        backgrounding = new Backgrounding(callback);
+        return this;
+    }
+
+    @Override
+    public ReconfigBuilderMain inBackground(BackgroundCallback callback, Object context)
+    {
+        backgrounding = new Backgrounding(callback, context);
+        return this;
+    }
+
+    @Override
+    public ReconfigBuilderMain inBackground(BackgroundCallback callback, Executor executor)
+    {
+        backgrounding = new Backgrounding(callback, executor);
+        return this;
+    }
+
+    @Override
+    public ReconfigBuilderMain inBackground(BackgroundCallback callback, Object context, Executor executor)
+    {
+        backgrounding = new Backgrounding(client, callback, context, executor);
+        return this;
+    }
+
+    @Override
+    public StatConfigureEnsembleable withNewMembers(String... server)
+    {
+        return withNewMembers((server != null) ? Arrays.asList(server) : null);
+    }
+
+    @Override
+    public StatConfigureEnsembleable withNewMembers(List<String> servers)
+    {
+        newMembers = (servers != null) ? ImmutableList.copyOf(servers) : ImmutableList.<String>of();
+        return new StatConfigureEnsembleable()
+        {
+            @Override
+            public Ensembleable<byte[]> fromConfig(long config) throws Exception
+            {
+                fromConfig = config;
+                return this;
+            }
+
+            @Override
+            public byte[] forEnsemble() throws Exception
+            {
+                return ReconfigBuilderImpl.this.forEnsemble();
+            }
+
+            @Override
+            public ConfigureEnsembleable storingStatIn(Stat stat)
+            {
+                responseStat = stat;
+                return this;
+            }
+        };
+    }
+
+    @Override
+    public LeaveStatConfigEnsembleable joining(String... server)
+    {
+        return joining((server != null) ? Arrays.asList(server) : null);
+    }
+
+    @Override
+    public LeaveStatConfigEnsembleable joining(List<String> servers)
+    {
+        joining = (servers != null) ? ImmutableList.copyOf(servers) : ImmutableList.<String>of();
+
+        return new LeaveStatConfigEnsembleable()
+        {
+            @Override
+            public byte[] forEnsemble() throws Exception
+            {
+                return ReconfigBuilderImpl.this.forEnsemble();
+            }
+
+            @Override
+            public ConfigureEnsembleable storingStatIn(Stat stat)
+            {
+                responseStat = stat;
+                return this;
+            }
+
+            @Override
+            public Ensembleable<byte[]> fromConfig(long config) throws Exception
+            {
+                fromConfig = config;
+                return this;
+            }
+
+            @Override
+            public JoinStatConfigEnsembleable leaving(String... server)
+            {
+                return ReconfigBuilderImpl.this.leaving(server);
+            }
+
+            @Override
+            public JoinStatConfigEnsembleable leaving(List<String> servers)
+            {
+                return ReconfigBuilderImpl.this.leaving(servers);
+            }
+        };
+    }
+
+    @Override
+    public JoinStatConfigEnsembleable leaving(String... server)
+    {
+        return leaving((server != null) ? Arrays.asList(server) : null);
+    }
+
+    @Override
+    public JoinStatConfigEnsembleable leaving(List<String> servers)
+    {
+        leaving = (servers != null) ? ImmutableList.copyOf(servers) : ImmutableList.<String>of();
+
+        return new JoinStatConfigEnsembleable()
+        {
+            @Override
+            public byte[] forEnsemble() throws Exception
+            {
+                return ReconfigBuilderImpl.this.forEnsemble();
+            }
+
+            @Override
+            public ConfigureEnsembleable storingStatIn(Stat stat)
+            {
+                responseStat = stat;
+                return this;
+            }
+
+            @Override
+            public Ensembleable<byte[]> fromConfig(long config) throws Exception
+            {
+                fromConfig = config;
+                return this;
+            }
+
+            @Override
+            public LeaveStatConfigEnsembleable joining(String... server)
+            {
+                return joining((server != null) ? Arrays.asList(server) : null);
+            }
+
+            @Override
+            public LeaveStatConfigEnsembleable joining(List<String> servers)
+            {
+                return ReconfigBuilderImpl.this.joining(servers);
+            }
+        };
+    }
+
+    @Override
+    public void performBackgroundOperation(final OperationAndData<Void> data) throws Exception
+    {
+        final TimeTrace trace = client.getZookeeperClient().startTracer("ReconfigBuilderImpl-Background");
+        AsyncCallback.DataCallback callback = new AsyncCallback.DataCallback()
+        {
+            @Override
+            public void processResult(int rc, String path, Object ctx, byte[] bytes, Stat stat)
+            {
+                trace.commit();
+                if ( (responseStat != null) && (stat != null) )
+                {
+                    DataTree.copyStat(stat, responseStat);
+                }
+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.RECONFIG, rc, path, null, ctx, stat, bytes, null, null, null, null);
+                client.processBackgroundOperation(data, event);
+            }
+        };
+        client.getZooKeeper().reconfig(joining, leaving, newMembers, fromConfig, callback, backgrounding.getContext());
+    }
+
+    private byte[] ensembleInForeground() throws Exception
+    {
+        TimeTrace trace = client.getZookeeperClient().startTracer("ReconfigBuilderImpl-Foreground");
+        byte[] responseData = RetryLoop.callWithRetry
+            (
+                client.getZookeeperClient(),
+                new Callable<byte[]>()
+                {
+                    @Override
+                    public byte[] call() throws Exception
+                    {
+                        return client.getZooKeeper().reconfig(joining, leaving, newMembers, fromConfig, responseStat);
+                    }
+                }
+            );
+        trace.commit();
+        return responseData;
+    }
+}
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
new file mode 100644
index 0000000..c1772f1
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/RemoveWatchesBuilderImpl.java
@@ -0,0 +1,321 @@
+/**

+ * Licensed to the Apache Software Foundation (ASF) under one

+ * or more contributor license agreements.  See the NOTICE file

+ * distributed with this work for additional information

+ * regarding copyright ownership.  The ASF licenses this file

+ * to you under the Apache License, Version 2.0 (the

+ * "License"); you may not use this file except in compliance

+ * with the License.  You may obtain a copy of the License at

+ *

+ *   http://www.apache.org/licenses/LICENSE-2.0

+ *

+ * Unless required by applicable law or agreed to in writing,

+ * software distributed under the License is distributed on an

+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

+ * KIND, either express or implied.  See the License for the

+ * specific language governing permissions and limitations

+ * under the License.

+ */

+package org.apache.curator.framework.imps;

+

+import java.util.concurrent.Callable;

+import java.util.concurrent.Executor;

+

+import org.apache.curator.RetryLoop;

+import org.apache.curator.TimeTrace;

+import org.apache.curator.framework.api.BackgroundCallback;

+import org.apache.curator.framework.api.BackgroundPathable;

+import org.apache.curator.framework.api.BackgroundPathableQuietlyable;

+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.RemoveWatchesLocal;

+import org.apache.curator.framework.api.RemoveWatchesBuilder;

+import org.apache.curator.framework.api.RemoveWatchesType;

+import org.apache.curator.utils.DebugUtils;

+import org.apache.zookeeper.AsyncCallback;

+import org.apache.zookeeper.KeeperException;

+import org.apache.zookeeper.Watcher;

+import org.apache.zookeeper.Watcher.WatcherType;

+import org.apache.zookeeper.ZooKeeper;

+

+

+public class RemoveWatchesBuilderImpl implements RemoveWatchesBuilder, RemoveWatchesType, RemoveWatchesLocal, BackgroundOperation<String>

+{

+    private CuratorFrameworkImpl client;

+    private Watcher watcher;

+    private WatcherType watcherType;

+    private boolean guaranteed;

+    private boolean local;

+    private boolean quietly;    

+    private Backgrounding backgrounding;

+    

+    public RemoveWatchesBuilderImpl(CuratorFrameworkImpl client)

+    {

+        this.client = client;

+        this.watcher = null;

+        this.watcherType = WatcherType.Any;

+        this.guaranteed = false;

+        this.local = false;

+        this.quietly = false;

+        this.backgrounding = new Backgrounding();

+    }

+

+    void internalRemoval(Watcher watcher, String path) throws Exception

+    {

+        this.watcher = watcher;

+        watcherType = WatcherType.Any;

+        quietly = true;

+        guaranteed = true;

+        if ( Boolean.getBoolean(DebugUtils.PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND) )

+        {

+            this.backgrounding = new Backgrounding();

+            pathInForeground(path);

+        }

+        else

+        {

+            this.backgrounding = new Backgrounding(true);

+            pathInBackground(path);

+        }

+    }

+

+    @Override

+    public RemoveWatchesType remove(Watcher watcher)

+    {

+        if(watcher == null) {

+            this.watcher = null;

+        } else {

+            //Try and get the namespaced version of the watcher.

+            this.watcher = client.getNamespaceWatcherMap().get(watcher);

+            

+            //If this is not present then default to the original watcher. This shouldn't happen in practice unless the user

+            //has added a watch directly to the ZK client rather than via the CuratorFramework.

+            if(this.watcher == null) {

+                this.watcher = watcher;

+            }

+        }

+

+        return this;

+    }

+    

+    @Override

+    public RemoveWatchesType remove(CuratorWatcher watcher)

+    {

+        this.watcher = watcher == null ? null : client.getNamespaceWatcherMap().get(watcher);

+        return this;

+    }    

+

+    @Override

+    public RemoveWatchesType removeAll()

+    {

+        this.watcher = null;

+        return this;

+    }

+

+    @Override

+    public RemoveWatchesLocal ofType(WatcherType watcherType)

+    {

+        this.watcherType = watcherType;

+        

+        return this;

+    }

+

+    @Override

+    public Pathable<Void> inBackground(BackgroundCallback callback, Object context)

+    {

+        backgrounding = new Backgrounding(callback, context);

+        return this;

+    }

+

+    @Override

+    public Pathable<Void> inBackground(BackgroundCallback callback, Object context, Executor executor)

+    {

+        backgrounding = new Backgrounding(client, callback, context, executor);

+        return this;

+    }

+

+    @Override

+    public Pathable<Void> inBackground(BackgroundCallback callback)

+    {

+        backgrounding = new Backgrounding(callback);

+        return this;

+    }

+

+    @Override

+    public Pathable<Void> inBackground(BackgroundCallback callback, Executor executor)

+    {

+        backgrounding = new Backgrounding(client, callback, executor);

+        return this;

+    }

+

+    @Override

+    public Pathable<Void> inBackground()

+    {

+        backgrounding = new Backgrounding(true);

+        return this;

+    }

+

+    @Override

+    public Pathable<Void> inBackground(Object context)

+    {

+        backgrounding = new Backgrounding(context);

+        return this;

+    }

+    

+    @Override

+    public RemoveWatchesLocal guaranteed()

+    {

+        guaranteed = true;

+        return this;

+    }    

+

+    @Override

+    public BackgroundPathableQuietlyable<Void> locally()

+    {

+        local = true;

+        return this;

+    }

+    

+    @Override

+    public BackgroundPathable<Void> quietly()

+    {

+        quietly = true;

+        return this;

+    }

+    

+    @Override

+    public Void forPath(String path) throws Exception

+    {

+        final String adjustedPath = client.fixForNamespace(path);

+        

+        if(backgrounding.inBackground())

+        {

+            pathInBackground(adjustedPath);

+        }

+        else

+        {

+            pathInForeground(adjustedPath);

+        }        

+        

+        return null;

+    }    

+    

+    private void pathInBackground(final String path)

+    {

+        OperationAndData.ErrorCallback<String>  errorCallback = null;

+        

+        //Only need an error callback if we're in guaranteed mode

+        if(guaranteed)

+        {

+            errorCallback = new OperationAndData.ErrorCallback<String>()

+            {

+                @Override

+                public void retriesExhausted(OperationAndData<String> operationAndData)

+                {

+                    client.getFailedRemoveWatcherManager().addFailedOperation(new FailedRemoveWatchManager.FailedRemoveWatchDetails(path, watcher));

+                }            

+            };

+        }

+        

+        client.processBackgroundOperation(new OperationAndData<String>(this, path, backgrounding.getCallback(),

+                                                                       errorCallback, backgrounding.getContext(), !local), null);

+    }

+    

+    private void pathInForeground(final String path) throws Exception

+    {

+        //For the local case we don't want to use the normal retry loop and we don't want to block until a connection is available.

+        //We just execute the removeWatch, and if it fails, ZK will just remove local watches.

+        if(local)

+        {

+            ZooKeeper zkClient = client.getZooKeeper();

+            if(watcher == null)

+            {

+                client.getNamespaceWatcherMap().clear();

+                zkClient.removeAllWatches(path, watcherType, local);    

+            }

+            else

+            {

+                client.getNamespaceWatcherMap().removeWatcher(watcher);

+                zkClient.removeWatches(path, watcher, watcherType, local);

+            }

+        }

+        else

+        {

+            RetryLoop.callWithRetry(client.getZookeeperClient(), 

+                    new Callable<Void>()

+                    {

+                        @Override

+                        public Void call() throws Exception

+                        {

+                            try

+                            {

+                                ZooKeeper zkClient = client.getZookeeperClient().getZooKeeper();    

+                                

+                                if(watcher == null)

+                                {

+                                    client.getNamespaceWatcherMap().clear();

+                                    zkClient.removeAllWatches(path, watcherType, local);

+                                }

+                                else

+                                {

+                                    client.getNamespaceWatcherMap().removeWatcher(watcher);

+                                    zkClient.removeWatches(path, watcher, watcherType, local);

+                                }

+                            }

+                            catch(Exception e)

+                            {

+                                if( RetryLoop.isRetryException(e) && guaranteed )

+                                {

+                                    //Setup the guaranteed handler

+                                    client.getFailedRemoveWatcherManager().addFailedOperation(new FailedRemoveWatchManager.FailedRemoveWatchDetails(path, watcher));

+                                    throw e;

+                                }

+                                else if(e instanceof KeeperException.NoWatcherException && quietly)

+                                {

+                                    //Ignore

+                                }

+                                else

+                                {

+                                    //Rethrow

+                                    throw e;

+                                }

+                            }

+                     

+                            return null;

+                        }

+            });

+        }

+    }

+    

+    @Override

+    public void performBackgroundOperation(final OperationAndData<String> operationAndData)

+            throws Exception

+    {

+        final TimeTrace   trace = client.getZookeeperClient().startTracer("RemoteWatches-Background");

+        

+        AsyncCallback.VoidCallback callback = new AsyncCallback.VoidCallback()

+        {

+            @Override

+            public void processResult(int rc, String path, Object ctx)

+            {

+                trace.commit();

+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.REMOVE_WATCHES, rc, path, null, ctx, null, null, null, null, null, null);

+                client.processBackgroundOperation(operationAndData, event);                

+            }

+        };

+        

+        ZooKeeper zkClient = client.getZooKeeper();

+        if(watcher == null)

+        {

+            client.getNamespaceWatcherMap().clear();

+            zkClient.removeAllWatches(operationAndData.getData(), watcherType, local, callback, operationAndData.getContext());

+        }

+        else

+        {

+            client.getNamespaceWatcherMap().removeWatcher(watcher);

+            zkClient.removeWatches(operationAndData.getData(), watcher, watcherType, local, callback, operationAndData.getContext());

+        }

+        

+    }

+}
\ No newline at end of file
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
index f7b2480..17e88f8 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetACLBuilderImpl.java
@@ -138,7 +138,7 @@
                 public void processResult(int rc, String path, Object ctx, Stat stat)
                 {
                     trace.commit();
-                    CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.SET_ACL, rc, path, null, ctx, stat, null, null, null, null);
+                    CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.SET_ACL, rc, path, null, ctx, stat, null, null, null, null, null);
                     client.processBackgroundOperation(operationAndData, event);
                 }
             },
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
index 4117930..7057c11 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SetDataBuilderImpl.java
@@ -28,13 +28,11 @@
 import org.apache.curator.framework.api.SetDataBackgroundVersionable;
 import org.apache.curator.framework.api.SetDataBuilder;
 import org.apache.curator.framework.api.VersionPathAndBytesable;
-import org.apache.curator.framework.api.transaction.CuratorTransactionBridge;
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionSetDataBuilder;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.data.Stat;
-
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
 
@@ -53,12 +51,12 @@
         compress = false;
     }
 
-    TransactionSetDataBuilder   asTransactionSetDataBuilder(final CuratorTransactionImpl curatorTransaction, final CuratorMultiTransactionRecord transaction)
+    <T> TransactionSetDataBuilder<T> asTransactionSetDataBuilder(final T context, final CuratorMultiTransactionRecord transaction)
     {
-        return new TransactionSetDataBuilder()
+        return new TransactionSetDataBuilder<T>()
         {
             @Override
-            public CuratorTransactionBridge forPath(String path, byte[] data) throws Exception
+            public T forPath(String path, byte[] data) throws Exception
             {
                 if ( compress )
                 {
@@ -67,26 +65,26 @@
 
                 String      fixedPath = client.fixForNamespace(path);
                 transaction.add(Op.setData(fixedPath, data, version), OperationType.SET_DATA, path);
-                return curatorTransaction;
+                return context;
             }
 
             @Override
-            public CuratorTransactionBridge forPath(String path) throws Exception
+            public T forPath(String path) throws Exception
             {
                 return forPath(path, client.getDefaultData());
             }
 
             @Override
-            public PathAndBytesable<CuratorTransactionBridge> withVersion(int version)
+            public PathAndBytesable<T> withVersion(int version)
             {
                 SetDataBuilderImpl.this.withVersion(version);
                 return this;
             }
 
             @Override
-            public VersionPathAndBytesable<CuratorTransactionBridge> compressed() {
+            public VersionPathAndBytesable<T> compressed()
+            {
                 compress = true;
-
                 return this;
             }
         };
@@ -219,7 +217,7 @@
                 public void processResult(int rc, String path, Object ctx, Stat stat)
                 {
                     trace.commit();
-                    CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.SET_DATA, rc, path, null, ctx, stat, null, null, null, null);
+                    CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.SET_DATA, rc, path, null, ctx, stat, null, null, null, null, null);
                     client.processBackgroundOperation(operationAndData, event);
                 }
             },
@@ -246,7 +244,7 @@
         Stat        resultStat = null;
         if ( backgrounding.inBackground()  )
         {
-            client.processBackgroundOperation(new OperationAndData<PathAndBytes>(this, new PathAndBytes(path, data), backgrounding.getCallback(), null, backgrounding.getContext()), null);
+            client.processBackgroundOperation(new OperationAndData<>(this, new PathAndBytes(path, data), backgrounding.getCallback(), null, backgrounding.getContext()), null);
         }
         else
         {
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/StandardInternalConnectionHandler.java
similarity index 68%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/imps/StandardInternalConnectionHandler.java
index bc8e6bf..be0c726 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/StandardInternalConnectionHandler.java
@@ -16,10 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.imps;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+class StandardInternalConnectionHandler implements InternalConnectionHandler
 {
+    @Override
+    public void suspendConnection(CuratorFrameworkImpl client)
+    {
+        client.setToSuspended();
+    }
+
+    @Override
+    public void checkNewConnection(CuratorFrameworkImpl client)
+    {
+        client.checkInstanceIndex();
+    }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java
index 09dfbae..be4d33a 100755
--- a/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java
@@ -93,7 +93,7 @@
             public void processResult(int rc, String path, Object ctx)
             {
                 trace.commit();
-                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.SYNC, rc, path, path, ctx, null, null, null, null, null);
+                CuratorEvent event = new CuratorEventImpl(client, CuratorEventType.SYNC, rc, path, path, ctx, null, null, null, null, null, null);
                 client.processBackgroundOperation(operationAndData, event);
             }
         };
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/TransactionOpImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/TransactionOpImpl.java
new file mode 100644
index 0000000..381842b
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/TransactionOpImpl.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.api.transaction.TransactionCheckBuilder;
+import org.apache.curator.framework.api.transaction.TransactionCreateBuilder;
+import org.apache.curator.framework.api.transaction.TransactionDeleteBuilder;
+import org.apache.curator.framework.api.transaction.TransactionOp;
+import org.apache.curator.framework.api.transaction.TransactionSetDataBuilder;
+
+public class TransactionOpImpl implements TransactionOp
+{
+    private final CuratorFrameworkImpl client;
+
+    public TransactionOpImpl(CuratorFrameworkImpl client)
+    {
+        this.client = client;
+    }
+
+    @Override
+    public TransactionCreateBuilder<CuratorOp> create()
+    {
+        ExtractingCuratorOp op = new ExtractingCuratorOp();
+        return new CreateBuilderImpl(client).<CuratorOp>asTransactionCreateBuilder(op, op.getRecord());
+    }
+
+    @Override
+    public TransactionDeleteBuilder<CuratorOp> delete()
+    {
+        ExtractingCuratorOp op = new ExtractingCuratorOp();
+        return new DeleteBuilderImpl(client).<CuratorOp>asTransactionDeleteBuilder(op, op.getRecord());
+    }
+
+    @Override
+    public TransactionSetDataBuilder<CuratorOp> setData()
+    {
+        ExtractingCuratorOp op = new ExtractingCuratorOp();
+        return new SetDataBuilderImpl(client).<CuratorOp>asTransactionSetDataBuilder(op, op.getRecord());
+    }
+
+    @Override
+    public TransactionCheckBuilder<CuratorOp> check()
+    {
+        ExtractingCuratorOp op = new ExtractingCuratorOp();
+        return CuratorTransactionImpl.<CuratorOp>makeTransactionCheckBuilder(client, op, op.getRecord());
+    }
+
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalFacade.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalFacade.java
new file mode 100644
index 0000000..30a992e
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalFacade.java
@@ -0,0 +1,198 @@
+/**
+ * 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.RetryLoop;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.CuratorListener;
+import org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.curator.framework.listen.Listenable;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.utils.DebugUtils;
+import org.apache.curator.utils.EnsurePath;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+
+class WatcherRemovalFacade extends CuratorFrameworkImpl implements WatcherRemoveCuratorFramework
+{
+    private final CuratorFrameworkImpl client;
+    private final WatcherRemovalManager removalManager;
+
+    WatcherRemovalFacade(CuratorFrameworkImpl client)
+    {
+        super(client);
+        this.client = client;
+        removalManager = new WatcherRemovalManager(client, getNamespaceWatcherMap());
+    }
+
+    @Override
+    public WatcherRemoveCuratorFramework newWatcherRemoveCuratorFramework()
+    {
+        return client.newWatcherRemoveCuratorFramework();
+    }
+
+    WatcherRemovalManager getRemovalManager()
+    {
+        return removalManager;
+    }
+
+    @Override
+    public QuorumVerifier getCurrentConfig()
+    {
+        return client.getCurrentConfig();
+    }
+
+    @Override
+    public void removeWatchers()
+    {
+        removalManager.removeWatchers();
+
+        if ( Boolean.getBoolean(DebugUtils.PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY) )
+        {
+            if ( !getNamespaceWatcherMap().isEmpty() )
+            {
+                throw new RuntimeException("NamespaceWatcherMap is not empty: " + getNamespaceWatcherMap());
+            }
+        }
+    }
+
+    @Override
+    WatcherRemovalManager getWatcherRemovalManager()
+    {
+        return removalManager;
+    }
+
+    @Override
+    public CuratorFramework nonNamespaceView()
+    {
+        return client.nonNamespaceView();
+    }
+
+    @Override
+    public CuratorFramework usingNamespace(String newNamespace)
+    {
+        return client.usingNamespace(newNamespace);
+    }
+
+    @Override
+    public String getNamespace()
+    {
+        return client.getNamespace();
+    }
+
+    @Override
+    public void start()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void close()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @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 void sync(String path, Object context)
+    {
+        client.sync(path, context);
+    }
+
+    @Override
+    public CuratorZookeeperClient getZookeeperClient()
+    {
+        return client.getZookeeperClient();
+    }
+
+    @Override
+    RetryLoop newRetryLoop()
+    {
+        return client.newRetryLoop();
+    }
+
+    @Override
+    ZooKeeper getZooKeeper() throws Exception
+    {
+        return client.getZooKeeper();
+    }
+
+    @Override
+    <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operationAndData, CuratorEvent event)
+    {
+        client.processBackgroundOperation(operationAndData, event);
+    }
+
+    @Override
+    void logError(String reason, Throwable e)
+    {
+        client.logError(reason, e);
+    }
+
+    @Override
+    String unfixForNamespace(String path)
+    {
+        return client.unfixForNamespace(path);
+    }
+
+    @Override
+    String fixForNamespace(String path)
+    {
+        return client.fixForNamespace(path);
+    }
+    
+    @Override
+    String fixForNamespace(String path, boolean isSequential)
+    {
+    	return client.fixForNamespace(path, isSequential);
+    }
+
+    @Override
+    public EnsurePath newNamespaceAwareEnsurePath(String path)
+    {
+        return client.newNamespaceAwareEnsurePath(path);
+    }
+
+    @Override
+    FailedDeleteManager getFailedDeleteManager()
+    {
+        return client.getFailedDeleteManager();
+    }
+}
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
new file mode 100644
index 0000000..1e6fe94
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/WatcherRemovalManager.java
@@ -0,0 +1,141 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+public class WatcherRemovalManager
+{
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final CuratorFrameworkImpl client;
+    private final NamespaceWatcherMap namespaceWatcherMap;
+    private final Set<WrappedWatcher> entries = Sets.newHashSet();  // guarded by sync
+
+    WatcherRemovalManager(CuratorFrameworkImpl client, NamespaceWatcherMap namespaceWatcherMap)
+    {
+        this.client = client;
+        this.namespaceWatcherMap = namespaceWatcherMap;
+    }
+
+    synchronized Watcher add(String path, Watcher watcher)
+    {
+        path = Preconditions.checkNotNull(path, "path cannot be null");
+        watcher = Preconditions.checkNotNull(watcher, "watcher cannot be null");
+
+        WrappedWatcher wrappedWatcher = new WrappedWatcher(watcher, path);
+        entries.add(wrappedWatcher);
+        return wrappedWatcher;
+    }
+
+    @VisibleForTesting
+    synchronized Set<? extends Watcher> getEntries()
+    {
+        return Sets.newHashSet(entries);
+    }
+
+    void removeWatchers()
+    {
+        HashSet<WrappedWatcher> localEntries;
+        synchronized(this)
+        {
+            localEntries = Sets.newHashSet(entries);
+        }
+        for ( WrappedWatcher entry : localEntries )
+        {
+            try
+            {
+                log.debug("Removing watcher for path: " + entry.path);
+                RemoveWatchesBuilderImpl builder = new RemoveWatchesBuilderImpl(client);
+                namespaceWatcherMap.removeWatcher(entry.watcher);
+                builder.internalRemoval(entry, entry.path);
+            }
+            catch ( Exception e )
+            {
+                log.error("Could not remove watcher for path: " + entry.path);
+            }
+        }
+    }
+
+    private synchronized void internalRemove(WrappedWatcher entry)
+    {
+        namespaceWatcherMap.removeWatcher(entry.watcher);
+        entries.remove(entry);
+    }
+
+    private class WrappedWatcher implements Watcher
+    {
+        private final Watcher watcher;
+        private final String path;
+
+        WrappedWatcher(Watcher watcher, String path)
+        {
+            this.watcher = watcher;
+            this.path = path;
+        }
+
+        @Override
+        public void process(WatchedEvent event)
+        {
+            if ( event.getType() != Event.EventType.None )
+            {
+                internalRemove(this);
+            }
+            watcher.process(event);
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if ( this == o )
+            {
+                return true;
+            }
+            if ( o == null || getClass() != o.getClass() )
+            {
+                return false;
+            }
+
+            WrappedWatcher entry = (WrappedWatcher)o;
+
+            //noinspection SimplifiableIfStatement
+            if ( !watcher.equals(entry.watcher) )
+            {
+                return false;
+            }
+            return path.equals(entry.path);
+
+        }
+
+        @Override
+        public int hashCode()
+        {
+            int result = watcher.hashCode();
+            result = 31 * result + path.hashCode();
+            return result;
+        }
+    }
+}
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 a9d0ab1..4bebbd5 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
@@ -50,8 +50,12 @@
         watched = false;
     }
 
-    Watcher getWatcher()
+    Watcher getWatcher(CuratorFrameworkImpl client, String unfixedPath)
     {
+        if ( (watcher != null) && (client.getWatcherRemovalManager() != null) )
+        {
+            return client.getWatcherRemovalManager().add(unfixedPath, watcher);
+        }
         return watcher;
     }
 
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
index 3ca1d66..f9f245a 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
@@ -18,7 +18,10 @@
  */
 package org.apache.curator.framework.state;
 
+import org.apache.curator.connection.ConnectionHandlingPolicy;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
 
 /**
  * Represents state changes in the connection to ZK
@@ -39,8 +42,7 @@
 
     /**
      * There has been a loss of connection. Leaders, locks, etc. should suspend
-     * until the connection is re-established. If the connection times-out you will
-     * receive a {@link #LOST} notice
+     * until the connection is re-established.
      */
     SUSPENDED
     {
@@ -62,9 +64,20 @@
     },
 
     /**
-     * The connection is confirmed to be lost. Close any locks, leaders, etc. and
-     * attempt to re-create them. NOTE: it is possible to get a {@link #RECONNECTED}
-     * state after this but you should still consider any locks, etc. as dirty/unstable
+     * <p>
+     *     Curator will set the LOST state when it believes that the ZooKeeper session
+     *     has expired. ZooKeeper connections have a session. When the session expires, clients must take appropriate
+     *     action. In Curator, this is complicated by the fact that Curator internally manages the ZooKeeper
+     *     connection. Curator will set the LOST state when any of the following occurs:
+     *     a) ZooKeeper returns a {@link Watcher.Event.KeeperState#Expired} or {@link KeeperException.Code#SESSIONEXPIRED};
+     *     b) Curator closes the internally managed ZooKeeper instance; c) The session timeout
+     *     elapses during a network partition.
+     * </p>
+     *
+     * <p>
+     *     NOTE: see {@link CuratorFrameworkFactory.Builder#connectionHandlingPolicy(ConnectionHandlingPolicy)} for an important note about a
+     *     change in meaning to LOST since 3.0.0
+     * </p>
      */
     LOST
     {
@@ -87,7 +100,9 @@
         {
             return true;
         }
-    };
+    }
+
+    ;
     
     /**
      * Check if this state indicates that Curator has a connection to ZooKeeper
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateErrorPolicy.java
similarity index 60%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateErrorPolicy.java
index bc8e6bf..9016484 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateErrorPolicy.java
@@ -16,10 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.state;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * Recipes should use the configured error policy to decide how to handle
+ * errors such as {@link ConnectionState} changes.
+ *
+ * @since 3.0.0
+ */
+public interface ConnectionStateErrorPolicy
 {
+    /**
+     * Returns true if the given state should cause the recipe to
+     * act as though the connection has been lost. i.e. locks should
+     * exit, etc.
+     *
+     * @param state the state
+     * @return true/false
+     */
+    boolean isErrorState(ConnectionState state);
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
index 8cc37aa..cbb8d16 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java
@@ -65,6 +65,8 @@
     private final Logger log = LoggerFactory.getLogger(getClass());
     private final BlockingQueue<ConnectionState> eventQueue = new ArrayBlockingQueue<ConnectionState>(QUEUE_SIZE);
     private final CuratorFramework client;
+    private final int sessionTimeoutMs;
+    private final int sessionExpirationPercent;
     private final ListenerContainer<ConnectionStateListener> listeners = new ListenerContainer<ConnectionStateListener>();
     private final AtomicBoolean initialConnectMessageSent = new AtomicBoolean(false);
     private final ExecutorService service;
@@ -72,6 +74,8 @@
 
     // guarded by sync
     private ConnectionState currentConnectionState;
+    // guarded by sync
+    private long startOfSuspendedEpoch = 0;
 
     private enum State
     {
@@ -83,10 +87,14 @@
     /**
      * @param client        the client
      * @param threadFactory thread factory to use or null for a default
+     * @param sessionTimeoutMs the ZK session timeout in milliseconds
+     * @param sessionExpirationPercent percentage of negotiated session timeout to use when simulating a session timeout. 0 means don't simulate at all
      */
-    public ConnectionStateManager(CuratorFramework client, ThreadFactory threadFactory)
+    public ConnectionStateManager(CuratorFramework client, ThreadFactory threadFactory, int sessionTimeoutMs, int sessionExpirationPercent)
     {
         this.client = client;
+        this.sessionTimeoutMs = sessionTimeoutMs;
+        this.sessionExpirationPercent = sessionExpirationPercent;
         if ( threadFactory == null )
         {
             threadFactory = ThreadUtils.newThreadFactory("ConnectionStateManager");
@@ -137,7 +145,7 @@
 
     /**
      * Change to {@link ConnectionState#SUSPENDED} only if not already suspended and not lost
-     * 
+     *
      * @return true if connection is set to SUSPENDED
      */
     public synchronized boolean setToSuspended()
@@ -152,7 +160,7 @@
             return false;
         }
 
-        currentConnectionState = ConnectionState.SUSPENDED;
+        setCurrentConnectionState(ConnectionState.SUSPENDED);
         postState(ConnectionState.SUSPENDED);
 
         return true;
@@ -177,7 +185,7 @@
         {
             return false;
         }
-        currentConnectionState = newConnectionState;
+        setCurrentConnectionState(newConnectionState);
 
         ConnectionState localState = newConnectionState;
         boolean isNegativeMessage = ((newConnectionState == ConnectionState.LOST) || (newConnectionState == ConnectionState.SUSPENDED) || (newConnectionState == ConnectionState.READ_ONLY));
@@ -242,25 +250,37 @@
         {
             try
             {
-                final ConnectionState newState = eventQueue.take();
-
-                if ( listeners.size() == 0 )
+                int lastNegotiatedSessionTimeoutMs = client.getZookeeperClient().getLastNegotiatedSessionTimeoutMs();
+                int useSessionTimeoutMs = (lastNegotiatedSessionTimeoutMs > 0) ? lastNegotiatedSessionTimeoutMs : sessionTimeoutMs;
+                int pollMaxMs = (useSessionTimeoutMs * 2) / 3; // 2/3 of session timeout
+                final ConnectionState newState = eventQueue.poll(pollMaxMs, TimeUnit.MILLISECONDS);
+                if ( newState != null )
                 {
-                    log.warn("There are no ConnectionStateListeners registered.");
-                }
+                    if ( listeners.size() == 0 )
+                    {
+                        log.warn("There are no ConnectionStateListeners registered.");
+                    }
 
-                listeners.forEach
-                    (
-                        new Function<ConnectionStateListener, Void>()
-                        {
-                            @Override
-                            public Void apply(ConnectionStateListener listener)
+                    listeners.forEach
+                        (
+                            new Function<ConnectionStateListener, Void>()
                             {
-                                listener.stateChanged(client, newState);
-                                return null;
+                                @Override
+                                public Void apply(ConnectionStateListener listener)
+                                {
+                                    listener.stateChanged(client, newState);
+                                    return null;
+                                }
                             }
-                        }
-                    );
+                        );
+                }
+                else if ( sessionExpirationPercent > 0 )
+                {
+                    synchronized(this)
+                    {
+                        checkSessionExpiration();
+                    }
+                }
             }
             catch ( InterruptedException e )
             {
@@ -270,4 +290,36 @@
             }
         }
     }
+
+    private void checkSessionExpiration()
+    {
+        if ( (currentConnectionState == ConnectionState.SUSPENDED) && (startOfSuspendedEpoch != 0) )
+        {
+            long elapsedMs = System.currentTimeMillis() - startOfSuspendedEpoch;
+            int lastNegotiatedSessionTimeoutMs = client.getZookeeperClient().getLastNegotiatedSessionTimeoutMs();
+            int useSessionTimeoutMs = (lastNegotiatedSessionTimeoutMs > 0) ? lastNegotiatedSessionTimeoutMs : sessionTimeoutMs;
+            useSessionTimeoutMs = (useSessionTimeoutMs * sessionExpirationPercent) / 100;
+            if ( elapsedMs >= useSessionTimeoutMs )
+            {
+                log.warn(String.format("Session timeout has elapsed while SUSPENDED. Injecting a session expiration. Elapsed ms: %d. Adjusted session timeout ms: %d", elapsedMs, useSessionTimeoutMs));
+                try
+                {
+                    // LOL - this method was proposed by me (JZ) in 2013 for totally unrelated reasons
+                    // it got added to ZK 3.5 and now does exactly what we need
+                    // https://issues.apache.org/jira/browse/ZOOKEEPER-1730
+                    client.getZookeeperClient().getZooKeeper().getTestable().injectSessionExpiration();
+                }
+                catch ( Exception e )
+                {
+                    log.error("Could not inject session expiration", e);
+                }
+            }
+        }
+    }
+
+    private void setCurrentConnectionState(ConnectionState newConnectionState)
+    {
+        currentConnectionState = newConnectionState;
+        startOfSuspendedEpoch = (currentConnectionState == ConnectionState.SUSPENDED) ? System.currentTimeMillis() : 0;
+    }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/state/SessionConnectionStateErrorPolicy.java
similarity index 70%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/state/SessionConnectionStateErrorPolicy.java
index bc8e6bf..c494e2b 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/SessionConnectionStateErrorPolicy.java
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.state;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * This policy treats only {@link ConnectionState#LOST} as an error
+ */
+public class SessionConnectionStateErrorPolicy implements ConnectionStateErrorPolicy
 {
+    @Override
+    public boolean isErrorState(ConnectionState state)
+    {
+        return state == ConnectionState.LOST;
+    }
 }
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/state/StandardConnectionStateErrorPolicy.java
similarity index 66%
copy from curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
copy to curator-framework/src/main/java/org/apache/curator/framework/state/StandardConnectionStateErrorPolicy.java
index bc8e6bf..9346f6e 100644
--- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java
+++ b/curator-framework/src/main/java/org/apache/curator/framework/state/StandardConnectionStateErrorPolicy.java
@@ -16,10 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.curator.framework.api;
+package org.apache.curator.framework.state;
 
-public interface ACLVersionBackgroundPathable<T> extends
-    ACLable<Versionable<BackgroundPathable<T>>>,
-    Versionable<BackgroundPathable<T>>
+/**
+ * This policy treats {@link ConnectionState#SUSPENDED} and {@link ConnectionState#LOST}
+ * as errors
+ */
+public class StandardConnectionStateErrorPolicy implements ConnectionStateErrorPolicy
 {
+    @Override
+    public boolean isErrorState(ConnectionState state)
+    {
+        return ((state == ConnectionState.SUSPENDED) || (state == ConnectionState.LOST));
+    }
 }
diff --git a/curator-framework/src/site/confluence/index.confluence b/curator-framework/src/site/confluence/index.confluence
index efde81b..13df0de 100644
--- a/curator-framework/src/site/confluence/index.confluence
+++ b/curator-framework/src/site/confluence/index.confluence
@@ -6,6 +6,8 @@
 * Automatic connection management:
 ** There are potential error cases that require ZooKeeper clients to recreate a connection and/or retry operations. Curator
  automatically and transparently (mostly) handles these cases.
+** Watches for NodeDataChanged events and calls updateServerList() as needed.
+** Watches are automatically removed by Curator recipes
 * Cleaner API:
 ** simplifies the raw ZooKeeper methods, events, etc.
 ** provides a modern, fluent interface
@@ -42,7 +44,12 @@
 |getData()|Begins an operation to get a ZNode's data. Call additional methods (watch, background or get stat) and finalize the operation by calling forPath()|
 |setData()|Begins an operation to set a ZNode's data. Call additional methods (version or background) and finalize the operation by calling forPath()|
 |getChildren()|Begins an operation to get a ZNode's list of children ZNodes. Call additional methods (watch, background or get stat) and finalize the operation by calling forPath()|
-|inTransaction()|Begins an atomic ZooKeeper transaction. Combine create, setData, check, and/or delete operations and then commit() as a unit.|
+|transactionOp()|Used to allocate operations to be used with transaction().|
+|transaction()|Atomically submit a set of operations as a transaction.|
+|getACL()|Begins an operation to return a ZNode's ACL settings. Call additional methods and finalize the operation by calling forPath()|
+|setACL()|Begins an operation to set a ZNode's ACL settings. Call additional methods and finalize the operation by calling forPath()|
+|getConfig()|Begins an operation to return the last committed configuration. Call additional methods and finalize the operation by calling forEnsemble()|
+|reconfig()|Begins an operation to change the configuration. Call additional methods and finalize the operation by calling forEnsemble()|
 
 h3. Notifications
 Notifications for background operations and watches are published via the ClientListener interface. You register listeners with the
@@ -59,10 +66,16 @@
 |CREATE|getResultCode() and getPath()|
 |DELETE|getResultCode() and getPath()|
 |EXISTS|getResultCode(), getPath() and getStat()|
-|GET_DATA|getResultCode(), getPath(), getStat() and getData()|
-|SET_DATA|getResultCode(), getPath() and getStat()|
+|GET\_DATA|getResultCode(), getPath(), getStat() and getData()|
+|SET\_DATA|getResultCode(), getPath() and getStat()|
 |CHILDREN|getResultCode(), getPath(), getStat(), getChildren()|
+|SYNC|getResultCode(), getStat()|
+|GET\_ACL|getResultCode(), getACLList()|
+|SET\_ACL|getResultCode()|
+|TRANSACTION|getResultCode(), getOpResults()|
 |WATCHED|getWatchedEvent()|
+|GET\_CONFIG|getResultCode(), getData()|
+|RECONFIG|getResultCode(), getData()|
 
 h2. Namespaces
 Because a ZooKeeper cluster is a shared environment, it's vital that a namespace convention is
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
index a6dc7ab..01646fc 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
@@ -114,6 +114,7 @@
         Timing timing = new Timing();
         CuratorFramework client = CuratorFrameworkFactory.builder().
             connectString(server.getConnectString()).
+            sessionTimeoutMs(timing.session()).
             retryPolicy(new RetryOneTime(1)).
             build();
 
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransaction.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransactionNew.java
similarity index 78%
copy from curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransaction.java
copy to curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransactionNew.java
index c18af99..d302119 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransaction.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransactionNew.java
@@ -18,18 +18,16 @@
  */
 package org.apache.curator.framework.imps;
 
-import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.api.CompressionProvider;
+import org.apache.curator.framework.api.transaction.CuratorOp;
 import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class TestCompressionInTransaction extends BaseClassForTests
+public class TestCompressionInTransactionNew extends BaseClassForTests
 {
     @Test
     public void testSetData() throws Exception
@@ -43,11 +41,13 @@
             client.start();
 
             //Create uncompressed data in a transaction
-            client.inTransaction().create().forPath(path, data).and().commit();
+            CuratorOp op = client.transactionOp().create().forPath(path, data);
+            client.transaction().forOperations(op);
             Assert.assertEquals(data, client.getData().forPath(path));
 
             //Create compressed data in transaction
-            client.inTransaction().setData().compressed().forPath(path, data).and().commit();
+            op = client.transactionOp().setData().compressed().forPath(path, data);
+            client.transaction().forOperations(op);
             Assert.assertEquals(data, client.getData().decompressed().forPath(path));
         }
         finally
@@ -71,16 +71,18 @@
             client.start();
 
             //Create the nodes
-            client.inTransaction().create().compressed().forPath(path1).and().
-            create().forPath(path2).and().commit();
+            CuratorOp op1 = client.transactionOp().create().compressed().forPath(path1);
+            CuratorOp op2 = client.transactionOp().create().forPath(path2);
+            client.transaction().forOperations(op1, op2);
 
             //Check they exist
             Assert.assertNotNull(client.checkExists().forPath(path1));
             Assert.assertNotNull(client.checkExists().forPath(path2));
             
             //Set the nodes, path1 compressed, path2 uncompressed.
-            client.inTransaction().setData().compressed().forPath(path1, data1).and().
-            setData().forPath(path2, data2).and().commit();
+            op1 = client.transactionOp().setData().compressed().forPath(path1, data1);
+            op2 = client.transactionOp().setData().forPath(path2, data2);
+            client.transaction().forOperations(op1, op2);
             
             Assert.assertNotEquals(data1, client.getData().forPath(path1));
             Assert.assertEquals(data1, client.getData().decompressed().forPath(path1));
@@ -107,8 +109,10 @@
         {
             client.start();
 
-            client.inTransaction().create().compressed().forPath(path1, data1).and().
-            create().compressed().forPath(path2, data2).and().commit();
+            CuratorOp op1 = client.transactionOp().create().compressed().forPath(path1, data1);
+            CuratorOp op2 = client.transactionOp().create().compressed().forPath(path2, data2);
+
+            client.transaction().forOperations(op1, op2);
 
             Assert.assertNotEquals(data1, client.getData().forPath(path1));
             Assert.assertEquals(data1, client.getData().decompressed().forPath(path1));
@@ -141,8 +145,9 @@
         {
             client.start();
 
-            client.inTransaction().create().compressed().forPath(path1, data1).and().
-            create().forPath(path2, data2).and().commit();
+            CuratorOp op1 = client.transactionOp().create().compressed().forPath(path1, data1);
+            CuratorOp op2 = client.transactionOp().create().forPath(path2, data2);
+            client.transaction().forOperations(op1, op2);
 
             Assert.assertNotEquals(data1, client.getData().forPath(path1));
             Assert.assertEquals(data1, client.getData().decompressed().forPath(path1));
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransaction.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransactionOld.java
similarity index 97%
rename from curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransaction.java
rename to curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransactionOld.java
index c18af99..ebf591b 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransaction.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCompressionInTransactionOld.java
@@ -29,7 +29,8 @@
 
 import java.util.concurrent.atomic.AtomicInteger;
 
-public class TestCompressionInTransaction extends BaseClassForTests
+@SuppressWarnings("deprecation")
+public class TestCompressionInTransactionOld extends BaseClassForTests
 {
     @Test
     public void testSetData() throws Exception
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
new file mode 100755
index 0000000..4e9e78c
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestCreateReturningStat.java
@@ -0,0 +1,199 @@
+/**

+ * Licensed to the Apache Software Foundation (ASF) under one

+ * or more contributor license agreements.  See the NOTICE file

+ * distributed with this work for additional information

+ * regarding copyright ownership.  The ASF licenses this file

+ * to you under the Apache License, Version 2.0 (the

+ * "License"); you may not use this file except in compliance

+ * with the License.  You may obtain a copy of the License at

+ *

+ *   http://www.apache.org/licenses/LICENSE-2.0

+ *

+ * Unless required by applicable law or agreed to in writing,

+ * software distributed under the License is distributed on an

+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

+ * KIND, either express or implied.  See the License for the

+ * specific language governing permissions and limitations

+ * under the License.

+ */

+package org.apache.curator.framework.imps;

+

+import java.util.concurrent.CountDownLatch;

+import java.util.concurrent.atomic.AtomicReference;

+

+import org.apache.curator.framework.CuratorFramework;

+import org.apache.curator.framework.CuratorFrameworkFactory;

+import org.apache.curator.framework.api.BackgroundCallback;

+import org.apache.curator.framework.api.CuratorEvent;

+import org.apache.curator.framework.api.CuratorEventType;

+import org.apache.curator.framework.api.ExistsBuilder;

+import org.apache.curator.framework.api.PathAndBytesable;

+import org.apache.curator.retry.RetryOneTime;

+import org.apache.curator.test.BaseClassForTests;

+import org.apache.curator.test.Timing;

+import org.apache.curator.utils.CloseableUtils;

+import org.apache.zookeeper.data.Stat;

+import org.testng.Assert;

+import org.testng.annotations.Test;

+

+public class TestCreateReturningStat extends BaseClassForTests

+{

+    private CuratorFramework createClient()

+    {

+        return CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+    }

+    

+    private void compare(CuratorFramework client, String path,

+                         Stat expected) throws Exception

+    {

+        Stat queriedStat = client.checkExists().forPath(path);

+        

+        Assert.assertEquals(queriedStat, expected);

+    }

+    

+    @Test

+    public void testCreateReturningStat() throws Exception

+    {

+        CuratorFramework client = createClient();

+        try

+        {

+            client.start();

+            

+            String path = "/bla";

+            Stat stat = new Stat();

+            client.create().storingStatIn(stat).forPath(path);

+            

+            compare(client, path, stat);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testCreateReturningStatIncludingParents() throws Exception

+    {

+        CuratorFramework client = createClient();

+        try

+        {

+            client.start();

+            

+            String path = "/bla/bla";

+            Stat stat = new Stat();

+            client.create().creatingParentsIfNeeded().storingStatIn(stat).forPath(path);

+            

+            compare(client, path, stat);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testCreateReturningStatIncludingParentsReverse() throws Exception

+    {

+        CuratorFramework client = createClient();

+        try

+        {

+            client.start();

+            

+            String path = "/bla/bla";

+            Stat stat = new Stat();

+            client.create().storingStatIn(stat).creatingParentsIfNeeded().forPath(path);

+            

+            compare(client, path, stat);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testCreateReturningStatCompressed() throws Exception

+    {

+        CuratorFramework client = createClient();

+        try

+        {

+            client.start();

+            

+            String path = "/bla";

+            Stat stat = new Stat();

+            client.create().compressed().storingStatIn(stat).forPath(path);

+            

+            compare(client, path, stat);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testCreateReturningStatWithProtected() throws Exception

+    {

+        CuratorFramework client = createClient();

+        try

+        {

+            client.start();

+            

+            String path = "/bla";

+            Stat stat = new Stat();

+            path = client.create().withProtection().storingStatIn(stat).forPath(path);

+            

+            compare(client, path, stat);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testCreateReturningStatInBackground() throws Exception

+    {

+        Timing timing = new Timing();

+        CuratorFramework client = createClient();

+        try

+        {

+            client.start();

+            

+            String path = "/bla";

+            Stat stat = new Stat();

+

+            final CountDownLatch latch = new CountDownLatch(1);

+            final AtomicReference<Stat> statRef = new AtomicReference<>();

+            BackgroundCallback callback = new BackgroundCallback() {

+                

+                @Override

+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception {

+                    if(event.getType() == CuratorEventType.CREATE)

+                    {

+                        statRef.set(event.getStat());

+                    

+                        latch.countDown();

+                    }

+                }

+            };

+            

+            client.create().storingStatIn(stat).inBackground(callback).forPath(path);

+            

+            if(!timing.awaitLatch(latch))

+            {

+                Assert.fail("Timed out awaing latch");

+            }

+            

+            compare(client, path, statRef.get());

+            compare(client, path, stat);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+}

diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
new file mode 100644
index 0000000..f96592a
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
@@ -0,0 +1,183 @@
+/**
+ * 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 com.google.common.collect.Queues;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.KillSession;
+import org.apache.curator.test.Timing;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public class TestEnabledSessionExpiredState extends BaseClassForTests
+{
+    private final Timing timing = new Timing();
+
+    private CuratorFramework client;
+    private BlockingQueue<ConnectionState> states;
+
+    @BeforeMethod
+    @Override
+    public void setup() throws Exception
+    {
+        super.setup();
+
+        client = CuratorFrameworkFactory.builder()
+            .connectString(server.getConnectString())
+            .connectionTimeoutMs(timing.connection())
+            .sessionTimeoutMs(timing.session())
+            .retryPolicy(new RetryOneTime(1))
+            .build();
+        client.start();
+
+        states = Queues.newLinkedBlockingQueue();
+        ConnectionStateListener listener = new ConnectionStateListener()
+        {
+            @Override
+            public void stateChanged(CuratorFramework client, ConnectionState newState)
+            {
+                states.add(newState);
+            }
+        };
+        client.getConnectionStateListenable().addListener(listener);
+    }
+
+    @AfterMethod
+    @Override
+    public void teardown() throws Exception
+    {
+        try
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+        finally
+        {
+            super.teardown();
+        }
+    }
+
+    @Test
+    public void testResetCausesLost() throws Exception
+    {
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
+        client.checkExists().forPath("/");  // establish initial connection
+
+        client.getZookeeperClient().reset();
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST);
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED);
+    }
+
+    @Test
+    public void testInjectedWatchedEvent() throws Exception
+    {
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        Watcher watcher = new Watcher()
+        {
+            @Override
+            public void process(WatchedEvent event)
+            {
+                if ( event.getType() == Event.EventType.None )
+                {
+                    if ( event.getState() == Event.KeeperState.Expired )
+                    {
+                        latch.countDown();
+                    }
+                }
+            }
+        };
+        client.checkExists().usingWatcher(watcher).forPath("/");
+        server.stop();
+        Assert.assertTrue(timing.forSessionSleep().awaitLatch(latch));
+    }
+
+    @Test
+    public void testKillSession() throws Exception
+    {
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
+
+        KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+
+        Assert.assertEquals(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST);
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED);
+    }
+
+    @Test
+    public void testReconnectWithoutExpiration() throws Exception
+    {
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
+        server.stop();
+        try
+        {
+            client.checkExists().forPath("/");  // any API call that will invoke the retry policy, etc.
+        }
+        catch ( KeeperException.ConnectionLossException ignore )
+        {
+        }
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED);
+        server.restart();
+        client.checkExists().forPath("/");
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED);
+    }
+
+    @Test
+    public void testSessionExpirationFromTimeout() throws Exception
+    {
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
+        server.stop();
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED);
+        Assert.assertEquals(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST);
+    }
+
+    @Test
+    public void testSessionExpirationFromTimeoutWithRestart() throws Exception
+    {
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED);
+        server.stop();
+        timing.forSessionSleep().sleep();
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED);
+        Assert.assertEquals(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST);
+        server.restart();
+        client.checkExists().forPath("/");
+        Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED);
+
+        Assert.assertNull(states.poll(timing.multiple(.5).milliseconds(), TimeUnit.MILLISECONDS));  // there should be no other events
+    }
+
+    @Override
+    protected boolean enabledSessionExpiredStateAware()
+    {
+        return true;
+    }
+}
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFailedDeleteManager.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFailedDeleteManager.java
index 6599745..943529f 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFailedDeleteManager.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFailedDeleteManager.java
@@ -22,7 +22,6 @@
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
-import org.apache.curator.framework.imps.FailedDeleteManager.FailedDeleteManagerListener;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
@@ -291,11 +290,11 @@
         
         final AtomicBoolean pathAdded = new AtomicBoolean(false);
         
-        ((CuratorFrameworkImpl)client).getFailedDeleteManager().debugListener = new FailedDeleteManagerListener()
+        ((CuratorFrameworkImpl)client).getFailedDeleteManager().debugListener = new FailedOperationManager.FailedOperationManagerListener<String>()
         {
             
             @Override
-            public void pathAddedForDelete(String path)
+            public void pathAddedForGuaranteedOperation(String path)
             {
                 pathAdded.set(true);
             }
@@ -325,11 +324,11 @@
         
         final AtomicBoolean pathAdded = new AtomicBoolean(false);
         
-        ((CuratorFrameworkImpl)client).getFailedDeleteManager().debugListener = new FailedDeleteManagerListener()
+        ((CuratorFrameworkImpl)client).getFailedDeleteManager().debugListener = new FailedOperationManager.FailedOperationManagerListener<String>()
         {
             
             @Override
-            public void pathAddedForDelete(String path)
+            public void pathAddedForGuaranteedOperation(String path)
             {
                 pathAdded.set(true);
             }
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 8aa37c0..5addcb7 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,7 +59,7 @@
     @Override
     public void setup() throws Exception
     {
-        System.setProperty("container.checkIntervalMs", "1000");
+        System.setProperty("znode.container.checkIntervalMs", "1000");
         super.setup();
     }
 
@@ -67,7 +67,7 @@
     @Override
     public void teardown() throws Exception
     {
-        System.clearProperty("container.checkIntervalMs");
+        System.clearProperty("znode.container.checkIntervalMs");
         super.teardown();
     }
 
@@ -102,6 +102,81 @@
     }
 
     @Test
+    public void testCreateOrSetData() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            String name = client.create().forPath("/hey", "there".getBytes());
+            Assert.assertEquals(name, "/hey");
+            name = client.create().orSetData().forPath("/hey", "other".getBytes());
+            Assert.assertEquals(name, "/hey");
+            Assert.assertEquals(client.getData().forPath("/hey"), "other".getBytes());
+
+            name = client.create().orSetData().creatingParentsIfNeeded().forPath("/a/b/c", "there".getBytes());
+            Assert.assertEquals(name, "/a/b/c");
+            name = client.create().orSetData().creatingParentsIfNeeded().forPath("/a/b/c", "what".getBytes());
+            Assert.assertEquals(name, "/a/b/c");
+            Assert.assertEquals(client.getData().forPath("/a/b/c"), "what".getBytes());
+
+            final BlockingQueue<CuratorEvent> queue = new LinkedBlockingQueue<>();
+            BackgroundCallback backgroundCallback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    queue.add(event);
+                }
+            };
+            client.create().orSetData().inBackground(backgroundCallback).forPath("/a/b/c", "another".getBytes());
+
+            CuratorEvent event = queue.poll(new Timing().milliseconds(), TimeUnit.MILLISECONDS);
+            Assert.assertNotNull(event);
+            Assert.assertEquals(event.getResultCode(), KeeperException.Code.OK.intValue());
+            Assert.assertEquals(event.getType(), CuratorEventType.CREATE);
+            Assert.assertEquals(event.getPath(), "/a/b/c");
+            Assert.assertEquals(event.getName(), "/a/b/c");
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testQuietDelete() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            client.delete().quietly().forPath("/foo/bar");
+
+            final BlockingQueue<Integer> rc = new LinkedBlockingQueue<>();
+            BackgroundCallback backgroundCallback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    rc.add(event.getResultCode());
+                }
+            };
+            client.delete().quietly().inBackground(backgroundCallback).forPath("/foo/bar/hey");
+
+            Integer code = rc.poll(new Timing().milliseconds(), TimeUnit.MILLISECONDS);
+            Assert.assertNotNull(code);
+            Assert.assertEquals(code.intValue(), KeeperException.Code.OK.intValue());
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
     public void testNamespaceWithWatcher() throws Exception
     {
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java
index 26cc941..83dab6b 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java
@@ -20,6 +20,7 @@
 package org.apache.curator.framework.imps;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Queues;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.BackgroundCallback;
@@ -30,21 +31,25 @@
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.TestingServer;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.KeeperException.Code;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
-import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
 public class TestFrameworkBackground extends BaseClassForTests
 {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
     @Test
     public void testListenerConnectedAtStart() throws Exception
     {
@@ -149,24 +154,21 @@
         {
             client.start();
 
-            final CountDownLatch latch = new CountDownLatch(3);
-            final List<String> paths = Lists.newArrayList();
-            BackgroundCallback callback = new BackgroundCallback()
+            final BlockingQueue<String> paths = Queues.newLinkedBlockingQueue();
+                BackgroundCallback callback = new BackgroundCallback()
             {
                 @Override
                 public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
                 {
                     paths.add(event.getPath());
-                    latch.countDown();
                 }
             };
             client.create().inBackground(callback).forPath("/one");
+            Assert.assertEquals(paths.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), "/one");
             client.create().inBackground(callback).forPath("/one/two");
+            Assert.assertEquals(paths.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), "/one/two");
             client.create().inBackground(callback).forPath("/one/two/three");
-
-            latch.await();
-
-            Assert.assertEquals(paths, Arrays.asList("/one", "/one/two", "/one/two/three"));
+            Assert.assertEquals(paths.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), "/one/two/three");
         }
         finally
         {
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 cefc1e7..d7b79d3 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
@@ -134,7 +134,9 @@
 
             client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), 1, new RetryOneTime(1));
             client.start();
-            CreateBuilderImpl createBuilder = (CreateBuilderImpl)client.create().withProtection();
+            
+            CreateBuilderImpl createBuilder = (CreateBuilderImpl)client.create();
+            createBuilder.withProtection();
 
             client.create().forPath(createBuilder.adjustPath(TEST_PATH));
 
@@ -237,7 +239,6 @@
     @Test
     public void testGetAclNoStat() throws Exception
     {
-
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
         client.start();
         try
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
new file mode 100644
index 0000000..7565590
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java
@@ -0,0 +1,402 @@
+/**
+ * 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 com.google.common.collect.Lists;
+import org.apache.curator.ensemble.EnsembleProvider;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.CuratorEventType;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.InstanceSpec;
+import org.apache.curator.test.TestingCluster;
+import org.apache.curator.test.TestingZooKeeperServer;
+import org.apache.curator.test.Timing;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class TestReconfiguration extends BaseClassForTests
+{
+    private final Timing timing = new Timing();
+    private TestingCluster cluster;
+    private EnsembleProvider ensembleProvider;
+
+    @BeforeMethod
+    @Override
+    public void setup() throws Exception
+    {
+        super.setup();
+
+        CloseableUtils.closeQuietly(server);
+        server = null;
+        cluster = new TestingCluster(3);
+        cluster.start();
+    }
+
+    @AfterMethod
+    @Override
+    public void teardown() throws Exception
+    {
+        CloseableUtils.closeQuietly(cluster);
+        ensembleProvider = null;
+
+        super.teardown();
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    public void testApiPermutations() throws Exception
+    {
+        // not an actual test. Specifies all possible API possibilities
+
+        Watcher watcher = null;
+        Stat stat = null;
+        CuratorFramework client = null;
+
+        client.getConfig().forEnsemble();
+        client.getConfig().inBackground().forEnsemble();
+        client.getConfig().usingWatcher(watcher).forEnsemble();
+        client.getConfig().usingWatcher(watcher).inBackground().forEnsemble();
+        client.getConfig().storingStatIn(stat).forEnsemble();
+        client.getConfig().storingStatIn(stat).inBackground().forEnsemble();
+        client.getConfig().storingStatIn(stat).usingWatcher(watcher).forEnsemble();
+        client.getConfig().storingStatIn(stat).usingWatcher(watcher).inBackground().forEnsemble();
+
+        // ---------
+
+        client.reconfig().leaving().forEnsemble();
+        client.reconfig().joining().forEnsemble();
+        client.reconfig().leaving().joining().forEnsemble();
+        client.reconfig().joining().leaving().forEnsemble();
+        client.reconfig().withNewMembers().forEnsemble();
+
+        client.reconfig().leaving().fromConfig(0).forEnsemble();
+        client.reconfig().joining().fromConfig(0).forEnsemble();
+        client.reconfig().leaving().joining().fromConfig(0).forEnsemble();
+        client.reconfig().joining().leaving().fromConfig(0).forEnsemble();
+        client.reconfig().withNewMembers().fromConfig(0).forEnsemble();
+
+        client.reconfig().leaving().storingStatIn(stat).forEnsemble();
+        client.reconfig().joining().storingStatIn(stat).forEnsemble();
+        client.reconfig().leaving().joining().storingStatIn(stat).forEnsemble();
+        client.reconfig().joining().leaving().storingStatIn(stat).forEnsemble();
+        client.reconfig().withNewMembers().storingStatIn(stat).forEnsemble();
+
+        client.reconfig().leaving().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().joining().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().leaving().joining().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().joining().leaving().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().withNewMembers().storingStatIn(stat).fromConfig(0).forEnsemble();
+
+        client.reconfig().inBackground().leaving().forEnsemble();
+        client.reconfig().inBackground().joining().forEnsemble();
+        client.reconfig().inBackground().leaving().joining().forEnsemble();
+        client.reconfig().inBackground().joining().leaving().forEnsemble();
+        client.reconfig().inBackground().withNewMembers().forEnsemble();
+
+        client.reconfig().inBackground().leaving().fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().joining().fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().leaving().joining().fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().joining().leaving().fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().withNewMembers().fromConfig(0).forEnsemble();
+
+        client.reconfig().inBackground().leaving().storingStatIn(stat).forEnsemble();
+        client.reconfig().inBackground().joining().storingStatIn(stat).forEnsemble();
+        client.reconfig().inBackground().leaving().joining().storingStatIn(stat).forEnsemble();
+        client.reconfig().inBackground().joining().leaving().storingStatIn(stat).forEnsemble();
+        client.reconfig().inBackground().withNewMembers().storingStatIn(stat).forEnsemble();
+
+        client.reconfig().inBackground().leaving().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().joining().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().leaving().joining().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().joining().leaving().storingStatIn(stat).fromConfig(0).forEnsemble();
+        client.reconfig().inBackground().withNewMembers().storingStatIn(stat).fromConfig(0).forEnsemble();
+    }
+
+    @Test
+    public void testBasicGetConfig() throws Exception
+    {
+        try ( CuratorFramework client = newClient())
+        {
+            client.start();
+            byte[] configData = client.getConfig().forEnsemble();
+            QuorumVerifier quorumVerifier = toQuorumVerifier(configData);
+            System.out.println(quorumVerifier);
+            assertConfig(quorumVerifier, cluster.getInstances());
+            Assert.assertEquals(EnsembleTracker.configToConnectionString(quorumVerifier), ensembleProvider.getConnectionString());
+        }
+    }
+
+    @Test
+    public void testAdd() throws Exception
+    {
+        try ( CuratorFramework client = newClient())
+        {
+            client.start();
+
+            QuorumVerifier oldConfig = toQuorumVerifier(client.getConfig().forEnsemble());
+            assertConfig(oldConfig, cluster.getInstances());
+
+            CountDownLatch latch = setChangeWaiter(client);
+            try ( TestingCluster newCluster = new TestingCluster(TestingCluster.makeSpecs(1, false)) )
+            {
+                newCluster.start();
+
+                client.reconfig().joining(toReconfigSpec(newCluster.getInstances())).fromConfig(oldConfig.getVersion()).forEnsemble();
+
+                Assert.assertTrue(timing.awaitLatch(latch));
+
+                byte[] newConfigData = client.getConfig().forEnsemble();
+                QuorumVerifier newConfig = toQuorumVerifier(newConfigData);
+                List<InstanceSpec> newInstances = Lists.newArrayList(cluster.getInstances());
+                newInstances.addAll(newCluster.getInstances());
+                assertConfig(newConfig, newInstances);
+                Assert.assertEquals(EnsembleTracker.configToConnectionString(newConfig), ensembleProvider.getConnectionString());
+            }
+        }
+    }
+
+    @Test
+    public void testAddAsync() throws Exception
+    {
+        try ( CuratorFramework client = newClient())
+        {
+            client.start();
+
+            QuorumVerifier oldConfig = toQuorumVerifier(client.getConfig().forEnsemble());
+            assertConfig(oldConfig, cluster.getInstances());
+
+            CountDownLatch latch = setChangeWaiter(client);
+            try ( TestingCluster newCluster = new TestingCluster(TestingCluster.makeSpecs(1, false)) )
+            {
+                newCluster.start();
+
+                final CountDownLatch callbackLatch = new CountDownLatch(1);
+                BackgroundCallback callback = new BackgroundCallback()
+                {
+                    @Override
+                    public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                    {
+                        if ( event.getType() == CuratorEventType.RECONFIG )
+                        {
+                            callbackLatch.countDown();
+                        }
+                    }
+                };
+                client.reconfig().inBackground(callback).joining(toReconfigSpec(newCluster.getInstances())).fromConfig(oldConfig.getVersion()).forEnsemble();
+
+                Assert.assertTrue(timing.awaitLatch(callbackLatch));
+                Assert.assertTrue(timing.awaitLatch(latch));
+
+                byte[] newConfigData = client.getConfig().forEnsemble();
+                QuorumVerifier newConfig = toQuorumVerifier(newConfigData);
+                List<InstanceSpec> newInstances = Lists.newArrayList(cluster.getInstances());
+                newInstances.addAll(newCluster.getInstances());
+                assertConfig(newConfig, newInstances);
+                Assert.assertEquals(EnsembleTracker.configToConnectionString(newConfig), ensembleProvider.getConnectionString());
+            }
+        }
+    }
+
+    @Test
+    public void testAddAndRemove() throws Exception
+    {
+        try ( CuratorFramework client = newClient())
+        {
+            client.start();
+
+            QuorumVerifier oldConfig = toQuorumVerifier(client.getConfig().forEnsemble());
+            assertConfig(oldConfig, cluster.getInstances());
+
+            CountDownLatch latch = setChangeWaiter(client);
+
+            try ( TestingCluster newCluster = new TestingCluster(TestingCluster.makeSpecs(1, false)) )
+            {
+                newCluster.start();
+
+                Collection<InstanceSpec> oldInstances = cluster.getInstances();
+                InstanceSpec us = cluster.findConnectionInstance(client.getZookeeperClient().getZooKeeper());
+                InstanceSpec removeSpec = oldInstances.iterator().next();
+                if ( us.equals(removeSpec) ) {
+                    Iterator<InstanceSpec> iterator = oldInstances.iterator();
+                    iterator.next();
+                    removeSpec = iterator.next();
+                }
+
+                Collection<InstanceSpec> instances = newCluster.getInstances();
+                client.reconfig().leaving(Integer.toString(removeSpec.getServerId())).joining(toReconfigSpec(instances)).fromConfig(oldConfig.getVersion()).forEnsemble();
+
+                Assert.assertTrue(timing.awaitLatch(latch));
+
+                byte[] newConfigData = client.getConfig().forEnsemble();
+                QuorumVerifier newConfig = toQuorumVerifier(newConfigData);
+                ArrayList<InstanceSpec> newInstances = Lists.newArrayList(oldInstances);
+                newInstances.addAll(instances);
+                newInstances.remove(removeSpec);
+                assertConfig(newConfig, newInstances);
+                Assert.assertEquals(EnsembleTracker.configToConnectionString(newConfig), ensembleProvider.getConnectionString());
+            }
+        }
+    }
+
+    @Test
+    public void testNewMembers() throws Exception
+    {
+        cluster.close();
+        cluster = new TestingCluster(5);
+        List<TestingZooKeeperServer> servers = cluster.getServers();
+        List<InstanceSpec> smallCluster = Lists.newArrayList();
+        for ( int i = 0; i < 3; ++i )   // only start 3 of the 5
+        {
+            TestingZooKeeperServer server = servers.get(i);
+            server.start();
+            smallCluster.add(server.getInstanceSpec());
+        }
+
+        try ( CuratorFramework client = newClient())
+        {
+            client.start();
+
+            QuorumVerifier oldConfig = toQuorumVerifier(client.getConfig().forEnsemble());
+            Assert.assertEquals(oldConfig.getAllMembers().size(), 5);
+            assertConfig(oldConfig, cluster.getInstances());
+
+            CountDownLatch latch = setChangeWaiter(client);
+
+            client.reconfig().withNewMembers(toReconfigSpec(smallCluster)).forEnsemble();
+
+            Assert.assertTrue(timing.awaitLatch(latch));
+            byte[] newConfigData = client.getConfig().forEnsemble();
+            QuorumVerifier newConfig = toQuorumVerifier(newConfigData);
+            Assert.assertEquals(newConfig.getAllMembers().size(), 3);
+            assertConfig(newConfig, smallCluster);
+            Assert.assertEquals(EnsembleTracker.configToConnectionString(newConfig), ensembleProvider.getConnectionString());
+        }
+    }
+
+    private CuratorFramework newClient()
+    {
+        final AtomicReference<String> connectString = new AtomicReference<>(cluster.getConnectString());
+        ensembleProvider = new EnsembleProvider()
+        {
+            @Override
+            public void start() throws Exception
+            {
+            }
+
+            @Override
+            public boolean updateServerListEnabled()
+            {
+                return false;
+            }
+
+            @Override
+            public String getConnectionString()
+            {
+                return connectString.get();
+            }
+
+            @Override
+            public void close() throws IOException
+            {
+            }
+
+            @Override
+            public void setConnectionString(String connectionString)
+            {
+                connectString.set(connectionString);
+            }
+        };
+        return CuratorFrameworkFactory.builder()
+            .ensembleProvider(ensembleProvider)
+            .sessionTimeoutMs(timing.session())
+            .connectionTimeoutMs(timing.connection())
+            .retryPolicy(new ExponentialBackoffRetry(timing.forSleepingABit().milliseconds(), 3))
+            .build();
+    }
+
+    private CountDownLatch setChangeWaiter(CuratorFramework client) throws Exception
+    {
+        final CountDownLatch latch = new CountDownLatch(1);
+        Watcher watcher = new Watcher()
+        {
+            @Override
+            public void process(WatchedEvent event)
+            {
+                if ( event.getType() == Event.EventType.NodeDataChanged )
+                {
+                    latch.countDown();
+                }
+            }
+        };
+        client.getConfig().usingWatcher(watcher).forEnsemble();
+        return latch;
+    }
+
+    private void assertConfig(QuorumVerifier config, Collection<InstanceSpec> instances)
+    {
+        for ( InstanceSpec instance : instances )
+        {
+            QuorumPeer.QuorumServer quorumServer = config.getAllMembers().get((long)instance.getServerId());
+            Assert.assertNotNull(quorumServer, String.format("Looking for %s - found %s", instance.getServerId(), config.getAllMembers()));
+            Assert.assertEquals(quorumServer.clientAddr.getPort(), instance.getPort());
+        }
+    }
+
+    private List<String> toReconfigSpec(Collection<InstanceSpec> instances) throws Exception
+    {
+        String localhost = new InetSocketAddress((InetAddress)null, 0).getAddress().getHostAddress();
+        List<String> specs = Lists.newArrayList();
+        for ( InstanceSpec instance : instances ) {
+            specs.add("server." + instance.getServerId() + "=" + localhost + ":" + instance.getElectionPort() + ":" + instance.getQuorumPort() + ";" + instance.getPort());
+        }
+        return specs;
+    }
+
+    private static QuorumVerifier toQuorumVerifier(byte[] bytes) throws Exception
+    {
+        Assert.assertNotNull(bytes);
+        Properties properties = new Properties();
+        properties.load(new ByteArrayInputStream(bytes));
+        return new QuorumMaj(properties);
+    }
+}
\ No newline at end of file
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
new file mode 100644
index 0000000..a7c137a
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestRemoveWatches.java
@@ -0,0 +1,666 @@
+/**

+ * Licensed to the Apache Software Foundation (ASF) under one

+ * or more contributor license agreements.  See the NOTICE file

+ * distributed with this work for additional information

+ * regarding copyright ownership.  The ASF licenses this file

+ * to you under the Apache License, Version 2.0 (the

+ * "License"); you may not use this file except in compliance

+ * with the License.  You may obtain a copy of the License at

+ *

+ *   http://www.apache.org/licenses/LICENSE-2.0

+ *

+ * Unless required by applicable law or agreed to in writing,

+ * software distributed under the License is distributed on an

+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY

+ * KIND, either express or implied.  See the License for the

+ * specific language governing permissions and limitations

+ * under the License.

+ */

+package org.apache.curator.framework.imps;

+

+import org.apache.curator.framework.CuratorFramework;

+import org.apache.curator.framework.CuratorFrameworkFactory;

+import org.apache.curator.framework.api.BackgroundCallback;

+import org.apache.curator.framework.api.CuratorEvent;

+import org.apache.curator.framework.api.CuratorEventType;

+import org.apache.curator.framework.api.CuratorListener;

+import org.apache.curator.framework.api.CuratorWatcher;

+import org.apache.curator.framework.imps.FailedRemoveWatchManager.FailedRemoveWatchDetails;

+import org.apache.curator.framework.state.ConnectionState;

+import org.apache.curator.framework.state.ConnectionStateListener;

+import org.apache.curator.retry.ExponentialBackoffRetry;

+import org.apache.curator.retry.RetryOneTime;

+import org.apache.curator.test.BaseClassForTests;

+import org.apache.curator.test.Timing;

+import org.apache.curator.utils.CloseableUtils;

+import org.apache.zookeeper.KeeperException;

+import org.apache.zookeeper.WatchedEvent;

+import org.apache.zookeeper.Watcher;

+import org.apache.zookeeper.Watcher.Event.EventType;

+import org.apache.zookeeper.Watcher.WatcherType;

+import org.testng.Assert;

+import org.testng.annotations.Test;

+import java.util.concurrent.CountDownLatch;

+import java.util.concurrent.atomic.AtomicBoolean;

+import java.util.concurrent.atomic.AtomicReference;

+

+public class TestRemoveWatches extends BaseClassForTests

+{

+    private AtomicReference<ConnectionState> registerConnectionStateListener(CuratorFramework client)

+    {

+        final AtomicReference<ConnectionState> state = new AtomicReference<ConnectionState>();

+        client.getConnectionStateListenable().addListener(new ConnectionStateListener()

+        {

+            

+            @Override

+            public void stateChanged(CuratorFramework client, ConnectionState newState)

+            {

+                state.set(newState);

+                synchronized(state)

+                {

+                    state.notify();

+                }

+            }

+        });

+        

+        return state;

+    }

+    

+    private boolean blockUntilDesiredConnectionState(AtomicReference<ConnectionState> stateRef, Timing timing, final ConnectionState desiredState)

+    {

+        if(stateRef.get() == desiredState)

+        {

+            return true;

+        }

+

+        //noinspection SynchronizationOnLocalVariableOrMethodParameter

+        synchronized(stateRef)

+        {

+            if(stateRef.get() == desiredState)

+            {

+                return true;

+            }

+            

+            try

+            {

+                stateRef.wait(timing.milliseconds());

+                return stateRef.get() == desiredState;

+            }

+            catch(InterruptedException e)

+            {

+                Thread.currentThread().interrupt();

+                return false;

+            }

+        }

+    }

+    

+    @Test

+    public void testRemoveCuratorDefaultWatcher() throws Exception

+    {

+        Timing timing = new Timing();

+        CuratorFramework client = CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            final String path = "/";            

+            client.getCuratorListenable().addListener(new CuratorListener()

+            {                

+                @Override

+                public void eventReceived(CuratorFramework client, CuratorEvent event)

+                        throws Exception

+                {

+                    if(event.getType() == CuratorEventType.WATCHED && event.getWatchedEvent().getType() == EventType.DataWatchRemoved) {                        

+                        removedLatch.countDown();

+                    }        

+                }

+            });

+                        

+            client.checkExists().watched().forPath(path);

+            

+            client.watches().removeAll().forPath(path);

+            

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testRemoveCuratorWatch() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            final String path = "/";            

+            CuratorWatcher watcher = new CuratorWatcher()

+            {

+                

+                @Override

+                public void process(WatchedEvent event) throws Exception

+                {

+                    if(event.getPath().equals(path) && event.getType() == EventType.DataWatchRemoved) {

+                        removedLatch.countDown();

+                    }

+                }

+            };

+                        

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+            

+            client.watches().remove(watcher).forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }    

+    

+    @Test

+    public void testRemoveWatch() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            final String path = "/";    

+            Watcher watcher = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);

+            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+

+            client.watches().remove(watcher).forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testRemoveWatchInBackgroundWithCallback() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {            

+            client.start();

+         

+            //Make sure that the event fires on both the watcher and the callback.

+            final CountDownLatch removedLatch = new CountDownLatch(2);

+            final String path = "/";

+            Watcher watcher = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);

+            

+            BackgroundCallback callback = new BackgroundCallback()

+            {

+                

+                @Override

+                public void processResult(CuratorFramework client, CuratorEvent event)

+                        throws Exception

+                {

+                    if(event.getType() == CuratorEventType.REMOVE_WATCHES && event.getPath().equals(path)) {

+                        removedLatch.countDown();

+                    }

+                }

+            };

+            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+

+            client.watches().remove(watcher).ofType(WatcherType.Any).inBackground(callback).forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+            

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testRemoveWatchInBackgroundWithNoCallback() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final String path = "/";

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            Watcher watcher = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);

+            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+

+            client.watches().remove(watcher).inBackground().forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+            

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }        

+    

+    @Test

+    public void testRemoveAllWatches() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final String path = "/";

+            final CountDownLatch removedLatch = new CountDownLatch(2);

+            

+            Watcher watcher1 = new CountDownWatcher(path, removedLatch, EventType.ChildWatchRemoved);            

+            Watcher watcher2 = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);                        

+            

+            client.getChildren().usingWatcher(watcher1).forPath(path);

+            client.checkExists().usingWatcher(watcher2).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+

+            client.watches().removeAll().forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }  

+    

+    @Test

+    public void testRemoveAllDataWatches() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFramework client = CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final String path = "/";

+            final AtomicBoolean removedFlag = new AtomicBoolean(false);

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            Watcher watcher1 = new BooleanWatcher(path, removedFlag, EventType.ChildWatchRemoved);            

+            Watcher watcher2 = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);                        

+            

+            client.getChildren().usingWatcher(watcher1).forPath(path);

+            client.checkExists().usingWatcher(watcher2).forPath(path);

+            

+            client.watches().removeAll().ofType(WatcherType.Data).forPath(path);

+            

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+            Assert.assertEquals(removedFlag.get(), false);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testRemoveAllChildWatches() throws Exception

+    {       

+        Timing timing = new Timing();

+        CuratorFramework client = CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final String path = "/";

+            final AtomicBoolean removedFlag = new AtomicBoolean(false);

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            Watcher watcher1 = new BooleanWatcher(path, removedFlag, EventType.DataWatchRemoved);            

+            Watcher watcher2 = new CountDownWatcher(path, removedLatch, EventType.ChildWatchRemoved);                        

+                        

+            client.checkExists().usingWatcher(watcher1).forPath(path);

+            client.getChildren().usingWatcher(watcher2).forPath(path);

+            

+            client.watches().removeAll().ofType(WatcherType.Children).forPath(path);

+            

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+            Assert.assertEquals(removedFlag.get(), false);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }     

+    

+    @Test

+    public void testRemoveLocalWatch() throws Exception {

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            AtomicReference<ConnectionState> stateRef = registerConnectionStateListener(client);

+            

+            final String path = "/";

+            

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            Watcher watcher = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);        

+            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+

+            //Stop the server so we can check if we can remove watches locally when offline

+            server.stop();

+            

+            Assert.assertTrue(blockUntilDesiredConnectionState(stateRef, timing, ConnectionState.SUSPENDED));

+                       

+            client.watches().removeAll().locally().forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testRemoveLocalWatchInBackground() throws Exception {

+        Timing timing = new Timing();

+        CuratorFrameworkImpl client = (CuratorFrameworkImpl)CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            AtomicReference<ConnectionState> stateRef = registerConnectionStateListener(client);

+            

+            final String path = "/";

+            

+            final CountDownLatch removedLatch = new CountDownLatch(1);

+            

+            Watcher watcher = new CountDownWatcher(path, removedLatch, EventType.DataWatchRemoved);        

+            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            Assert.assertTrue(!client.getNamespaceWatcherMap().isEmpty());

+

+            //Stop the server so we can check if we can remove watches locally when offline

+            server.stop();

+            

+            Assert.assertTrue(blockUntilDesiredConnectionState(stateRef, timing, ConnectionState.SUSPENDED));

+                       

+            client.watches().removeAll().locally().inBackground().forPath(path);

+            Assert.assertTrue(client.getNamespaceWatcherMap().isEmpty());

+

+            Assert.assertTrue(timing.awaitLatch(removedLatch), "Timed out waiting for watch removal");

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }    

+    

+    /**

+     * Test the case where we try and remove an unregistered watcher. In this case we expect a NoWatcherException to

+     * be thrown. 

+     * @throws Exception

+     */

+    @Test(expectedExceptions=KeeperException.NoWatcherException.class)

+    public void testRemoveUnregisteredWatcher() throws Exception

+    {

+        CuratorFramework client = CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final String path = "/";            

+            Watcher watcher = new Watcher() {

+                @Override

+                public void process(WatchedEvent event)

+                {

+                }                

+            };

+            

+            client.watches().remove(watcher).forPath(path);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    /**

+     * Test the case where we try and remove an unregistered watcher but have the quietly flag set. In this case we expect success. 

+     * @throws Exception

+     */

+    @Test

+    public void testRemoveUnregisteredWatcherQuietly() throws Exception

+    {

+        Timing timing = new Timing();

+        CuratorFramework client = CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            final AtomicBoolean watcherRemoved = new AtomicBoolean(false);

+            

+            final String path = "/";            

+            Watcher watcher = new BooleanWatcher(path, watcherRemoved, EventType.DataWatchRemoved);

+            

+            client.watches().remove(watcher).quietly().forPath(path);

+            

+            timing.sleepABit();

+            

+            //There should be no watcher removed as none were registered.

+            Assert.assertEquals(watcherRemoved.get(), false);

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testGuaranteedRemoveWatch() throws Exception {

+        Timing timing = new Timing();

+        CuratorFramework client = CuratorFrameworkFactory.builder().

+                connectString(server.getConnectString()).

+                retryPolicy(new RetryOneTime(1)).

+                build();

+        try

+        {

+            client.start();

+            

+            AtomicReference<ConnectionState> stateRef = registerConnectionStateListener(client);

+                       

+            String path = "/";

+            

+            CountDownLatch removeLatch = new CountDownLatch(1);

+            

+            Watcher watcher = new CountDownWatcher(path, removeLatch, EventType.DataWatchRemoved);            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            

+            server.stop();           

+            

+            Assert.assertTrue(blockUntilDesiredConnectionState(stateRef, timing, ConnectionState.SUSPENDED));

+            

+            //Remove the watch while we're not connected

+            try 

+            {

+                client.watches().remove(watcher).guaranteed().forPath(path);

+                Assert.fail();

+            }

+            catch(KeeperException.ConnectionLossException e)

+            {

+                //Expected

+            }

+            

+            server.restart();

+            

+            timing.awaitLatch(removeLatch);            

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }

+    

+    @Test

+    public void testGuaranteedRemoveWatchInBackground() throws Exception {

+        Timing timing = new Timing();

+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(),

+                                                                    new ExponentialBackoffRetry(100, 3));

+        try

+        {

+            client.start();

+            

+            AtomicReference<ConnectionState> stateRef = registerConnectionStateListener(client);

+                        

+            final CountDownLatch guaranteeAddedLatch = new CountDownLatch(1);

+            

+            ((CuratorFrameworkImpl)client).getFailedRemoveWatcherManager().debugListener = new FailedOperationManager.FailedOperationManagerListener<FailedRemoveWatchManager.FailedRemoveWatchDetails>()

+            {

+

+                @Override

+                public void pathAddedForGuaranteedOperation(

+                        FailedRemoveWatchDetails detail)

+                {

+                    guaranteeAddedLatch.countDown();

+                }

+            };

+            

+            String path = "/";

+            

+            CountDownLatch removeLatch = new CountDownLatch(1);

+            

+            Watcher watcher = new CountDownWatcher(path, removeLatch, EventType.DataWatchRemoved);            

+            client.checkExists().usingWatcher(watcher).forPath(path);

+            

+            server.stop();           

+            Assert.assertTrue(blockUntilDesiredConnectionState(stateRef, timing, ConnectionState.SUSPENDED));

+            

+            //Remove the watch while we're not connected

+            client.watches().remove(watcher).guaranteed().inBackground().forPath(path);

+            

+            timing.awaitLatch(guaranteeAddedLatch);

+            

+            server.restart();

+            

+            timing.awaitLatch(removeLatch);            

+        }

+        finally

+        {

+            CloseableUtils.closeQuietly(client);

+        }

+    }    

+    

+    private static class CountDownWatcher implements Watcher {

+        private String path;

+        private EventType eventType;

+        private CountDownLatch removeLatch;

+        

+        public CountDownWatcher(String path, CountDownLatch removeLatch, EventType eventType) {

+            this.path = path;

+            this.eventType = eventType;

+            this.removeLatch = removeLatch;            

+        }

+        

+        @Override

+        public void process(WatchedEvent event)

+        {

+            if(event.getPath() == null || event.getType() == null) {

+                return;

+            }

+            

+            if(event.getPath().equals(path) && event.getType() == eventType) {

+                removeLatch.countDown();

+            }

+        }  

+    }

+    

+    private static class BooleanWatcher implements Watcher {

+        private String path;

+        private EventType eventType;

+        private AtomicBoolean removedFlag;

+        

+        public BooleanWatcher(String path, AtomicBoolean removedFlag, EventType eventType) {

+            this.path = path;

+            this.eventType = eventType;

+            this.removedFlag = removedFlag;            

+        }

+        

+        @Override

+        public void process(WatchedEvent event)

+        {

+            if(event.getPath() == null || event.getType() == null) {

+                return;

+            }

+            

+            if(event.getPath().equals(path) && event.getType() == eventType) {

+                removedFlag.set(true);

+            }

+        }  

+    }    

+}

diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsNew.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsNew.java
new file mode 100644
index 0000000..eaf94f8
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsNew.java
@@ -0,0 +1,218 @@
+/**
+ * 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 com.google.common.collect.Iterables;
+import com.google.common.collect.Queues;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.api.transaction.CuratorOp;
+import org.apache.curator.framework.api.transaction.CuratorTransactionResult;
+import org.apache.curator.framework.api.transaction.OperationType;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+public class TestTransactionsNew extends BaseClassForTests
+{
+    @Test
+    public void testCheckVersion() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            client.create().forPath("/foo");
+            Stat stat = client.setData().forPath("/foo", "new".getBytes());  // up the version
+
+            CuratorOp statOp = client.transactionOp().check().withVersion(stat.getVersion() + 1).forPath("/foo");
+            CuratorOp createOp = client.transactionOp().create().forPath("/bar");
+            try
+            {
+                client.transaction().forOperations(statOp, createOp);
+                Assert.fail();
+            }
+            catch ( KeeperException.BadVersionException correct )
+            {
+                // correct
+            }
+
+            Assert.assertNull(client.checkExists().forPath("/bar"));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testWithNamespace() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.builder().connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace("galt").build();
+        try
+        {
+            client.start();
+            CuratorOp createOp1 = client.transactionOp().create().forPath("/foo", "one".getBytes());
+            CuratorOp createOp2 = client.transactionOp().create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test-", "one".getBytes());
+            CuratorOp setDataOp = client.transactionOp().setData().forPath("/foo", "two".getBytes());
+            CuratorOp createOp3 = client.transactionOp().create().forPath("/foo/bar");
+            CuratorOp deleteOp = client.transactionOp().delete().forPath("/foo/bar");
+
+            Collection<CuratorTransactionResult> results = client.transaction().forOperations(createOp1, createOp2, setDataOp, createOp3, deleteOp);
+
+            Assert.assertTrue(client.checkExists().forPath("/foo") != null);
+            Assert.assertTrue(client.usingNamespace(null).checkExists().forPath("/galt/foo") != null);
+            Assert.assertEquals(client.getData().forPath("/foo"), "two".getBytes());
+            Assert.assertTrue(client.checkExists().forPath("/foo/bar") == null);
+
+            CuratorTransactionResult ephemeralResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/test-"));
+            Assert.assertNotNull(ephemeralResult);
+            Assert.assertNotEquals(ephemeralResult.getResultPath(), "/test-");
+            Assert.assertTrue(ephemeralResult.getResultPath().startsWith("/test-"));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testBasic() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            CuratorOp createOp1 = client.transactionOp().create().forPath("/foo");
+            CuratorOp createOp2 = client.transactionOp().create().forPath("/foo/bar", "snafu".getBytes());
+
+            Collection<CuratorTransactionResult> results = client.transaction().forOperations(createOp1, createOp2);
+
+            Assert.assertTrue(client.checkExists().forPath("/foo/bar") != null);
+            Assert.assertEquals(client.getData().forPath("/foo/bar"), "snafu".getBytes());
+
+            CuratorTransactionResult fooResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/foo"));
+            CuratorTransactionResult fooBarResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/foo/bar"));
+            Assert.assertNotNull(fooResult);
+            Assert.assertNotNull(fooBarResult);
+            Assert.assertNotSame(fooResult, fooBarResult);
+            Assert.assertEquals(fooResult.getResultPath(), "/foo");
+            Assert.assertEquals(fooBarResult.getResultPath(), "/foo/bar");
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testBackground() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            CuratorOp createOp1 = client.transactionOp().create().forPath("/foo");
+            CuratorOp createOp2 = client.transactionOp().create().forPath("/foo/bar", "snafu".getBytes());
+
+            final BlockingQueue<List<CuratorTransactionResult>> queue = Queues.newLinkedBlockingQueue();
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    queue.add(event.getOpResults());
+                }
+            };
+            client.transaction().inBackground(callback).forOperations(createOp1, createOp2);
+            Collection<CuratorTransactionResult> results = queue.poll(5, TimeUnit.SECONDS);
+
+            Assert.assertNotNull(results);
+            Assert.assertTrue(client.checkExists().forPath("/foo/bar") != null);
+            Assert.assertEquals(client.getData().forPath("/foo/bar"), "snafu".getBytes());
+
+            CuratorTransactionResult fooResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/foo"));
+            CuratorTransactionResult fooBarResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/foo/bar"));
+            Assert.assertNotNull(fooResult);
+            Assert.assertNotNull(fooBarResult);
+            Assert.assertNotSame(fooResult, fooBarResult);
+            Assert.assertEquals(fooResult.getResultPath(), "/foo");
+            Assert.assertEquals(fooBarResult.getResultPath(), "/foo/bar");
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testBackgroundWithNamespace() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.builder().connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace("galt").build();
+        try
+        {
+            client.start();
+            CuratorOp createOp1 = client.transactionOp().create().forPath("/foo", "one".getBytes());
+            CuratorOp createOp2 = client.transactionOp().create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath("/test-", "one".getBytes());
+            CuratorOp setDataOp = client.transactionOp().setData().forPath("/foo", "two".getBytes());
+            CuratorOp createOp3 = client.transactionOp().create().forPath("/foo/bar");
+            CuratorOp deleteOp = client.transactionOp().delete().forPath("/foo/bar");
+
+            final BlockingQueue<List<CuratorTransactionResult>> queue = Queues.newLinkedBlockingQueue();
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    queue.add(event.getOpResults());
+                }
+            };
+            client.transaction().inBackground(callback).forOperations(createOp1, createOp2, setDataOp, createOp3, deleteOp);
+
+            Collection<CuratorTransactionResult> results = queue.poll(5, TimeUnit.SECONDS);
+
+            Assert.assertNotNull(results);
+            Assert.assertTrue(client.checkExists().forPath("/foo") != null);
+            Assert.assertTrue(client.usingNamespace(null).checkExists().forPath("/galt/foo") != null);
+            Assert.assertEquals(client.getData().forPath("/foo"), "two".getBytes());
+            Assert.assertTrue(client.checkExists().forPath("/foo/bar") == null);
+
+            CuratorTransactionResult ephemeralResult = Iterables.find(results, CuratorTransactionResult.ofTypeAndPath(OperationType.CREATE, "/test-"));
+            Assert.assertNotNull(ephemeralResult);
+            Assert.assertNotEquals(ephemeralResult.getResultPath(), "/test-");
+            Assert.assertTrue(ephemeralResult.getResultPath().startsWith("/test-"));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+}
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
similarity index 95%
rename from curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java
rename to curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
index fedc779..2c42d61 100644
--- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactions.java
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestTransactionsOld.java
@@ -25,6 +25,7 @@
 import org.apache.curator.framework.api.transaction.OperationType;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.utils.CloseableUtils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -33,15 +34,16 @@
 import org.testng.annotations.Test;
 import java.util.Collection;
 
-public class TestTransactions extends BaseClassForTests
+@SuppressWarnings("deprecation")
+public class TestTransactionsOld extends BaseClassForTests
 {
     @Test
     public void     testCheckVersion() throws Exception
     {
         CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
-        client.start();
         try
         {
+            client.start();
             client.create().forPath("/foo");
             Stat        stat = client.setData().forPath("/foo", "new".getBytes());  // up the version
 
@@ -65,7 +67,7 @@
         }
         finally
         {
-            client.close();
+            CloseableUtils.closeQuietly(client);
         }
     }
 
@@ -73,9 +75,9 @@
     public void     testWithNamespace() throws Exception
     {
         CuratorFramework        client = CuratorFrameworkFactory.builder().connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace("galt").build();
-        client.start();
         try
         {
+            client.start();
             Collection<CuratorTransactionResult>    results =
                 client.inTransaction()
                     .create().forPath("/foo", "one".getBytes())
@@ -102,7 +104,7 @@
         }
         finally
         {
-            client.close();
+            CloseableUtils.closeQuietly(client);
         }
     }
 
@@ -153,9 +155,9 @@
     public void     testBasic() throws Exception
     {
         CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
-        client.start();
         try
         {
+            client.start();
             Collection<CuratorTransactionResult>    results =
                 client.inTransaction()
                     .create().forPath("/foo")
@@ -177,7 +179,7 @@
         }
         finally
         {
-            client.close();
+            CloseableUtils.closeQuietly(client);
         }
     }
 }
diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
new file mode 100644
index 0000000..e20c450
--- /dev/null
+++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestWatcherRemovalManager.java
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.Timing;
+import org.apache.curator.test.WatchersDebug;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+public class TestWatcherRemovalManager extends BaseClassForTests
+{
+    @Test
+    public void testBasic() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            internalTryBasic(client);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testBasicNamespace1() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            internalTryBasic(client.usingNamespace("foo"));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testBasicNamespace2() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.builder()
+            .connectString(server.getConnectString())
+            .retryPolicy(new RetryOneTime(1))
+            .namespace("hey")
+            .build();
+        try
+        {
+            client.start();
+            internalTryBasic(client);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testBasicNamespace3() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.builder()
+            .connectString(server.getConnectString())
+            .retryPolicy(new RetryOneTime(1))
+            .namespace("hey")
+            .build();
+        try
+        {
+            client.start();
+            internalTryBasic(client.usingNamespace("lakjsf"));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testSameWatcher() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            WatcherRemovalFacade removerClient = (WatcherRemovalFacade)client.newWatcherRemoveCuratorFramework();
+
+            Watcher watcher = new Watcher()
+            {
+                @Override
+                public void process(WatchedEvent event)
+                {
+                    // NOP
+                }
+            };
+
+            removerClient.getData().usingWatcher(watcher).forPath("/");
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 1);
+            removerClient.getData().usingWatcher(watcher).forPath("/");
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 1);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testTriggered() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            WatcherRemovalFacade removerClient = (WatcherRemovalFacade)client.newWatcherRemoveCuratorFramework();
+
+            final CountDownLatch latch = new CountDownLatch(1);
+            Watcher watcher = new Watcher()
+            {
+                @Override
+                public void process(WatchedEvent event)
+                {
+                    if ( event.getType() == Event.EventType.NodeCreated )
+                    {
+                        latch.countDown();
+                    }
+                }
+            };
+
+            removerClient.checkExists().usingWatcher(watcher).forPath("/yo");
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 1);
+            removerClient.create().forPath("/yo");
+
+            Assert.assertTrue(new Timing().awaitLatch(latch));
+
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 0);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
+    public void testResetFromWatcher() throws Exception
+    {
+        Timing timing = new Timing();
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            final WatcherRemovalFacade removerClient = (WatcherRemovalFacade)client.newWatcherRemoveCuratorFramework();
+
+            final CountDownLatch createdLatch = new CountDownLatch(1);
+            final CountDownLatch deletedLatch = new CountDownLatch(1);
+            Watcher watcher = new Watcher()
+            {
+                @Override
+                public void process(WatchedEvent event)
+                {
+                    if ( event.getType() == Event.EventType.NodeCreated )
+                    {
+                        try
+                        {
+                            removerClient.checkExists().usingWatcher(this).forPath("/yo");
+                        }
+                        catch ( Exception e )
+                        {
+                            e.printStackTrace();
+                        }
+                        createdLatch.countDown();
+                    }
+                    else if ( event.getType() == Event.EventType.NodeDeleted )
+                    {
+                        deletedLatch.countDown();
+                    }
+                }
+            };
+
+            removerClient.checkExists().usingWatcher(watcher).forPath("/yo");
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 1);
+            removerClient.create().forPath("/yo");
+
+            Assert.assertTrue(timing.awaitLatch(createdLatch));
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 1);
+
+            removerClient.delete().forPath("/yo");
+
+            Assert.assertTrue(timing.awaitLatch(deletedLatch));
+
+            Assert.assertEquals(removerClient.getRemovalManager().getEntries().size(), 0);
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    private void internalTryBasic(CuratorFramework client) throws Exception
+    {
+        WatcherRemoveCuratorFramework removerClient = client.newWatcherRemoveCuratorFramework();
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        Watcher watcher = new Watcher()
+        {
+            @Override
+            public void process(WatchedEvent event)
+            {
+                if ( event.getType() == Event.EventType.DataWatchRemoved )
+                {
+                    latch.countDown();
+                }
+            }
+        };
+        removerClient.checkExists().usingWatcher(watcher).forPath("/hey");
+
+        List<String> existWatches = WatchersDebug.getExistWatches(client.getZookeeperClient().getZooKeeper());
+        Assert.assertEquals(existWatches.size(), 1);
+
+        removerClient.removeWatchers();
+
+        Assert.assertTrue(new Timing().awaitLatch(latch));
+
+        existWatches = WatchersDebug.getExistWatches(client.getZookeeperClient().getZooKeeper());
+        Assert.assertEquals(existWatches.size(), 0);
+    }
+}
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index bbcb749..50a32bf 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-recipes</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
     <packaging>bundle</packaging>
 
     <name>Curator Recipes</name>
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
index eb81893..8517c79 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
@@ -24,6 +24,7 @@
 import com.google.common.base.Preconditions;
 
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.listen.ListenerContainer;
@@ -56,7 +57,7 @@
 public class NodeCache implements Closeable
 {
     private final Logger log = LoggerFactory.getLogger(getClass());
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final String path;
     private final boolean dataIsCompressed;
     private final AtomicReference<ChildData> data = new AtomicReference<ChildData>(null);
@@ -139,7 +140,7 @@
      */
     public NodeCache(CuratorFramework client, String path, boolean dataIsCompressed)
     {
-        this.client = client;
+        this.client = client.newWatcherRemoveCuratorFramework();
         this.path = PathUtils.validatePath(path);
         this.dataIsCompressed = dataIsCompressed;
     }
@@ -181,8 +182,8 @@
     {
         if ( state.compareAndSet(State.STARTED, State.CLOSED) )
         {
+            client.removeWatchers();
             listeners.clear();
-            client.clearWatcherReferences(watcher);
             client.getConnectionStateListenable().removeListener(connectionStateListener);
 
             // TODO
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
index ae30da9..1ff3f38 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
@@ -27,6 +27,7 @@
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.EnsureContainers;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
@@ -43,16 +44,13 @@
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Exchanger;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
@@ -68,7 +66,7 @@
 public class PathChildrenCache implements Closeable
 {
     private final Logger log = LoggerFactory.getLogger(getClass());
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final String path;
     private final CloseableExecutorService executorService;
     private final boolean cacheData;
@@ -218,7 +216,7 @@
      */
     public PathChildrenCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, final CloseableExecutorService executorService)
     {
-        this.client = client;
+        this.client = client.newWatcherRemoveCuratorFramework();
         this.path = PathUtils.validatePath(path);
         this.cacheData = cacheData;
         this.dataIsCompressed = dataIsCompressed;
@@ -321,7 +319,7 @@
      */
     public void rebuild() throws Exception
     {
-        Preconditions.checkState(!executorService.isShutdown(), "cache has been closed");
+        Preconditions.checkState(state.get() == State.STARTED, "cache has been closed");
 
         ensurePath();
 
@@ -353,7 +351,7 @@
     public void rebuildNode(String fullPath) throws Exception
     {
         Preconditions.checkArgument(ZKPaths.getPathAndNode(fullPath).getPath().equals(path), "Node is not part of this cache: " + fullPath);
-        Preconditions.checkState(!executorService.isShutdown(), "cache has been closed");
+        Preconditions.checkState(state.get() == State.STARTED, "cache has been closed");
 
         ensurePath();
         internalRebuildNode(fullPath);
@@ -376,8 +374,7 @@
             client.getConnectionStateListenable().removeListener(connectionStateListener);
             listeners.clear();
             executorService.close();
-            client.clearWatcherReferences(childrenWatcher);
-            client.clearWatcherReferences(dataWatcher);
+            client.removeWatchers();
 
             // TODO
             // This seems to enable even more GC - I'm not sure why yet - it
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
index 3f7d8d4..0ee1832 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
@@ -25,6 +25,7 @@
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.UnhandledErrorListener;
@@ -260,18 +261,24 @@
 
         private void doRefreshChildren() throws Exception
         {
-            client.getChildren().usingWatcher(this).inBackground(this).forPath(path);
+            if ( treeState.get() == TreeState.STARTED )
+            {
+                client.getChildren().usingWatcher(this).inBackground(this).forPath(path);
+            }
         }
 
         private void doRefreshData() throws Exception
         {
-            if ( dataIsCompressed )
+            if ( treeState.get() == TreeState.STARTED )
             {
-                client.getData().decompressed().usingWatcher(this).inBackground(this).forPath(path);
-            }
-            else
-            {
-                client.getData().usingWatcher(this).inBackground(this).forPath(path);
+                if ( dataIsCompressed )
+                {
+                    client.getData().decompressed().usingWatcher(this).inBackground(this).forPath(path);
+                }
+                else
+                {
+                    client.getData().usingWatcher(this).inBackground(this).forPath(path);
+                }
             }
         }
 
@@ -296,7 +303,7 @@
         void wasDeleted() throws Exception
         {
             ChildData oldChildData = childData.getAndSet(null);
-            client.clearWatcherReferences(this);
+            client.watches().remove(this).ofType(WatcherType.Any).locally().inBackground().forPath(path);
             ConcurrentMap<String, TreeNode> childMap = children.getAndSet(null);
             if ( childMap != null )
             {
@@ -498,7 +505,7 @@
     private final AtomicBoolean isInitialized = new AtomicBoolean(false);
 
     private final TreeNode root;
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final CloseableExecutorService executorService;
     private final boolean cacheData;
     private final boolean dataIsCompressed;
@@ -550,7 +557,8 @@
         this.createParentNodes = createParentNodes;
         this.selector = Preconditions.checkNotNull(selector, "selector cannot be null");
         this.root = new TreeNode(validatePath(path), null);
-        this.client = Preconditions.checkNotNull(client, "client cannot be null");
+        Preconditions.checkNotNull(client, "client cannot be null");
+        this.client = client.newWatcherRemoveCuratorFramework();
         this.cacheData = cacheData;
         this.dataIsCompressed = dataIsCompressed;
         this.maxDepth = maxDepth;
@@ -586,6 +594,7 @@
     {
         if ( treeState.compareAndSet(TreeState.STARTED, TreeState.CLOSED) )
         {
+            client.removeWatchers();
             client.getConnectionStateListenable().removeListener(connectionStateListener);
             listeners.clear();
             executorService.close();
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
index 7f507bb..bb8aa73 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
@@ -23,6 +23,7 @@
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.listen.ListenerContainer;
@@ -64,7 +65,7 @@
 public class LeaderLatch implements Closeable
 {
     private final Logger log = LoggerFactory.getLogger(getClass());
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final String latchPath;
     private final String id;
     private final AtomicReference<State> state = new AtomicReference<State>(State.LATENT);
@@ -144,7 +145,7 @@
      */
     public LeaderLatch(CuratorFramework client, String latchPath, String id, CloseMode closeMode)
     {
-        this.client = Preconditions.checkNotNull(client, "client cannot be null");
+        this.client = Preconditions.checkNotNull(client, "client cannot be null").newWatcherRemoveCuratorFramework();
         this.latchPath = PathUtils.validatePath(latchPath);
         this.id = Preconditions.checkNotNull(id, "id cannot be null");
         this.closeMode = Preconditions.checkNotNull(closeMode, "closeMode cannot be null");
@@ -160,20 +161,20 @@
         Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Cannot be started more than once");
 
         startTask.set(AfterConnectionEstablished.execute(client, new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                try
                 {
-                    @Override
-                    public void run()
-                    {
-                        try
-                        {
-                            internalStart();
-                        }
-                        finally
-                        {
-                            startTask.set(null);
-                        }
-                    }
-                }));
+                    internalStart();
+                }
+                finally
+                {
+                    startTask.set(null);
+                }
+            }
+        }));
     }
 
     /**
@@ -207,6 +208,7 @@
         try
         {
             setNode(null);
+            client.removeWatchers();
         }
         catch ( Exception e )
         {
@@ -606,7 +608,10 @@
             {
                 try
                 {
-                    reset();
+                    if ( client.getConnectionStateErrorPolicy().isErrorState(ConnectionState.SUSPENDED) || !hasLeadership.get() )
+                    {
+                        reset();
+                    }
                 }
                 catch ( Exception e )
                 {
@@ -618,6 +623,14 @@
             }
 
             case SUSPENDED:
+            {
+                if ( client.getConnectionStateErrorPolicy().isErrorState(ConnectionState.SUSPENDED) )
+                {
+                    setLeadership(false);
+                }
+                break;
+            }
+
             case LOST:
             {
                 setLeadership(false);
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelectorListenerAdapter.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelectorListenerAdapter.java
index 7402fa7..19c0e92 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelectorListenerAdapter.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelectorListenerAdapter.java
@@ -30,7 +30,7 @@
     @Override
     public void stateChanged(CuratorFramework client, ConnectionState newState)
     {
-        if ( (newState == ConnectionState.SUSPENDED) || (newState == ConnectionState.LOST) )
+        if ( client.getConnectionStateErrorPolicy().isErrorState(newState) )
         {
             throw new CancelLeadershipException();
         }
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreMutex.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreMutex.java
index 88b5f5d..444b10d 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreMutex.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreMutex.java
@@ -20,6 +20,7 @@
 
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -29,6 +30,7 @@
 public class InterProcessSemaphoreMutex implements InterProcessLock
 {
     private final InterProcessSemaphoreV2 semaphore;
+    private final WatcherRemoveCuratorFramework watcherRemoveClient;
     private volatile Lease lease;
 
     /**
@@ -37,7 +39,8 @@
      */
     public InterProcessSemaphoreMutex(CuratorFramework client, String path)
     {
-        this.semaphore = new InterProcessSemaphoreV2(client, path, 1);
+        watcherRemoveClient = client.newWatcherRemoveCuratorFramework();
+        this.semaphore = new InterProcessSemaphoreV2(watcherRemoveClient, path, 1);
     }
 
     @Override
@@ -66,6 +69,7 @@
         try
         {
             lease.close();
+            watcherRemoveClient.removeWatchers();
         }
         finally
         {
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java
index 8524075..36dbff4 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessSemaphoreV2.java
@@ -22,15 +22,16 @@
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
-
-import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.RetryLoop;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.PathAndBytesable;
 import org.apache.curator.framework.imps.CuratorFrameworkState;
 import org.apache.curator.framework.recipes.shared.SharedCountListener;
 import org.apache.curator.framework.recipes.shared.SharedCountReader;
 import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.curator.utils.PathUtils;
 import org.apache.curator.utils.ThreadUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.CreateMode;
@@ -39,13 +40,13 @@
 import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import org.apache.curator.utils.PathUtils;
 
 /**
  * <p>
@@ -79,7 +80,7 @@
 {
     private final Logger log = LoggerFactory.getLogger(getClass());
     private final InterProcessMutex lock;
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final String leasesPath;
     private final Watcher watcher = new Watcher()
     {
@@ -123,7 +124,7 @@
 
     private InterProcessSemaphoreV2(CuratorFramework client, String path, int maxLeases, SharedCountReader count)
     {
-        this.client = client;
+        this.client = client.newWatcherRemoveCuratorFramework();
         path = PathUtils.validatePath(path);
         lock = new InterProcessMutex(client, ZKPaths.makePath(path, LOCK_PARENT));
         this.maxLeases = (count != null) ? count.getCount() : maxLeases;
@@ -353,36 +354,43 @@
             String nodeName = ZKPaths.getNodeFromPath(path);
             builder.add(makeLease(path));
 
-            synchronized(this)
+            try
             {
-                for(;;)
+                synchronized(this)
                 {
-                    List<String> children = client.getChildren().usingWatcher(watcher).forPath(leasesPath);
-                    if ( !children.contains(nodeName) )
+                    for(;;)
                     {
-                        log.error("Sequential path not found: " + path);
-                        return InternalAcquireResult.RETRY_DUE_TO_MISSING_NODE;
-                    }
-
-                    if ( children.size() <= maxLeases )
-                    {
-                        break;
-                    }
-                    if ( hasWait )
-                    {
-                        long thisWaitMs = getThisWaitMs(startMs, waitMs);
-                        if ( thisWaitMs <= 0 )
+                        List<String> children = client.getChildren().usingWatcher(watcher).forPath(leasesPath);
+                        if ( !children.contains(nodeName) )
                         {
-                            return InternalAcquireResult.RETURN_NULL;
+                            log.error("Sequential path not found: " + path);
+                            return InternalAcquireResult.RETRY_DUE_TO_MISSING_NODE;
                         }
-                        wait(thisWaitMs);
-                    }
-                    else
-                    {
-                        wait();
+
+                        if ( children.size() <= maxLeases )
+                        {
+                            break;
+                        }
+                        if ( hasWait )
+                        {
+                            long thisWaitMs = getThisWaitMs(startMs, waitMs);
+                            if ( thisWaitMs <= 0 )
+                            {
+                                return InternalAcquireResult.RETURN_NULL;
+                            }
+                            wait(thisWaitMs);
+                        }
+                        else
+                        {
+                            wait();
+                        }
                     }
                 }
             }
+            finally
+            {
+                client.removeWatchers();
+            }
         }
         finally
         {
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java
index f712945..dc2f681 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java
@@ -24,12 +24,12 @@
 import com.google.common.collect.Lists;
 import org.apache.curator.RetryLoop;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.CuratorWatcher;
 import org.apache.curator.framework.imps.CuratorFrameworkState;
 import org.apache.curator.utils.PathUtils;
 import org.apache.curator.utils.ThreadUtils;
 import org.apache.curator.utils.ZKPaths;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -43,7 +43,7 @@
 
 public class LockInternals
 {
-    private final CuratorFramework                  client;
+    private final WatcherRemoveCuratorFramework     client;
     private final String                            path;
     private final String                            basePath;
     private final LockInternalsDriver               driver;
@@ -101,7 +101,7 @@
         this.lockName = lockName;
         this.maxLeases = maxLeases;
 
-        this.client = client;
+        this.client = client.newWatcherRemoveCuratorFramework();
         this.basePath = PathUtils.validatePath(path);
         this.path = ZKPaths.makePath(path, lockName);
     }
@@ -117,8 +117,9 @@
         revocable.set(entry);
     }
 
-    void releaseLock(String lockPath) throws Exception
+    final void releaseLock(String lockPath) throws Exception
     {
+        client.removeWatchers();
         revocable.set(null);
         deleteOurPath(lockPath);
     }
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java
new file mode 100644
index 0000000..2644a8d
--- /dev/null
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.curator.framework.recipes.locks;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * <p>
+ *     Utility for safely acquiring a lock and releasing it using Java 7's
+ *     try-with-resource feature.
+ * </p>
+ *
+ * <p>
+ *     Canonical usage:
+ * <code><pre>
+ *     InterProcessMutex mutex = new InterProcessMutex(...) // or any InterProcessLock
+ *     try ( Locker locker = new Locker(mutex, maxTimeout, unit) )
+ *     {
+ *         // do work
+ *     }
+ * </pre></code>
+ * </p>
+ */
+public class Locker implements AutoCloseable
+{
+    private final InterProcessLock lock;
+    private final AtomicBoolean acquired = new AtomicBoolean(false);
+
+    /**
+     * @param lock a lock implementation (e.g. {@link InterProcessMutex}, {@link InterProcessSemaphoreV2}, etc.)
+     * @param timeout max timeout to acquire lock
+     * @param unit time unit of timeout
+     * @throws Exception Curator errors or {@link TimeoutException} if the lock cannot be acquired within the timeout
+     */
+    public Locker(InterProcessLock lock, long timeout, TimeUnit unit) throws Exception
+    {
+        this.lock = lock;
+        acquired.set(acquireLock(lock, timeout, unit));
+        if ( !acquired.get() )
+        {
+            throw new TimeoutException("Could not acquire lock within timeout of " + unit.toMillis(timeout) + "ms");
+        }
+    }
+
+    /**
+     * @param lock a lock implementation (e.g. {@link InterProcessMutex}, {@link InterProcessSemaphoreV2}, etc.)
+     * @throws Exception errors
+     */
+    public Locker(InterProcessLock lock) throws Exception
+    {
+        this.lock = lock;
+        acquireLock(lock);
+        acquired.set(true);
+    }
+
+    @Override
+    /**
+     * Relase the lock if it has been acquired. Can be safely called multiple times.
+     * Only the first call will unlock.
+     */
+    public void close() throws Exception
+    {
+        if ( acquired.compareAndSet(true, false) )
+        {
+            releaseLock();
+        }
+    }
+
+    protected void releaseLock() throws Exception
+    {
+        lock.release();
+    }
+
+    protected void acquireLock(InterProcessLock lock) throws Exception
+    {
+        lock.acquire();
+    }
+
+    protected boolean acquireLock(InterProcessLock lock, long timeout, TimeUnit unit) throws Exception
+    {
+        return lock.acquire(timeout, unit);
+    }
+}
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
index 0d7ab9d..93c88f7 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNode.java
@@ -22,6 +22,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.ACLBackgroundPathAndBytesable;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CreateBuilder;
@@ -44,6 +45,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
@@ -59,7 +61,7 @@
 {
     private final AtomicReference<CountDownLatch> initialCreateLatch = new AtomicReference<CountDownLatch>(new CountDownLatch(1));
     private final Logger log = LoggerFactory.getLogger(getClass());
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final CreateModable<ACLBackgroundPathAndBytesable<String>> createMethod;
     private final AtomicReference<String> nodePath = new AtomicReference<String>(null);
     private final String basePath;
@@ -74,32 +76,43 @@
         @Override
         public void process(WatchedEvent event) throws Exception
         {
-            if ( event.getType() == EventType.NodeDeleted )
+            if ( isActive() )
             {
-                createNode();
-            }
-            else if ( event.getType() == EventType.NodeDataChanged )
-            {
-                watchNode();
+                if ( event.getType() == EventType.NodeDeleted )
+                {
+                    createNode();
+                }
+                else if ( event.getType() == EventType.NodeDataChanged )
+                {
+                    watchNode();
+                }
             }
         }
     };
+
     private final BackgroundCallback checkExistsCallback = new BackgroundCallback()
     {
         @Override
-        public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+        public void processResult(CuratorFramework dummy, CuratorEvent event) throws Exception
         {
-            if ( event.getResultCode() == KeeperException.Code.NONODE.intValue() )
+            if ( isActive() )
             {
-                createNode();
+                if ( event.getResultCode() == KeeperException.Code.NONODE.intValue() )
+                {
+                    createNode();
+                }
+                else
+                {
+                    boolean isEphemeral = event.getStat().getEphemeralOwner() != 0;
+                    if ( isEphemeral != mode.isEphemeral() )
+                    {
+                        log.warn("Existing node ephemeral state doesn't match requested state. Maybe the node was created outside of PersistentNode? " + basePath);
+                    }
+                }
             }
             else
             {
-                boolean isEphemeral = event.getStat().getEphemeralOwner() != 0;
-                if ( isEphemeral != mode.isEphemeral() )
-                {
-                    log.warn("Existing node ephemeral state doesn't match requested state. Maybe the node was created outside of PersistentNode? " + basePath);
-                }
+                client.removeWatchers();
             }
         }
     };
@@ -107,7 +120,7 @@
     {
 
         @Override
-        public void processResult(CuratorFramework client, CuratorEvent event)
+        public void processResult(CuratorFramework dummy, CuratorEvent event)
             throws Exception
         {
             //If the result is ok then initialisation is complete (if we're still initialising)
@@ -123,15 +136,18 @@
     private final ConnectionStateListener connectionStateListener = new ConnectionStateListener()
     {
         @Override
-        public void stateChanged(CuratorFramework client, ConnectionState newState)
+        public void stateChanged(CuratorFramework dummy, ConnectionState newState)
         {
-            if ( newState == ConnectionState.RECONNECTED )
+            if ( (newState == ConnectionState.RECONNECTED) && isActive() )
             {
                 createNode();
             }
         }
     };
 
+    @VisibleForTesting
+    volatile CountDownLatch debugCreateNodeLatch = null;
+
     private enum State
     {
         LATENT,
@@ -140,16 +156,16 @@
     }
 
     /**
-     * @param client        client instance
+     * @param givenClient        client instance
      * @param mode          creation mode
      * @param useProtection if true, call {@link CreateBuilder#withProtection()}
      * @param basePath the base path for the node
      * @param initData data for the node
      */
-    public PersistentNode(CuratorFramework client, final CreateMode mode, boolean useProtection, final String basePath, byte[] initData)
+    public PersistentNode(CuratorFramework givenClient, final CreateMode mode, boolean useProtection, final String basePath, byte[] initData)
     {
         this.useProtection = useProtection;
-        this.client = Preconditions.checkNotNull(client, "client cannot be null");
+        this.client = Preconditions.checkNotNull(givenClient, "client cannot be null").newWatcherRemoveCuratorFramework();
         this.basePath = PathUtils.validatePath(basePath);
         this.mode = Preconditions.checkNotNull(mode, "mode cannot be null");
         final byte[] data = Preconditions.checkNotNull(initData, "data cannot be null");
@@ -157,8 +173,13 @@
         backgroundCallback = new BackgroundCallback()
         {
             @Override
-            public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+            public void processResult(CuratorFramework dummy, CuratorEvent event) throws Exception
             {
+                if ( !isActive() )
+                {
+                    return;
+                }
+
                 String path = null;
                 boolean nodeExists = false;
                 if ( event.getResultCode() == KeeperException.Code.NODEEXISTS.intValue() )
@@ -172,7 +193,7 @@
                 }
                 else if ( event.getResultCode() == KeeperException.Code.NOAUTH.intValue() )
                 {
-                    log.warn("Client does not have authorisation to write node at path {}", event.getPath());
+                    log.warn("Client does not have authorization to write node at path {}", event.getPath());
                     authFailure.set(true);
                     return;
                 }
@@ -240,9 +261,24 @@
         return (localLatch == null) || localLatch.await(timeout, unit);
     }
 
+    @VisibleForTesting
+    final AtomicLong debugWaitMsForBackgroundBeforeClose = new AtomicLong(0);
+
     @Override
     public void close() throws IOException
     {
+        if ( debugWaitMsForBackgroundBeforeClose.get() > 0 )
+        {
+            try
+            {
+                Thread.sleep(debugWaitMsForBackgroundBeforeClose.get());
+            }
+            catch ( InterruptedException e )
+            {
+                Thread.currentThread().interrupt();
+            }
+        }
+
         if ( !state.compareAndSet(State.STARTED, State.CLOSED) )
         {
             return;
@@ -259,6 +295,8 @@
             ThreadUtils.checkInterrupted(e);
             throw new IOException(e);
         }
+
+        client.removeWatchers();
     }
 
     /**
@@ -320,6 +358,19 @@
             return;
         }
 
+        if ( debugCreateNodeLatch != null )
+        {
+            try
+            {
+                debugCreateNodeLatch.await();
+            }
+            catch ( InterruptedException e )
+            {
+                Thread.currentThread().interrupt();
+                return;
+            }
+        }
+
         try
         {
             String existingPath = nodePath.get();
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java
index 032dc7a..e5c7e8c 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java
@@ -21,6 +21,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorWatcher;
@@ -36,7 +37,7 @@
 
 class ChildrenCache implements Closeable
 {
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final String path;
     private final AtomicReference<Data> children = new AtomicReference<Data>(new Data(Lists.<String>newArrayList(), 0));
     private final AtomicBoolean isClosed = new AtomicBoolean(false);
@@ -79,7 +80,7 @@
 
     ChildrenCache(CuratorFramework client, String path)
     {
-        this.client = client;
+        this.client = client.newWatcherRemoveCuratorFramework();
         this.path = PathUtils.validatePath(path);
     }
 
@@ -91,6 +92,7 @@
     @Override
     public void close() throws IOException
     {
+        client.removeWatchers();
         isClosed.set(true);
         notifyFromCallback();
     }
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java
index 17a2943..e27f455 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java
@@ -22,6 +22,7 @@
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.WatcherRemoveCuratorFramework;
 import org.apache.curator.framework.api.CuratorWatcher;
 import org.apache.curator.framework.listen.ListenerContainer;
 import org.apache.curator.framework.state.ConnectionState;
@@ -46,7 +47,7 @@
 {
     private final Logger log = LoggerFactory.getLogger(getClass());
     private final ListenerContainer<SharedValueListener> listeners = new ListenerContainer<SharedValueListener>();
-    private final CuratorFramework client;
+    private final WatcherRemoveCuratorFramework client;
     private final String path;
     private final byte[] seedValue;
     private final AtomicReference<State> state = new AtomicReference<State>(State.LATENT);
@@ -88,7 +89,7 @@
      */
     public SharedValue(CuratorFramework client, String path, byte[] seedValue)
     {
-        this.client = client;
+        this.client = client.newWatcherRemoveCuratorFramework();
         this.path = PathUtils.validatePath(path);
         this.seedValue = Arrays.copyOf(seedValue, seedValue.length);
         currentValue = new AtomicReference<VersionedValue<byte[]>>(new VersionedValue<byte[]>(0, Arrays.copyOf(seedValue, seedValue.length)));
@@ -234,8 +235,9 @@
     @Override
     public void close() throws IOException
     {
-        client.getConnectionStateListenable().removeListener(connectionStateListener);
         state.set(State.CLOSED);
+        client.removeWatchers();
+        client.getConnectionStateListenable().removeListener(connectionStateListener);
         listeners.clear();
     }
 
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/imps/TestCleanState.java b/curator-recipes/src/test/java/org/apache/curator/framework/imps/TestCleanState.java
new file mode 100644
index 0000000..f90f463
--- /dev/null
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/imps/TestCleanState.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.framework.imps;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.test.WatchersDebug;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.ZooKeeper;
+
+public class TestCleanState
+{
+    public static void closeAndTestClean(CuratorFramework client)
+    {
+        if ( client == null )
+        {
+            return;
+        }
+
+        try
+        {
+            CuratorFrameworkImpl internalClient = (CuratorFrameworkImpl)client;
+            EnsembleTracker ensembleTracker = internalClient.getEnsembleTracker();
+            if ( ensembleTracker != null )
+            {
+                ensembleTracker.close();
+            }
+            ZooKeeper zooKeeper = internalClient.getZooKeeper();
+            if ( zooKeeper != null )
+            {
+                if ( WatchersDebug.getChildWatches(zooKeeper).size() != 0 )
+                {
+                    throw new AssertionError("One or more child watchers are still registered: " + WatchersDebug.getChildWatches(zooKeeper));
+                }
+                if ( WatchersDebug.getExistWatches(zooKeeper).size() != 0 )
+                {
+                    throw new AssertionError("One or more exists watchers are still registered: " + WatchersDebug.getExistWatches(zooKeeper));
+                }
+                if ( WatchersDebug.getDataWatches(zooKeeper).size() != 0 )
+                {
+                    throw new AssertionError("One or more data watchers are still registered: " + WatchersDebug.getDataWatches(zooKeeper));
+                }
+            }
+        }
+        catch ( IllegalStateException ignore )
+        {
+            // client already closed
+        }
+        catch ( Exception e )
+        {
+            e.printStackTrace();    // not sure what to do here
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    private TestCleanState()
+    {
+    }
+}
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 ab37785..f32c9b2 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
@@ -22,6 +22,7 @@
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.UnhandledErrorListener;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
@@ -123,7 +124,7 @@
             finally
             {
                 CloseableUtils.closeQuietly(cache);
-                CloseableUtils.closeQuietly(client);
+                TestCleanState.closeAndTestClean(client);
             }
         }
         finally
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 27af6ac..d6d495a 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
@@ -18,6 +18,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.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
@@ -98,7 +99,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -161,7 +162,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -204,7 +205,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -252,7 +253,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 }
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 14d061f..8792044 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
@@ -18,48 +18,23 @@
  */
 package org.apache.curator.framework.recipes.cache;
 
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Collections2;
 import com.google.common.collect.Lists;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.api.BackgroundCallback;
-import org.apache.curator.framework.api.CuratorEvent;
-import org.apache.curator.framework.api.Pathable;
 import org.apache.curator.framework.api.UnhandledErrorListener;
-import org.apache.curator.framework.imps.CuratorFrameworkImpl;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
 import org.apache.curator.test.KillSession;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
-import org.apache.log4j.Appender;
-import org.apache.log4j.AppenderSkeleton;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.log4j.SimpleLayout;
-import org.apache.log4j.spi.LoggingEvent;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.testng.Assert;
 import org.testng.annotations.Test;
-
-import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Exchanger;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -97,7 +72,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -139,7 +114,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -193,7 +168,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -242,7 +217,7 @@
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -251,6 +226,7 @@
     {
         Timing timing = new Timing();
 
+        PathChildrenCache cache = null;
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
         client.start();
         try
@@ -258,7 +234,7 @@
             final CountDownLatch updatedLatch = new CountDownLatch(1);
             final CountDownLatch addedLatch = new CountDownLatch(1);
             client.create().creatingParentsIfNeeded().forPath("/test");
-            PathChildrenCache cache = new PathChildrenCache(client, "/test", false);
+            cache = new PathChildrenCache(client, "/test", false);
             cache.getListenable().addListener
                 (
                     new PathChildrenCacheListener()
@@ -287,7 +263,8 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -300,22 +277,25 @@
         client.start();
         try
         {
-            PathChildrenCache cache = new PathChildrenCache(client, "/one/two/three", false);
-            cache.start();
-            timing.sleepABit();
+            try ( PathChildrenCache cache = new PathChildrenCache(client, "/one/two/three", false) )
+            {
+                cache.start();
+                timing.sleepABit();
 
-            try
-            {
-                client.create().forPath("/one/two/three/four");
+                try
+                {
+                    client.create().forPath("/one/two/three/four");
+                }
+                catch ( KeeperException.NoNodeException e )
+                {
+                    Assert.fail("Path should exist", e);
+                }
             }
-            catch ( KeeperException.NoNodeException e )
-            {
-                Assert.fail("Path should exist", e);
-            }
+            timing.sleepABit();
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -346,52 +326,52 @@
             final CountDownLatch removedLatch = new CountDownLatch(1);
             final CountDownLatch postRemovedLatch = new CountDownLatch(1);
             final CountDownLatch dataLatch = new CountDownLatch(1);
-            PathChildrenCache cache = new PathChildrenCache(client, "/test", true);
-            cache.getListenable().addListener
-                (
-                    new PathChildrenCacheListener()
-                    {
-                        @Override
-                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+            try ( PathChildrenCache cache = new PathChildrenCache(client, "/test", true) )
+            {
+                cache.getListenable().addListener
+                    (
+                        new PathChildrenCacheListener()
                         {
-                            if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_REMOVED )
+                            @Override
+                            public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                             {
-                                removedLatch.countDown();
-                                Assert.assertTrue(postRemovedLatch.await(10, TimeUnit.SECONDS));
-                            }
-                            else
-                            {
-                                try
+                                if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_REMOVED )
                                 {
-                                    Assert.assertEquals(event.getData().getData(), "two".getBytes());
+                                    removedLatch.countDown();
+                                    Assert.assertTrue(postRemovedLatch.await(10, TimeUnit.SECONDS));
                                 }
-                                finally
+                                else
                                 {
-                                    dataLatch.countDown();
+                                    try
+                                    {
+                                        Assert.assertEquals(event.getData().getData(), "two".getBytes());
+                                    }
+                                    finally
+                                    {
+                                        dataLatch.countDown();
+                                    }
                                 }
                             }
                         }
-                    }
-                );
-            cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
+                    );
+                cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
 
-            client.delete().forPath("/test/foo");
-            Assert.assertTrue(timing.awaitLatch(removedLatch));
-            client.create().forPath("/test/foo", "two".getBytes());
-            postRemovedLatch.countDown();
-            Assert.assertTrue(timing.awaitLatch(dataLatch));
+                client.delete().forPath("/test/foo");
+                Assert.assertTrue(timing.awaitLatch(removedLatch));
+                client.create().forPath("/test/foo", "two".getBytes());
+                postRemovedLatch.countDown();
+                Assert.assertTrue(timing.awaitLatch(dataLatch));
 
-            Throwable t = error.get();
-            if ( t != null )
-            {
-                Assert.fail("Assert", t);
+                Throwable t = error.get();
+                if ( t != null )
+                {
+                    Assert.fail("Assert", t);
+                }
             }
-
-            cache.close();
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -409,83 +389,83 @@
             client.create().forPath("/test/snafu", "original".getBytes());
 
             final CountDownLatch addedLatch = new CountDownLatch(2);
-            final PathChildrenCache cache = new PathChildrenCache(client, "/test", true);
-            cache.getListenable().addListener
-                (
-                    new PathChildrenCacheListener()
-                    {
-                        @Override
-                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+            try ( final PathChildrenCache cache = new PathChildrenCache(client, "/test", true) )
+            {
+                cache.getListenable().addListener
+                    (
+                        new PathChildrenCacheListener()
                         {
-                            if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_ADDED )
+                            @Override
+                            public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                             {
-                                if ( event.getData().getPath().equals("/test/test") )
+                                if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_ADDED )
                                 {
-                                    addedLatch.countDown();
+                                    if ( event.getData().getPath().equals("/test/test") )
+                                    {
+                                        addedLatch.countDown();
+                                    }
                                 }
-                            }
-                            else if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_UPDATED )
-                            {
-                                if ( event.getData().getPath().equals("/test/snafu") )
+                                else if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_UPDATED )
                                 {
-                                    addedLatch.countDown();
+                                    if ( event.getData().getPath().equals("/test/snafu") )
+                                    {
+                                        addedLatch.countDown();
+                                    }
                                 }
                             }
                         }
-                    }
-                );
-            cache.rebuildTestExchanger = new Exchanger<Object>();
-            ExecutorService service = Executors.newSingleThreadExecutor();
-            final AtomicReference<String> deletedPath = new AtomicReference<String>();
-            Future<Object> future = service.submit
-                (
-                    new Callable<Object>()
-                    {
-                        @Override
-                        public Object call() throws Exception
+                    );
+                cache.rebuildTestExchanger = new Exchanger<Object>();
+                ExecutorService service = Executors.newSingleThreadExecutor();
+                final AtomicReference<String> deletedPath = new AtomicReference<String>();
+                Future<Object> future = service.submit
+                    (
+                        new Callable<Object>()
                         {
-                            cache.rebuildTestExchanger.exchange(new Object());
-
-                            // simulate another process adding a node while we're rebuilding
-                            client.create().forPath("/test/test");
-
-                            List<ChildData> currentData = cache.getCurrentData();
-                            Assert.assertTrue(currentData.size() > 0);
-
-                            // simulate another process removing a node while we're rebuilding
-                            client.delete().forPath(currentData.get(0).getPath());
-                            deletedPath.set(currentData.get(0).getPath());
-
-                            cache.rebuildTestExchanger.exchange(new Object());
-
-                            ChildData childData = null;
-                            while ( childData == null )
+                            @Override
+                            public Object call() throws Exception
                             {
-                                childData = cache.getCurrentData("/test/snafu");
-                                Thread.sleep(1000);
+                                cache.rebuildTestExchanger.exchange(new Object());
+
+                                // simulate another process adding a node while we're rebuilding
+                                client.create().forPath("/test/test");
+
+                                List<ChildData> currentData = cache.getCurrentData();
+                                Assert.assertTrue(currentData.size() > 0);
+
+                                // simulate another process removing a node while we're rebuilding
+                                client.delete().forPath(currentData.get(0).getPath());
+                                deletedPath.set(currentData.get(0).getPath());
+
+                                cache.rebuildTestExchanger.exchange(new Object());
+
+                                ChildData childData = null;
+                                while ( childData == null )
+                                {
+                                    childData = cache.getCurrentData("/test/snafu");
+                                    Thread.sleep(1000);
+                                }
+                                Assert.assertEquals(childData.getData(), "original".getBytes());
+                                client.setData().forPath("/test/snafu", "grilled".getBytes());
+
+                                cache.rebuildTestExchanger.exchange(new Object());
+
+                                return null;
                             }
-                            Assert.assertEquals(childData.getData(), "original".getBytes());
-                            client.setData().forPath("/test/snafu", "grilled".getBytes());
-
-                            cache.rebuildTestExchanger.exchange(new Object());
-
-                            return null;
                         }
-                    }
-                );
-            cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
-            future.get();
+                    );
+                cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
+                future.get();
 
-            Assert.assertTrue(timing.awaitLatch(addedLatch));
-            Assert.assertNotNull(cache.getCurrentData("/test/test"));
-            Assert.assertNull(cache.getCurrentData(deletedPath.get()));
-            Assert.assertEquals(cache.getCurrentData("/test/snafu").getData(), "grilled".getBytes());
-
-            cache.close();
+                Assert.assertTrue(timing.awaitLatch(addedLatch));
+                Assert.assertNotNull(cache.getCurrentData("/test/test"));
+                Assert.assertNull(cache.getCurrentData(deletedPath.get()));
+                Assert.assertEquals(cache.getCurrentData("/test/snafu").getData(), "grilled".getBytes());
+            }
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -494,6 +474,7 @@
     public void testIssue27() throws Exception
     {
         Timing timing = new Timing();
+        PathChildrenCache cache = null;
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
         client.start();
         try
@@ -507,7 +488,7 @@
 
             final List<PathChildrenCacheEvent.Type> events = Lists.newArrayList();
             final Semaphore semaphore = new Semaphore(0);
-            PathChildrenCache cache = new PathChildrenCache(client, "/base", true);
+            cache = new PathChildrenCache(client, "/base", true);
             cache.getListenable().addListener
                 (
                     new PathChildrenCacheListener()
@@ -542,7 +523,8 @@
         }
         finally
         {
-            client.close();
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -551,6 +533,7 @@
     public void testIssue27Alt() throws Exception
     {
         Timing timing = new Timing();
+        PathChildrenCache cache = null;
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
         client.start();
         try
@@ -564,7 +547,7 @@
 
             final List<PathChildrenCacheEvent.Type> events = Lists.newArrayList();
             final Semaphore semaphore = new Semaphore(0);
-            PathChildrenCache cache = new PathChildrenCache(client, "/base", true);
+            cache = new PathChildrenCache(client, "/base", true);
             cache.getListenable().addListener
                 (
                     new PathChildrenCacheListener()
@@ -594,7 +577,8 @@
         }
         finally
         {
-            client.close();
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -602,6 +586,7 @@
     public void testKilledSession() throws Exception
     {
         Timing timing = new Timing();
+        PathChildrenCache cache = null;
         CuratorFramework client = null;
         try
         {
@@ -609,7 +594,7 @@
             client.start();
             client.create().forPath("/test");
 
-            PathChildrenCache cache = new PathChildrenCache(client, "/test", true);
+            cache = new PathChildrenCache(client, "/test", true);
             cache.start();
 
             final CountDownLatch childAddedLatch = new CountDownLatch(1);
@@ -646,14 +631,15 @@
             client.create().withMode(CreateMode.EPHEMERAL).forPath("/test/me", "data".getBytes());
             Assert.assertTrue(timing.awaitLatch(childAddedLatch));
 
-            KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
+            KillSession.kill(client.getZookeeperClient().getZooKeeper());
             Assert.assertTrue(timing.awaitLatch(lostLatch));
             Assert.assertTrue(timing.awaitLatch(reconnectedLatch));
             Assert.assertTrue(timing.awaitLatch(removedLatch));
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            CloseableUtils.closeQuietly(cache);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -677,7 +663,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -687,9 +673,9 @@
         Timing timing = new Timing();
         PathChildrenCache cache = null;
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
-        client.start();
         try
         {
+            client.start();
             client.create().creatingParentsIfNeeded().forPath("/test/one", "one".getBytes());
 
             final CountDownLatch latch = new CountDownLatch(1);
@@ -708,7 +694,7 @@
             };
             cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
 
-            latch.await();
+            Assert.assertTrue(timing.awaitLatch(latch));
 
             int saveCounter = counter.get();
             client.setData().forPath("/test/one", "alt".getBytes());
@@ -717,54 +703,54 @@
             Assert.assertEquals(saveCounter, counter.get());
 
             semaphore.release(1000);
+            timing.sleepABit();
         }
         finally
         {
             CloseableUtils.closeQuietly(cache);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
     private void internalTestMode(CuratorFramework client, boolean cacheData) throws Exception
     {
-        PathChildrenCache cache = new PathChildrenCache(client, "/test", cacheData);
-
-        final CountDownLatch latch = new CountDownLatch(2);
-        cache.getListenable().addListener
-            (
-                new PathChildrenCacheListener()
-                {
-                    @Override
-                    public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+        try ( PathChildrenCache cache = new PathChildrenCache(client, "/test", cacheData) )
+        {
+            final CountDownLatch latch = new CountDownLatch(2);
+            cache.getListenable().addListener
+                (
+                    new PathChildrenCacheListener()
                     {
-                        if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_ADDED )
+                        @Override
+                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                         {
-                            latch.countDown();
+                            if ( event.getType() == PathChildrenCacheEvent.Type.CHILD_ADDED )
+                            {
+                                latch.countDown();
+                            }
                         }
                     }
+                );
+            cache.start();
+
+            client.create().forPath("/test/one", "one".getBytes());
+            client.create().forPath("/test/two", "two".getBytes());
+            Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
+
+            for ( ChildData data : cache.getCurrentData() )
+            {
+                if ( cacheData )
+                {
+                    Assert.assertNotNull(data.getData());
+                    Assert.assertNotNull(data.getStat());
                 }
-            );
-        cache.start();
-
-        client.create().forPath("/test/one", "one".getBytes());
-        client.create().forPath("/test/two", "two".getBytes());
-        Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
-
-        for ( ChildData data : cache.getCurrentData() )
-        {
-            if ( cacheData )
-            {
-                Assert.assertNotNull(data.getData());
-                Assert.assertNotNull(data.getStat());
-            }
-            else
-            {
-                Assert.assertNull(data.getData());
-                Assert.assertNotNull(data.getStat());
+                else
+                {
+                    Assert.assertNull(data.getData());
+                    Assert.assertNotNull(data.getStat());
+                }
             }
         }
-
-        cache.close();
     }
 
     @Test
@@ -778,38 +764,38 @@
             client.create().forPath("/test");
 
             final BlockingQueue<PathChildrenCacheEvent.Type> events = new LinkedBlockingQueue<PathChildrenCacheEvent.Type>();
-            PathChildrenCache cache = new PathChildrenCache(client, "/test", true);
-            cache.getListenable().addListener
-                (
-                    new PathChildrenCacheListener()
-                    {
-                        @Override
-                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+            try ( PathChildrenCache cache = new PathChildrenCache(client, "/test", true) )
+            {
+                cache.getListenable().addListener
+                    (
+                        new PathChildrenCacheListener()
                         {
-                            if ( event.getData().getPath().equals("/test/one") )
+                            @Override
+                            public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                             {
-                                events.offer(event.getType());
+                                if ( event.getData().getPath().equals("/test/one") )
+                                {
+                                    events.offer(event.getType());
+                                }
                             }
                         }
-                    }
-                );
-            cache.start();
+                    );
+                cache.start();
 
-            client.create().forPath("/test/one", "hey there".getBytes());
-            Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_ADDED);
+                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.getCurrentData("/test/one").getData()), "sup!");
+                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.getCurrentData("/test/one").getData()), "sup!");
 
-            client.delete().forPath("/test/one");
-            Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
-
-            cache.close();
+                client.delete().forPath("/test/one");
+                Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
+            }
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -825,60 +811,62 @@
 
             final BlockingQueue<PathChildrenCacheEvent.Type> events = new LinkedBlockingQueue<PathChildrenCacheEvent.Type>();
             final ExecutorService exec = Executors.newSingleThreadExecutor();
-            PathChildrenCache cache = new PathChildrenCache(client, "/test", true, false, exec);
-            cache.getListenable().addListener
-                (
-                    new PathChildrenCacheListener()
-                    {
-                        @Override
-                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
+            try ( PathChildrenCache cache = new PathChildrenCache(client, "/test", true, false, exec) )
+            {
+                cache.getListenable().addListener
+                    (
+                        new PathChildrenCacheListener()
                         {
-                            if ( event.getData().getPath().equals("/test/one") )
+                            @Override
+                            public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
                             {
-                                events.offer(event.getType());
+                                if ( event.getData().getPath().equals("/test/one") )
+                                {
+                                    events.offer(event.getType());
+                                }
                             }
                         }
-                    }
-                );
-            cache.start();
+                    );
+                cache.start();
 
-            final BlockingQueue<PathChildrenCacheEvent.Type> events2 = new LinkedBlockingQueue<PathChildrenCacheEvent.Type>();
-            PathChildrenCache cache2 = new PathChildrenCache(client, "/test", true, false, exec);
-            cache2.getListenable().addListener(
-                    new PathChildrenCacheListener() {
-                        @Override
-                        public void childEvent(CuratorFramework client, PathChildrenCacheEvent event)
+                final BlockingQueue<PathChildrenCacheEvent.Type> events2 = new LinkedBlockingQueue<PathChildrenCacheEvent.Type>();
+                try ( PathChildrenCache cache2 = new PathChildrenCache(client, "/test", true, false, exec) )
+                {
+                    cache2.getListenable().addListener(
+                        new PathChildrenCacheListener()
+                        {
+                            @Override
+                            public void childEvent(CuratorFramework client, PathChildrenCacheEvent event)
                                 throws Exception
-                        {
-                            if ( event.getData().getPath().equals("/test/one") )
                             {
-                                events2.offer(event.getType());
+                                if ( event.getData().getPath().equals("/test/one") )
+                                {
+                                    events2.offer(event.getType());
+                                }
                             }
                         }
-                    }
-            );
-            cache2.start();
+                                                      );
+                    cache2.start();
 
-            client.create().forPath("/test/one", "hey there".getBytes());
-            Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_ADDED);
-            Assert.assertEquals(events2.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_ADDED);
+                    client.create().forPath("/test/one", "hey there".getBytes());
+                    Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_ADDED);
+                    Assert.assertEquals(events2.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(events2.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_UPDATED);
-            Assert.assertEquals(new String(cache.getCurrentData("/test/one").getData()), "sup!");
-            Assert.assertEquals(new String(cache2.getCurrentData("/test/one").getData()), "sup!");
+                    client.setData().forPath("/test/one", "sup!".getBytes());
+                    Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_UPDATED);
+                    Assert.assertEquals(events2.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_UPDATED);
+                    Assert.assertEquals(new String(cache.getCurrentData("/test/one").getData()), "sup!");
+                    Assert.assertEquals(new String(cache2.getCurrentData("/test/one").getData()), "sup!");
 
-            client.delete().forPath("/test/one");
-            Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
-            Assert.assertEquals(events2.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
-
-            cache.close();
-            cache2.close();
+                    client.delete().forPath("/test/one");
+                    Assert.assertEquals(events.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
+                    Assert.assertEquals(events2.poll(timing.forWaiting().seconds(), TimeUnit.SECONDS), PathChildrenCacheEvent.Type.CHILD_REMOVED);
+                }
+            }
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -894,25 +882,26 @@
             client.create().forPath("/test");
 
             final ExecuteCalledWatchingExecutorService exec = new ExecuteCalledWatchingExecutorService(Executors.newSingleThreadExecutor());
-            PathChildrenCache cache = new PathChildrenCache(client, "/test", true, false, exec);
+            try ( PathChildrenCache cache = new PathChildrenCache(client, "/test", true, false, exec) )
+            {
+                cache.start();
+                client.create().forPath("/test/one", "hey there".getBytes());
 
-            cache.start();
-            client.create().forPath("/test/one", "hey there".getBytes());
+                cache.rebuild();
+                Assert.assertEquals(new String(cache.getCurrentData("/test/one").getData()), "hey there");
+                Assert.assertTrue(exec.isExecuteCalled());
 
-            cache.rebuild();
-            Assert.assertEquals(new String(cache.getCurrentData("/test/one").getData()), "hey there");
-            Assert.assertTrue(exec.isExecuteCalled());
-
-            exec.setExecuteCalled(false);
-            cache.close();
+                exec.setExecuteCalled(false);
+            }
             Assert.assertFalse(exec.isExecuteCalled());
 
             client.delete().forPath("/test/one");
             timing.sleepABit();
             Assert.assertFalse(exec.isExecuteCalled());
         }
-        finally {
-            client.close();
+        finally
+        {
+            TestCleanState.closeAndTestClean(client);
         }
 
     }
@@ -931,35 +920,37 @@
         try
         {
             final CountDownLatch latch = new CountDownLatch(1);
-            final PathChildrenCache cache = new PathChildrenCache(client, "/test", false) {
+            try ( final PathChildrenCache cache = new PathChildrenCache(client, "/test", false) {
                 @Override
                 protected void handleException(Throwable e)
                 {
                     latch.countDown();
                 }
-            };
-            cache.start();
-
-            cache.offerOperation(new Operation()
+            } )
             {
+                cache.start();
 
-                @Override
-                public void invoke() throws Exception
+                cache.offerOperation(new Operation()
                 {
-                    Thread.sleep(5000);
-                }
-            });
 
-            Thread.sleep(1000);
+                    @Override
+                    public void invoke() throws Exception
+                    {
+                        Thread.sleep(5000);
+                    }
+                });
 
-            cache.close();
+                Thread.sleep(1000);
+
+            }
 
             latch.await(5, TimeUnit.SECONDS);
 
             Assert.assertTrue(latch.getCount() == 1, "Unexpected exception occurred");
-        } finally
+        }
+        finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 }
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 ad7c417..173fcd7 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
@@ -424,7 +424,6 @@
         assertEvent(TreeCacheEvent.Type.NODE_ADDED, "/test/me");
 
         KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
-        assertEvent(TreeCacheEvent.Type.CONNECTION_SUSPENDED);
         assertEvent(TreeCacheEvent.Type.CONNECTION_LOST);
         assertEvent(TreeCacheEvent.Type.CONNECTION_RECONNECTED);
         assertEvent(TreeCacheEvent.Type.NODE_REMOVED, "/test/me", "data".getBytes());
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 5f10c5e..4cb342c 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,6 +19,7 @@
 
 package org.apache.curator.framework.recipes.leader;
 
+import org.apache.curator.test.TestingZooKeeperMain;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.proto.CreateRequest;
 import org.apache.zookeeper.server.ByteBufferInputStream;
@@ -30,8 +31,6 @@
 import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
 
 /**
  * A connection factory that will behave like the NIOServerCnxnFactory except that
@@ -49,30 +48,20 @@
     /* How long after the first error, connections are rejected */
     public static final long LOCKOUT_DURATION_MS = 6000;
 
-    public ChaosMonkeyCnxnFactory() throws IOException
-    {
-    }
-
     @Override
     public void startup(ZooKeeperServer zks) throws IOException, InterruptedException
     {
         super.startup(new ChaosMonkeyZookeeperServer(zks));
     }
 
-    /**
-     * Build a connection with a Chaos Monkey ZookeeperServer
-     */
-    protected NIOServerCnxn createConnection(SocketChannel sock, SelectionKey sk) throws IOException
-    {
-        return new NIOServerCnxn(zkServer, sock, sk, this);
-    }
-
     public static class ChaosMonkeyZookeeperServer extends ZooKeeperServer
     {
+        private final ZooKeeperServer zks;
         private long firstError = 0;
 
         public ChaosMonkeyZookeeperServer(ZooKeeperServer zks)
         {
+            this.zks = zks;
             setTxnLogFactory(zks.getTxnLogFactory());
             setTickTime(zks.getTickTime());
             setMinSessionTimeout(zks.getMinSessionTimeout());
@@ -80,6 +69,20 @@
         }
 
         @Override
+        public void startup()
+        {
+            super.startup();
+            if ( zks instanceof TestingZooKeeperMain.TestZooKeeperServer )
+            {
+                ((TestingZooKeeperMain.TestZooKeeperServer)zks).noteStartup();
+            }
+            else
+            {
+                throw new RuntimeException("Unknown ZooKeeperServer: " + zks.getClass());
+            }
+        }
+
+        @Override
         public void submitRequest(Request si)
         {
             long remaining = firstError != 0 ? LOCKOUT_DURATION_MS - (System.currentTimeMillis() - firstError) : 0;
@@ -96,7 +99,7 @@
             log.debug("Applied : " + si.toString());
             super.submitRequest(si);
             // Raise an error if a lock is created
-            if ( si.type == ZooDefs.OpCode.create )
+            if ( (si.type == ZooDefs.OpCode.create) || (si.type == ZooDefs.OpCode.create2) )
             {
                 CreateRequest createRequest = new CreateRequest();
                 try
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 96e6d45..991e6fc 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
@@ -21,11 +21,15 @@
 
 import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Queues;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.framework.state.SessionConnectionStateErrorPolicy;
+import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
@@ -34,8 +38,10 @@
 import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorCompletionService;
@@ -52,6 +58,167 @@
     private static final int MAX_LOOPS = 5;
 
     @Test
+    public void testSessionErrorPolicy() throws Exception
+    {
+        Timing timing = new Timing();
+        LeaderLatch latch = null;
+        CuratorFramework client = null;
+        for ( int i = 0; i < 2; ++i )
+        {
+            boolean isSessionIteration = (i == 0);
+            try
+            {
+                client = CuratorFrameworkFactory.builder()
+                    .connectString(server.getConnectString())
+                    .connectionTimeoutMs(10000)
+                    .sessionTimeoutMs(60000)
+                    .retryPolicy(new RetryOneTime(1))
+                    .connectionStateErrorPolicy(isSessionIteration ? new SessionConnectionStateErrorPolicy() : new StandardConnectionStateErrorPolicy())
+                    .build();
+                final BlockingQueue<String> states = Queues.newLinkedBlockingQueue();
+                ConnectionStateListener stateListener = new ConnectionStateListener()
+                {
+                    @Override
+                    public void stateChanged(CuratorFramework client, ConnectionState newState)
+                    {
+                        states.add(newState.name());
+                    }
+                };
+                client.getConnectionStateListenable().addListener(stateListener);
+                client.start();
+
+                latch = new LeaderLatch(client, "/test");
+                LeaderLatchListener listener = new LeaderLatchListener()
+                {
+                    @Override
+                    public void isLeader()
+                    {
+                        states.add("true");
+                    }
+
+                    @Override
+                    public void notLeader()
+                    {
+                        states.add("false");
+                    }
+                };
+                latch.addListener(listener);
+                latch.start();
+                Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name());
+                Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
+                server.stop();
+                if ( isSessionIteration )
+                {
+                    Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED.name());
+                    server.restart();
+                    Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED.name());
+                    Assert.assertNull(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS));
+                }
+                else
+                {
+                    String s = states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS);
+                    Assert.assertTrue("false".equals(s) || ConnectionState.SUSPENDED.name().equals(s));
+                    s = states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS);
+                    Assert.assertTrue("false".equals(s) || ConnectionState.SUSPENDED.name().equals(s));
+                    server.restart();
+                    Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED.name());
+                    Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
+                }
+            }
+            finally
+            {
+                CloseableUtils.closeQuietly(latch);
+                CloseableUtils.closeQuietly(client);
+            }
+        }
+    }
+
+    @Test
+    public void testErrorPolicies() throws Exception
+    {
+        Timing timing = new Timing();
+        LeaderLatch latch = null;
+        CuratorFramework client = CuratorFrameworkFactory.builder()
+            .connectString(server.getConnectString())
+            .connectionTimeoutMs(1000)
+            .sessionTimeoutMs(timing.session())
+            .retryPolicy(new RetryOneTime(1))
+            .connectionStateErrorPolicy(new StandardConnectionStateErrorPolicy())
+            .build();
+        try
+        {
+            final BlockingQueue<String> states = Queues.newLinkedBlockingQueue();
+            ConnectionStateListener stateListener = new ConnectionStateListener()
+            {
+                @Override
+                public void stateChanged(CuratorFramework client, ConnectionState newState)
+                {
+                    states.add(newState.name());
+                }
+            };
+            client.getConnectionStateListenable().addListener(stateListener);
+            client.start();
+            latch = new LeaderLatch(client, "/test");
+            LeaderLatchListener listener = new LeaderLatchListener()
+            {
+                @Override
+                public void isLeader()
+                {
+                    states.add("true");
+                }
+
+                @Override
+                public void notLeader()
+                {
+                    states.add("false");
+                }
+            };
+            latch.addListener(listener);
+            latch.start();
+            Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name());
+            Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
+            server.close();
+            List<String> next = Lists.newArrayList();
+            next.add(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS));
+            next.add(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS));
+            Assert.assertTrue(next.equals(Arrays.asList(ConnectionState.SUSPENDED.name(), "false")) || next.equals(Arrays.asList("false", ConnectionState.SUSPENDED.name())), next.toString());
+            Assert.assertEquals(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST.name());
+            latch.close();
+            client.close();
+
+            timing.sleepABit();
+            states.clear();
+
+            server = new TestingServer();
+            client = CuratorFrameworkFactory.builder()
+                .connectString(server.getConnectString())
+                .connectionTimeoutMs(1000)
+                .sessionTimeoutMs(timing.session())
+                .retryPolicy(new RetryOneTime(1))
+                .connectionStateErrorPolicy(new SessionConnectionStateErrorPolicy())
+                .build();
+            client.getConnectionStateListenable().addListener(stateListener);
+            client.start();
+            latch = new LeaderLatch(client, "/test");
+            latch.addListener(listener);
+            latch.start();
+            Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name());
+            Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
+            server.close();
+            Assert.assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED.name());
+            next = Lists.newArrayList();
+            next.add(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS));
+            next.add(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS));
+            Assert.assertTrue(next.equals(Arrays.asList(ConnectionState.LOST.name(), "false")) || next.equals(Arrays.asList("false", ConnectionState.LOST.name())), next.toString());
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(latch);
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
     public void testProperCloseWithoutConnectionEstablished() throws Exception
     {
         server.stop();
@@ -96,7 +263,7 @@
         finally
         {
             CloseableUtils.closeQuietly(latch);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -126,7 +293,7 @@
         finally
         {
             CloseableUtils.closeQuietly(latch);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -158,7 +325,7 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -213,7 +380,7 @@
             {
                 CloseableUtils.closeQuietly(latch);
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -236,10 +403,9 @@
                 LeaderLatch latch = new LeaderLatch(client, PATH_NAME);
                 latch.start();
                 latches.add(latch);
+                waitForALeader(latches, timing);
             }
 
-            waitForALeader(latches, timing);
-
             //we need to close a Participant that doesn't be actual leader (first Participant) nor the last
             latches.get(PARTICIPANT_ID).close();
 
@@ -256,9 +422,8 @@
             {
                 CloseableUtils.closeQuietly(latch);
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
-
     }
 
     @Test
@@ -320,7 +485,7 @@
         finally
         {
             executorService.shutdownNow();
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -416,7 +581,7 @@
                     CloseableUtils.closeQuietly(latch);
                 }
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -504,7 +669,7 @@
                     CloseableUtils.closeQuietly(latch);
                 }
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -583,7 +748,7 @@
             {
                 CloseableUtils.closeQuietly(notifiedLeader);
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -639,7 +804,7 @@
         finally
         {
             CloseableUtils.closeQuietly(leader);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
             CloseableUtils.closeQuietly(server);
         }
     }
@@ -709,7 +874,7 @@
             {
                 CloseableUtils.closeQuietly(latch);
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -745,6 +910,6 @@
     {
         Timing timing = new Timing();
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
-        LeaderLatch latch = new LeaderLatch(client, "parent");
+        new LeaderLatch(client, "parent");
     }
 }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
index c7f415c..e9a7002 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java
@@ -20,10 +20,13 @@
 package org.apache.curator.framework.recipes.leader;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Queues;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.framework.state.SessionConnectionStateErrorPolicy;
+import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.KillSession;
@@ -33,6 +36,7 @@
 import org.testng.Assert;
 import org.testng.annotations.Test;
 import org.testng.internal.annotations.Sets;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
@@ -50,6 +54,97 @@
     private static final String PATH_NAME = "/one/two/me";
 
     @Test
+    public void testErrorPolicies() throws Exception
+    {
+        Timing timing = new Timing();
+        LeaderSelector selector = null;
+        CuratorFramework client = CuratorFrameworkFactory
+            .builder()
+            .connectString(server.getConnectString())
+            .connectionTimeoutMs(timing.connection())
+            .sessionTimeoutMs(timing.session())
+            .retryPolicy(new RetryOneTime(1))
+            .connectionStateErrorPolicy(new StandardConnectionStateErrorPolicy())
+            .build();
+        try
+        {
+            final BlockingQueue<String> changes = Queues.newLinkedBlockingQueue();
+
+            ConnectionStateListener stateListener = new ConnectionStateListener()
+            {
+                @Override
+                public void stateChanged(CuratorFramework client, ConnectionState newState)
+                {
+                    changes.add(newState.name());
+                }
+            };
+            client.getConnectionStateListenable().addListener(stateListener);
+            client.start();
+            LeaderSelectorListener listener = new LeaderSelectorListenerAdapter()
+            {
+                @Override
+                public void takeLeadership(CuratorFramework client) throws Exception
+                {
+                    changes.add("leader");
+                    try
+                    {
+                        Thread.currentThread().join();
+                    }
+                    catch ( InterruptedException e )
+                    {
+                        changes.add("release");
+                        Thread.currentThread().interrupt();
+                    }
+                }
+            };
+            selector = new LeaderSelector(client, "/test", listener);
+            selector.start();
+
+            Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name());
+            Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "leader");
+            server.close();
+            List<String> next = Lists.newArrayList();
+            next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS));
+            next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS));
+            Assert.assertTrue(next.equals(Arrays.asList(ConnectionState.SUSPENDED.name(), "release")) || next.equals(Arrays.asList("release", ConnectionState.SUSPENDED.name())), next.toString());
+            Assert.assertEquals(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST.name());
+
+            selector.close();
+            client.close();
+            timing.sleepABit();
+            changes.clear();
+
+            server = new TestingServer();
+            client = CuratorFrameworkFactory
+                .builder()
+                .connectString(server.getConnectString())
+                .connectionTimeoutMs(timing.connection())
+                .sessionTimeoutMs(timing.session())
+                .retryPolicy(new RetryOneTime(1))
+                .connectionStateErrorPolicy(new SessionConnectionStateErrorPolicy())
+                .build();
+            client.getConnectionStateListenable().addListener(stateListener);
+            client.start();
+            selector = new LeaderSelector(client, "/test", listener);
+            selector.start();
+
+            Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name());
+            Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "leader");
+            server.stop();
+            Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED.name());
+            next = Lists.newArrayList();
+            next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS));
+            next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS));
+            Assert.assertTrue(next.equals(Arrays.asList(ConnectionState.LOST.name(), "release")) || next.equals(Arrays.asList("release", ConnectionState.LOST.name())), next.toString());
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(selector);
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
     public void testLeaderNodeDeleteOnInterrupt() throws Exception
     {
         Timing timing = new Timing();
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java
index d13fea7..0085968 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java
@@ -158,8 +158,8 @@
         final CuratorFramework client =
             CuratorFrameworkFactory.builder()
                 .connectString(server.getConnectString())
-                .retryPolicy(new RetryNTimes(2, 1))
-                .connectionTimeoutMs(100)
+                .retryPolicy(new RetryNTimes(2, 100))
+                .connectionTimeoutMs(1000)
                 .sessionTimeoutMs(60000)
                 .build();
         final CountDownLatch latch = new CountDownLatch(1);
@@ -211,7 +211,7 @@
             CuratorFrameworkFactory.builder()
                 .connectString(server.getConnectString())
                 .retryPolicy(new RetryNTimes(0, 0))
-                .connectionTimeoutMs(100)
+                .connectionTimeoutMs(1000)
                 .sessionTimeoutMs(60000)
                 .build();
         final CountDownLatch latch = new CountDownLatch(1);
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMultiMutex.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMultiMutex.java
index 09b5fe6..dceff88 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMultiMutex.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMultiMutex.java
@@ -20,6 +20,7 @@
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.retry.RetryOneTime;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -84,13 +85,14 @@
             }
             catch ( Exception e )
             {
+                // ignore
             }
             Assert.assertFalse(goodLock.isAcquiredInThisProcess());
             Assert.assertTrue(otherGoodLock.isAcquiredInThisProcess());
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -142,13 +144,14 @@
             }
             catch ( Exception e )
             {
+                // ignore
             }
             Assert.assertFalse(goodLock.isAcquiredInThisProcess());
             Assert.assertTrue(goodLockWasLocked.get());
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
index a2c079e..c37d88d 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutex.java
@@ -20,6 +20,7 @@
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.KillSession;
 import org.apache.zookeeper.CreateMode;
@@ -106,7 +107,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -151,7 +152,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
index 99ea11f..febf499 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java
@@ -22,6 +22,7 @@
 import com.google.common.collect.Lists;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
@@ -31,7 +32,6 @@
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.utils.ZKPaths;
-import org.apache.zookeeper.CreateMode;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 import java.util.List;
@@ -56,6 +56,28 @@
     protected abstract InterProcessLock makeLock(CuratorFramework client);
 
     @Test
+    public void testLocker() throws Exception
+    {
+        final Timing timing = new Timing();
+        final CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new ExponentialBackoffRetry(100, 3));
+        try
+        {
+            client.start();
+
+            InterProcessLock lock = makeLock(client);
+            try ( Locker locker = new Locker(lock, timing.milliseconds(), TimeUnit.MILLISECONDS) )
+            {
+                Assert.assertTrue(lock.isAcquiredInThisProcess());
+            }
+            Assert.assertFalse(lock.isAcquiredInThisProcess());
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
     public void testWaitingProcessKilledServer() throws Exception
     {
         final Timing timing = new Timing();
@@ -70,7 +92,7 @@
                 @Override
                 public void stateChanged(CuratorFramework client, ConnectionState newState)
                 {
-                    if ( newState == ConnectionState.LOST )
+                    if ( !newState.isConnected() )
                     {
                         latch.countDown();
                     }
@@ -79,6 +101,7 @@
             client.getConnectionStateListenable().addListener(listener);
 
             final AtomicBoolean isFirst = new AtomicBoolean(true);
+            final Object result = new Object();
             ExecutorCompletionService<Object> service = new ExecutorCompletionService<Object>(Executors.newFixedThreadPool(2));
             for ( int i = 0; i < 2; ++i )
             {
@@ -98,22 +121,15 @@
                                     timing.sleepABit();
 
                                     server.stop();
-                                    Assert.assertTrue(timing.awaitLatch(latch));
+                                    Assert.assertTrue(timing.forWaiting().awaitLatch(latch));
                                     server.restart();
                                 }
                             }
                             finally
                             {
-                                try
-                                {
-                                    lock.release();
-                                }
-                                catch ( Exception e )
-                                {
-                                    // ignore
-                                }
+                                lock.release();
                             }
-                            return null;
+                            return result;
                         }
                     }
                 );
@@ -121,12 +137,12 @@
 
             for ( int i = 0; i < 2; ++i )
             {
-                service.take().get(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS);
+                Assert.assertEquals(service.take().get(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), result);
             }
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -176,7 +192,7 @@
 
             Assert.assertTrue(timing.acquireSemaphore(semaphore, 1));
             KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString());
-            Assert.assertTrue(timing.acquireSemaphore(semaphore, 1));
+            Assert.assertTrue(timing.forSessionSleep().acquireSemaphore(semaphore, 1));
         }
         finally
         {
@@ -195,7 +211,7 @@
 
         server.close();
 
-        System.setProperty("container.checkIntervalMs", "10");
+        System.setProperty("znode.container.checkIntervalMs", "10");
         try
         {
             server = new TestingServer();
@@ -256,7 +272,7 @@
         }
         finally
         {
-            System.clearProperty("container.checkIntervalMs");
+            System.clearProperty("znode.container.checkIntervalMs");
         }
     }
 
@@ -278,7 +294,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -344,7 +360,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -390,7 +406,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -407,7 +423,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -539,11 +555,14 @@
                 Assert.assertTrue(acquiredLatchForClient1.await(10, TimeUnit.SECONDS));
                 Assert.assertTrue(mutexForClient1.isAcquiredInThisProcess());
             }
+
+            future1.get();
+            future2.get();
         }
         finally
         {
-            CloseableUtils.closeQuietly(client1);
-            CloseableUtils.closeQuietly(client2);
+            TestCleanState.closeAndTestClean(client1);
+            TestCleanState.closeAndTestClean(client2);
         }
     }
 }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessReadWriteLock.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessReadWriteLock.java
index f7636ed..48e4805 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessReadWriteLock.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessReadWriteLock.java
@@ -16,14 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework.recipes.locks;
 
 import com.google.common.collect.Lists;
-import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 import java.util.Collection;
@@ -31,6 +32,7 @@
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -40,21 +42,22 @@
 public class TestInterProcessReadWriteLock extends BaseClassForTests
 {
     @Test
-    public void     testGetParticipantNodes() throws Exception
+    public void testGetParticipantNodes() throws Exception
     {
-        final int               READERS = 20;
-        final int               WRITERS = 8;
+        final int READERS = 20;
+        final int WRITERS = 8;
 
-        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         try
         {
             client.start();
 
-            final CountDownLatch              latch = new CountDownLatch(READERS + WRITERS);
-            final CountDownLatch              readLatch = new CountDownLatch(READERS);
-            final InterProcessReadWriteLock   lock = new InterProcessReadWriteLock(client, "/lock");
+            final CountDownLatch latch = new CountDownLatch(READERS + WRITERS);
+            final CountDownLatch readLatch = new CountDownLatch(READERS);
+            final InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/lock");
 
-            ExecutorService                   service = Executors.newCachedThreadPool();
+            final CountDownLatch exitLatch = new CountDownLatch(1);
+            ExecutorCompletionService<Void> service = new ExecutorCompletionService<Void>(Executors.newCachedThreadPool());
             for ( int i = 0; i < READERS; ++i )
             {
                 service.submit
@@ -65,8 +68,16 @@
                         public Void call() throws Exception
                         {
                             lock.readLock().acquire();
-                            latch.countDown();
-                            readLatch.countDown();
+                            try
+                            {
+                                latch.countDown();
+                                readLatch.countDown();
+                                exitLatch.await();
+                            }
+                            finally
+                            {
+                                lock.readLock().release();
+                            }
                             return null;
                         }
                     }
@@ -84,6 +95,14 @@
                             Assert.assertTrue(readLatch.await(10, TimeUnit.SECONDS));
                             latch.countDown();  // must be before as there can only be one writer
                             lock.writeLock().acquire();
+                            try
+                            {
+                                exitLatch.await();
+                            }
+                            finally
+                            {
+                                lock.writeLock().release();
+                            }
                             return null;
                         }
                     }
@@ -97,22 +116,28 @@
 
             Assert.assertEquals(readers.size(), READERS);
             Assert.assertEquals(writers.size(), WRITERS);
+
+            exitLatch.countDown();
+            for ( int i = 0; i < (READERS + WRITERS); ++i )
+            {
+                service.take().get();
+            }
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
     @Test
-    public void     testThatUpgradingIsDisallowed() throws Exception
+    public void testThatUpgradingIsDisallowed() throws Exception
     {
-        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         try
         {
             client.start();
 
-            InterProcessReadWriteLock   lock = new InterProcessReadWriteLock(client, "/lock");
+            InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/lock");
             lock.readLock().acquire();
             Assert.assertFalse(lock.writeLock().acquire(5, TimeUnit.SECONDS));
 
@@ -120,70 +145,80 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
     @Test
-    public void     testThatDowngradingRespectsThreads() throws Exception
+    public void testThatDowngradingRespectsThreads() throws Exception
     {
-        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         try
         {
             client.start();
 
-            final InterProcessReadWriteLock   lock = new InterProcessReadWriteLock(client, "/lock");
-            ExecutorService                   t1 = Executors.newSingleThreadExecutor();
-            ExecutorService                   t2 = Executors.newSingleThreadExecutor();
+            final InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/lock");
+            ExecutorService t1 = Executors.newSingleThreadExecutor();
+            ExecutorService t2 = Executors.newSingleThreadExecutor();
 
-            final CountDownLatch              latch = new CountDownLatch(1);
+            final CountDownLatch latch = new CountDownLatch(1);
 
-            Future<Object>                    f1 = t1.submit
-            (
-                new Callable<Object>()
-                {
-                    @Override
-                    public Object call() throws Exception
+            final CountDownLatch releaseLatch = new CountDownLatch(1);
+            Future<Object> f1 = t1.submit
+                (
+                    new Callable<Object>()
                     {
-                        lock.writeLock().acquire();
-                        latch.countDown();
-                        return null;
+                        @Override
+                        public Object call() throws Exception
+                        {
+                            lock.writeLock().acquire();
+                            latch.countDown();
+                            try
+                            {
+                                releaseLatch.await();
+                            }
+                            finally
+                            {
+                                lock.writeLock().release();
+                            }
+                            return null;
+                        }
                     }
-                }
-            );
+                );
 
-            Future<Object>                    f2 = t2.submit
-            (
-                new Callable<Object>()
-                {
-                    @Override
-                    public Object call() throws Exception
+            Future<Object> f2 = t2.submit
+                (
+                    new Callable<Object>()
                     {
-                        Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
-                        Assert.assertFalse(lock.readLock().acquire(5, TimeUnit.SECONDS));
-                        return null;
+                        @Override
+                        public Object call() throws Exception
+                        {
+                            Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
+                            Assert.assertFalse(lock.readLock().acquire(5, TimeUnit.SECONDS));
+                            return null;
+                        }
                     }
-                }
-            );
+                );
 
-            f1.get();
             f2.get();
+            releaseLatch.countDown();
+            f1.get();
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
     @Test
-    public void     testDowngrading() throws Exception
+    public void testDowngrading() throws Exception
     {
-        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
         try
         {
             client.start();
 
-            InterProcessReadWriteLock   lock = new InterProcessReadWriteLock(client, "/lock");
+            InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/lock");
             lock.writeLock().acquire();
             Assert.assertTrue(lock.readLock().acquire(5, TimeUnit.SECONDS));
             lock.writeLock().release();
@@ -192,60 +227,60 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
     @Test
-    public void     testBasic() throws Exception
+    public void testBasic() throws Exception
     {
-        final int               CONCURRENCY = 8;
-        final int               ITERATIONS = 100;
+        final int CONCURRENCY = 8;
+        final int ITERATIONS = 100;
 
-        final Random            random = new Random();
-        final AtomicInteger     concurrentCount = new AtomicInteger(0);
-        final AtomicInteger     maxConcurrentCount = new AtomicInteger(0);
-        final AtomicInteger     writeCount = new AtomicInteger(0);
-        final AtomicInteger     readCount = new AtomicInteger(0);
+        final Random random = new Random();
+        final AtomicInteger concurrentCount = new AtomicInteger(0);
+        final AtomicInteger maxConcurrentCount = new AtomicInteger(0);
+        final AtomicInteger writeCount = new AtomicInteger(0);
+        final AtomicInteger readCount = new AtomicInteger(0);
 
-        List<Future<Void>>  futures = Lists.newArrayList();
-        ExecutorService     service = Executors.newCachedThreadPool();
+        List<Future<Void>> futures = Lists.newArrayList();
+        ExecutorService service = Executors.newCachedThreadPool();
         for ( int i = 0; i < CONCURRENCY; ++i )
         {
-            Future<Void>    future = service.submit
-            (
-                new Callable<Void>()
-                {
-                    @Override
-                    public Void call() throws Exception
+            Future<Void> future = service.submit
+                (
+                    new Callable<Void>()
                     {
-                        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
-                        client.start();
-                        try
+                        @Override
+                        public Void call() throws Exception
                         {
-                            InterProcessReadWriteLock   lock = new InterProcessReadWriteLock(client, "/lock");
-                            for ( int i = 0; i < ITERATIONS; ++i )
+                            CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+                            client.start();
+                            try
                             {
-                                if ( random.nextInt(100) < 10 )
+                                InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/lock");
+                                for ( int i = 0; i < ITERATIONS; ++i )
                                 {
-                                    doLocking(lock.writeLock(), concurrentCount, maxConcurrentCount, random, 1);
-                                    writeCount.incrementAndGet();
-                                }
-                                else
-                                {
-                                    doLocking(lock.readLock(), concurrentCount, maxConcurrentCount, random, Integer.MAX_VALUE);
-                                    readCount.incrementAndGet();
+                                    if ( random.nextInt(100) < 10 )
+                                    {
+                                        doLocking(lock.writeLock(), concurrentCount, maxConcurrentCount, random, 1);
+                                        writeCount.incrementAndGet();
+                                    }
+                                    else
+                                    {
+                                        doLocking(lock.readLock(), concurrentCount, maxConcurrentCount, random, Integer.MAX_VALUE);
+                                        readCount.incrementAndGet();
+                                    }
                                 }
                             }
+                            finally
+                            {
+                                TestCleanState.closeAndTestClean(client);
+                            }
+                            return null;
                         }
-                        finally
-                        {
-                            CloseableUtils.closeQuietly(client);
-                        }
-                        return null;
                     }
-                }
-            );
+                );
             futures.add(future);
         }
 
@@ -262,17 +297,17 @@
     }
 
     @Test
-    public void     testSetNodeData() throws Exception
+    public void testSetNodeData() throws Exception
     {
-        CuratorFramework        client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
 
         try
         {
             client.start();
 
-            final byte[] nodeData = new byte[] { 1, 2, 3, 4 };
+            final byte[] nodeData = new byte[]{1, 2, 3, 4};
 
-            InterProcessReadWriteLock   lock = new InterProcessReadWriteLock(client, "/lock", nodeData);
+            InterProcessReadWriteLock lock = new InterProcessReadWriteLock(client, "/lock", nodeData);
 
             // mutate passed-in node data, lock has made copy
             nodeData[0] = 5;
@@ -284,13 +319,13 @@
 
             byte dataInZk[] = client.getData().forPath("/lock/" + children.get(0));
             Assert.assertNotNull(dataInZk);
-            Assert.assertEquals(new byte[] { 1, 2, 3, 4 }, dataInZk);
+            Assert.assertEquals(new byte[]{1, 2, 3, 4}, dataInZk);
 
             lock.writeLock().release();
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -299,7 +334,7 @@
         try
         {
             Assert.assertTrue(lock.acquire(10, TimeUnit.SECONDS));
-            int     localConcurrentCount;
+            int localConcurrentCount;
             synchronized(this)
             {
                 localConcurrentCount = concurrentCount.incrementAndGet();
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphore.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphore.java
index 631b7c7..2797b5f 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphore.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessSemaphore.java
@@ -20,13 +20,14 @@
 package org.apache.curator.framework.recipes.locks;
 
 import com.google.common.collect.Lists;
-import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.utils.CloseableUtils;
 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.shared.SharedCount;
 import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.test.Timing;
+import org.apache.curator.utils.CloseableUtils;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 import java.util.Collection;
@@ -100,10 +101,12 @@
 
             future1.get();
             future2.get();
+
+            count.close();
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -140,8 +143,8 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client1);
-            CloseableUtils.closeQuietly(client2);
+            TestCleanState.closeAndTestClean(client1);
+            TestCleanState.closeAndTestClean(client2);
         }
     }
 
@@ -226,7 +229,7 @@
                                     }
                                     finally
                                     {
-                                        client.close();
+                                        TestCleanState.closeAndTestClean(client);
                                     }
                                     return null;
                                 }
@@ -299,7 +302,7 @@
                             }
                             finally
                             {
-                                client.close();
+                                TestCleanState.closeAndTestClean(client);
                             }
                             return null;
                         }
@@ -401,7 +404,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -445,7 +448,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -463,7 +466,7 @@
         }
         finally
         {
-            client.close();
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -499,7 +502,7 @@
             {
                 CloseableUtils.closeQuietly(l);
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -528,47 +531,7 @@
             {
                 CloseableUtils.closeQuietly(l);
             }
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
-
-    @Test
-    public void testChildReaperCleansUpLockNodes() throws Exception
-    {
-        Timing timing = new Timing();
-        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
-        client.start();
-
-        ChildReaper childReaper = null;
-        try
-        {
-            InterProcessSemaphoreV2 semaphore = new InterProcessSemaphoreV2(client, "/test/lock", 1);
-            semaphore.returnLease(semaphore.acquire(timing.forWaiting().seconds(), TimeUnit.SECONDS));
-
-            Assert.assertTrue(client.getChildren().forPath("/test").size() > 0);
-
-            childReaper = new ChildReaper(
-                    client,
-                    "/test",
-                    Reaper.Mode.REAP_UNTIL_GONE,
-                    ChildReaper.newExecutorService(),
-                    1,
-                    "/test-leader",
-                    InterProcessSemaphoreV2.LOCK_SCHEMA
-            );
-            childReaper.start();
-
-            timing.forWaiting().sleepABit();
-
-            List<String> children = client.getChildren().forPath("/test");
-
-            Assert.assertEquals(children.size(), 0, "All children of /test should have been reaped");
-        }
-        finally
-        {
-            CloseableUtils.closeQuietly(childReaper);
-            CloseableUtils.closeQuietly(client);
-        }
-
-    }
 }
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 2aa8a72..c06d042 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
@@ -22,6 +22,7 @@
 import org.apache.curator.ensemble.EnsembleProvider;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.ExponentialBackoffRetry;
@@ -64,6 +65,17 @@
             final EnsembleProvider          provider = new EnsembleProvider()
             {
                 @Override
+                public void setConnectionString(String connectionString)
+                {
+                }
+
+                @Override
+                public boolean updateServerListEnabled()
+                {
+                    return false;
+                }
+
+                @Override
                 public void start() throws Exception
                 {
                 }
@@ -147,7 +159,7 @@
                             }
                             finally
                             {
-                                CloseableUtils.closeQuietly(client);
+                                TestCleanState.closeAndTestClean(client);
                             }
                             return null;
                         }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java
index b6159cc..0fa95cf 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockACLs.java
@@ -19,6 +19,7 @@
 
 package org.apache.curator.framework.recipes.locks;
 
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.RetryPolicy;
@@ -74,7 +75,7 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockCleanlinessWithFaults.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockCleanlinessWithFaults.java
index 457be75..dc14c11 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockCleanlinessWithFaults.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestLockCleanlinessWithFaults.java
@@ -20,6 +20,7 @@
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.test.BaseClassForTests;
 import org.apache.curator.utils.CloseableUtils;
@@ -67,7 +68,7 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 }
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
index f451feb..15c5f2e 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
@@ -19,11 +19,13 @@
 package org.apache.curator.framework.recipes.nodes;
 
 import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
@@ -39,6 +41,8 @@
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
@@ -46,6 +50,7 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
@@ -55,6 +60,7 @@
 @SuppressWarnings("deprecation")
 public class TestPersistentEphemeralNode extends BaseClassForTests
 {
+    private static final Logger log = LoggerFactory.getLogger(TestPersistentEphemeralNode.class);
     private static final String DIR = "/test";
     private static final String PATH = ZKPaths.makePath(DIR, "/foo");
 
@@ -64,19 +70,25 @@
     private final Timing timing = new Timing();
 
     @AfterMethod
+    @Override
     public void teardown() throws Exception
     {
-        for ( PersistentEphemeralNode node : createdNodes )
+        try
         {
-            node.close();
-        }
+            for ( PersistentEphemeralNode node : createdNodes )
+            {
+                CloseableUtils.closeQuietly(node);
+            }
 
-        for ( CuratorFramework curator : curatorInstances )
+            for ( CuratorFramework curator : curatorInstances )
+            {
+                TestCleanState.closeAndTestClean(curator);
+            }
+        }
+        finally
         {
-            curator.close();
+            super.teardown();
         }
-
-        super.teardown();
     }
 
     @Test
@@ -88,41 +100,44 @@
         try
         {
             client.start();
-            PersistentEphemeralNode node = new PersistentEphemeralNode(client, PersistentEphemeralNode.Mode.EPHEMERAL, "/abc/node", "hello".getBytes());
-            node.start();
-
-            final CountDownLatch connectedLatch = new CountDownLatch(1);
-            final CountDownLatch reconnectedLatch = new CountDownLatch(1);
-            ConnectionStateListener listener = new ConnectionStateListener()
+            try ( PersistentEphemeralNode node = new PersistentEphemeralNode(client, PersistentEphemeralNode.Mode.EPHEMERAL, "/abc/node", "hello".getBytes()) )
             {
-                @Override
-                public void stateChanged(CuratorFramework client, ConnectionState newState)
+                node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
+                node.start();
+
+                final CountDownLatch connectedLatch = new CountDownLatch(1);
+                final CountDownLatch reconnectedLatch = new CountDownLatch(1);
+                ConnectionStateListener listener = new ConnectionStateListener()
                 {
-                    if ( newState == ConnectionState.CONNECTED )
+                    @Override
+                    public void stateChanged(CuratorFramework client, ConnectionState newState)
                     {
-                        connectedLatch.countDown();
+                        if ( newState == ConnectionState.CONNECTED )
+                        {
+                            connectedLatch.countDown();
+                        }
+                        if ( newState == ConnectionState.RECONNECTED )
+                        {
+                            reconnectedLatch.countDown();
+                        }
                     }
-                    if ( newState == ConnectionState.RECONNECTED )
-                    {
-                        reconnectedLatch.countDown();
-                    }
-                }
-            };
-            client.getConnectionStateListenable().addListener(listener);
-            timing.sleepABit();
-            server.restart();
-            Assert.assertTrue(timing.awaitLatch(connectedLatch));
-            timing.sleepABit();
-            Assert.assertTrue(node.waitForInitialCreate(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS));
-            server.stop();
-            timing.sleepABit();
-            server.restart();
-            timing.sleepABit();
-            Assert.assertTrue(timing.awaitLatch(reconnectedLatch));
+                };
+                client.getConnectionStateListenable().addListener(listener);
+                timing.sleepABit();
+                server.restart();
+                Assert.assertTrue(timing.awaitLatch(connectedLatch));
+                timing.sleepABit();
+                Assert.assertTrue(node.waitForInitialCreate(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS));
+                server.stop();
+                timing.sleepABit();
+                server.restart();
+                timing.sleepABit();
+                Assert.assertTrue(timing.awaitLatch(reconnectedLatch));
+            }
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -132,10 +147,12 @@
         server.stop();
 
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
+        PersistentEphemeralNode node = null;
         try
         {
             client.start();
-            PersistentEphemeralNode node = new PersistentEphemeralNode(client, PersistentEphemeralNode.Mode.EPHEMERAL, "/abc/node", "hello".getBytes());
+            node = new PersistentEphemeralNode(client, PersistentEphemeralNode.Mode.EPHEMERAL, "/abc/node", "hello".getBytes());
+            node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
             node.start();
 
             final CountDownLatch connectedLatch = new CountDownLatch(1);
@@ -164,7 +181,8 @@
         }
         finally
         {
-            CloseableUtils.closeQuietly(client);
+            CloseableUtils.closeQuietly(node);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -215,6 +233,7 @@
         {
             client.start();
             node = new PersistentEphemeralNode(client, mode, PATH, "a".getBytes());
+            node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
             node.start();
             Assert.assertTrue(node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS));
 
@@ -243,11 +262,8 @@
         }
         finally
         {
-            if ( node != null )
-            {
-                node.close();
-            }
-            client.close();
+            CloseableUtils.closeQuietly(node);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -257,6 +273,7 @@
         CuratorFramework curator = newCurator();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         node.start();
         String path = null;
         try
@@ -267,7 +284,7 @@
         }
         finally
         {
-            node.close();  // After closing the path is set to null...
+            CloseableUtils.closeQuietly(node);
         }
 
         assertNodeDoesNotExist(curator, path);
@@ -279,6 +296,7 @@
         CuratorFramework curator = newCurator();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         node.start();
         node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
 
@@ -297,24 +315,27 @@
         CuratorFramework observer = newCurator();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
-        node.start();
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         try
         {
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             assertNodeExists(observer, node.getActualPath());
 
             // Register a watch that will fire when the node is deleted...
-            Trigger deletedTrigger = Trigger.deleted();
+            Trigger deletedTrigger = Trigger.deletedOrSetData();
             observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
 
-            killSession(curator);
+            node.debugCreateNodeLatch = new CountDownLatch(1);
+            KillSession.kill(curator.getZookeeperClient().getZooKeeper());
 
             // Make sure the node got deleted
-            assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
+            assertTrue(deletedTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS));
+            node.debugCreateNodeLatch.countDown();
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
 
@@ -325,19 +346,22 @@
         CuratorFramework observer = newCurator();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
-        node.start();
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         try
         {
+            node.start();
             node.waitForInitialCreate(5, TimeUnit.SECONDS);
             assertNodeExists(observer, node.getActualPath());
 
-            Trigger deletedTrigger = Trigger.deleted();
+            Trigger deletedTrigger = Trigger.deletedOrSetData();
             observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
 
-            killSession(curator);
+            node.debugCreateNodeLatch = new CountDownLatch(1);
+            KillSession.kill(curator.getZookeeperClient().getZooKeeper());
 
             // Make sure the node got deleted...
-            assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
+            assertTrue(deletedTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS));
+            node.debugCreateNodeLatch.countDown();
 
             // Check for it to be recreated...
             Trigger createdTrigger = Trigger.created();
@@ -346,7 +370,7 @@
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
 
@@ -357,9 +381,10 @@
         CuratorFramework observer = newCurator();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
-        node.start();
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         try
         {
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             String path = node.getActualPath();
             assertNodeExists(observer, path);
@@ -367,24 +392,27 @@
             // We should be able to disconnect multiple times and each time the node should be recreated.
             for ( int i = 0; i < 5; i++ )
             {
-                Trigger deletionTrigger = Trigger.deleted();
-                observer.checkExists().usingWatcher(deletionTrigger).forPath(path);
+                Trigger deletionTrigger = Trigger.deletedOrSetData();
+                Stat stat = observer.checkExists().usingWatcher(deletionTrigger).forPath(path);
+                Assert.assertNotNull(stat, "node should exist: " + path);
 
+                node.debugCreateNodeLatch = new CountDownLatch(1);
                 // Kill the session, thus cleaning up the node...
-                killSession(curator);
+                KillSession.kill(curator.getZookeeperClient().getZooKeeper());
 
                 // Make sure the node ended up getting deleted...
-                assertTrue(deletionTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
+                assertTrue(deletionTrigger.firedWithin(timing.multiple(1.5).forSessionSleep().seconds(), TimeUnit.SECONDS));
+                node.debugCreateNodeLatch.countDown();
 
                 // Now put a watch in the background looking to see if it gets created...
                 Trigger creationTrigger = Trigger.created();
-                Stat stat = observer.checkExists().usingWatcher(creationTrigger).forPath(path);
+                stat = observer.checkExists().usingWatcher(creationTrigger).forPath(path);
                 assertTrue(stat != null || creationTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
             }
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
 
@@ -401,6 +429,7 @@
         observer.getData().usingWatcher(dataChangedTrigger).forPath(PATH);
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         node.start();
         try
         {
@@ -409,10 +438,10 @@
 
             assertTrue(dataChangedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
 
-            Trigger deletedTrigger = Trigger.deleted();
+            Trigger deletedTrigger = Trigger.deletedOrSetData();
             observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
 
-            killSession(nodeCreator);
+            KillSession.kill(nodeCreator.getZookeeperClient().getZooKeeper());
 
             // Make sure the node got deleted...
             assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
@@ -434,9 +463,10 @@
         CuratorFramework curator = newCurator();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
-        node.start();
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         try
         {
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             String originalNode = node.getActualPath();
             assertNodeExists(curator, originalNode);
@@ -452,44 +482,7 @@
         }
         finally
         {
-            node.close();
-        }
-    }
-
-    @Test
-    public void testRecreatesNodeWhenItGetsDeletedAfterSetData() throws Exception
-    {
-        CuratorFramework curator = newCurator();
-
-        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
-        node.start();
-        try
-        {
-            node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
-            String originalNode = node.getActualPath();
-            assertNodeExists(curator, originalNode);
-
-            Trigger dataChangedTrigger = Trigger.dataChanged();
-            curator.getData().usingWatcher(dataChangedTrigger).forPath(originalNode);
-
-            // update the data of the node
-            node.setData(new byte[0]);
-
-            // wait for the data to be updated:
-            assertTrue(dataChangedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
-
-            // Delete the original node...
-            curator.delete().forPath(originalNode);
-
-            // Since we're using an ephemeral node, and the original session hasn't been interrupted the name of the new
-            // node that gets created is going to be exactly the same as the original.
-            Trigger createdWatchTrigger = Trigger.created();
-            Stat stat = curator.checkExists().usingWatcher(createdWatchTrigger).forPath(originalNode);
-            assertTrue(stat != null || createdWatchTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS));
-        }
-        finally
-        {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
 
@@ -498,14 +491,15 @@
     {
         CuratorFramework curator = newCurator();
 
-        PersistentEphemeralNode node1 = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL_SEQUENTIAL, PATH, new byte[0]);
-        node1.start();
-        try
+        try ( PersistentEphemeralNode node1 = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL_SEQUENTIAL, PATH, new byte[0]) )
         {
+            node1.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
+            node1.start();
             node1.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             String path1 = node1.getActualPath();
 
             PersistentEphemeralNode node2 = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL_SEQUENTIAL, PATH, new byte[0]);
+            node2.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
             node2.start();
             try
             {
@@ -519,10 +513,6 @@
                 node2.close();
             }
         }
-        finally
-        {
-            node1.close();
-        }
     }
 
     @Test
@@ -532,15 +522,16 @@
         byte[] data = "Hello World".getBytes();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, data);
-        node.start();
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         try
         {
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), data));
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
     
@@ -558,18 +549,19 @@
         byte[] data = "Hello World".getBytes();
              
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, data);
-        node.start();
+        node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         try
         {
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), data));
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
-    
+
     @Test
     public void testSetDataWhenDisconnected() throws Exception
     {
@@ -579,9 +571,10 @@
         byte[] updatedData = "Updated".getBytes();
              
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, initialData);
-        node.start();
         try
         {
+            node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), initialData));
             
@@ -612,7 +605,7 @@
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }    	
     }
 
@@ -625,9 +618,10 @@
         byte[] updatedData = "Updated".getBytes();
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, initialData);
-        node.start();
         try
         {
+            node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), initialData));
 
@@ -651,7 +645,7 @@
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
     
@@ -668,9 +662,10 @@
 
         PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.PROTECTED_EPHEMERAL, PATH,
                                                                    new byte[0]);
-        node.start();
         try
         {
+            node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
+            node.start();
             node.waitForInitialCreate(timing.forWaiting().seconds(), TimeUnit.SECONDS);
             assertNodeExists(curator, node.getActualPath());
 
@@ -687,7 +682,7 @@
         }
         finally
         {
-            node.close();
+            CloseableUtils.closeQuietly(node);
         }
     }
     
@@ -700,31 +695,30 @@
             .authorization("digest", "me1:pass1".getBytes())
             .retryPolicy(new RetryOneTime(1))
             .build();
-        client.start();
-        
-        ACL acl = new ACL(ZooDefs.Perms.WRITE, ZooDefs.Ids.AUTH_IDS);
-        List<ACL> aclList = Lists.newArrayList(acl);
-        client.create().withACL(aclList).forPath(DIR, new byte[0]);
-        client.close();
-        
+
         PersistentEphemeralNode node = null;
         try {
+            client.start();
+
+            ACL acl = new ACL(ZooDefs.Perms.WRITE, ZooDefs.Ids.AUTH_IDS);
+            List<ACL> aclList = Lists.newArrayList(acl);
+            client.create().withACL(aclList).forPath(DIR, new byte[0]);
+            client.close();
+
         	//New client without authentication
         	client = newCurator();
         
         	node = new PersistentEphemeralNode(client, PersistentEphemeralNode.Mode.EPHEMERAL, PATH,
                                                                    new byte[0]);
+            node.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
         	node.start();
         
             node.waitForInitialCreate(timing.seconds(), TimeUnit.SECONDS);
             assertNodeDoesNotExist(client, PATH);
             assertTrue(node.isAuthFailure());
         } finally {
-        	if(node != null) {
-        	    node.close();
-        	}
-        	
-        	client.close();
+            CloseableUtils.closeQuietly(node);
+            CloseableUtils.closeQuietly(client);
         }
     }
 
@@ -748,31 +742,30 @@
         return client;
     }
 
-    public void killSession(CuratorFramework curator) throws Exception
-    {
-        KillSession.kill(curator.getZookeeperClient().getZooKeeper(), curator.getZookeeperClient().getCurrentConnectionString());
-    }
-
     private static final class Trigger implements Watcher
     {
-        private final Event.EventType type;
+        private final Set<EventType> types;
         private final CountDownLatch latch;
 
-        public Trigger(Event.EventType type)
+        public Trigger(Event.EventType... types)
         {
-            assertNotNull(type);
+            assertNotNull(types);
 
-            this.type = type;
+            this.types = ImmutableSet.copyOf(types);
             this.latch = new CountDownLatch(1);
         }
 
         @Override
         public void process(WatchedEvent event)
         {
-            if ( type == event.getType() )
+            if ( types.contains(event.getType()) )
             {
                 latch.countDown();
             }
+            else if ( event.getType() != EventType.None )
+            {
+                log.warn("Unexpected watcher event: " + event);
+            }
         }
 
         public boolean firedWithin(long duration, TimeUnit unit)
@@ -783,6 +776,7 @@
             }
             catch ( InterruptedException e )
             {
+                Thread.currentThread().interrupt();
                 throw Throwables.propagate(e);
             }
         }
@@ -792,9 +786,9 @@
             return new Trigger(Event.EventType.NodeCreated);
         }
 
-        private static Trigger deleted()
+        private static Trigger deletedOrSetData()
         {
-            return new Trigger(Event.EventType.NodeDeleted);
+            return new Trigger(Event.EventType.NodeDeleted, EventType.NodeDataChanged);
         }
 
         private static Trigger dataChanged()
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
index c006dd7..20d6916 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentNode.java
@@ -43,6 +43,7 @@
         {
             client.start();
             pen = new PersistentNode(client, CreateMode.PERSISTENT, false, "/test", TEST_DATA);
+            pen.debugWaitMsForBackgroundBeforeClose.set(timing.forSleepingABit().milliseconds());
             pen.start();
             Assert.assertTrue(pen.waitForInitialCreate(timing.milliseconds(), TimeUnit.MILLISECONDS));
             client.close(); // cause session to end - force checks that node is persistent
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedDelayQueue.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedDelayQueue.java
index 74cd2ee..a76f449 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedDelayQueue.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedDelayQueue.java
@@ -138,10 +138,13 @@
     @Test
     public void testSorting() throws Exception
     {
+        Timing                          timing = new Timing();
+
         //Need to use a fairly large number to ensure that sorting can take some time.
         final int QTY = 1000;
 
-        Timing                          timing = new Timing();
+        final int DELAY_MS = timing.multiple(.1).milliseconds();
+
         DistributedDelayQueue<Long>     putQueue = null;
         DistributedDelayQueue<Long>     getQueue = null;
         CuratorFramework                client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
@@ -157,7 +160,7 @@
             //been added prior to the consumption starting. Otherwise it's possible to start
             //processing entries before they've all been added so the ordering will be 
             //incorrect.
-            long delay = System.currentTimeMillis() + 5000;            
+            long delay = System.currentTimeMillis() + DELAY_MS;
             for ( long i = 0; i < QTY; ++i )
             {
                 data.put(delay, i);
@@ -184,7 +187,7 @@
             long lastValue = -1;
             for ( int i = 0; i < QTY; ++i )
             {
-                Long value = consumer.take(6, TimeUnit.SECONDS);
+                Long value = consumer.take(DELAY_MS * 2, TimeUnit.MILLISECONDS);
                 Assert.assertNotNull(value);
                 Assert.assertEquals(value, new Long(lastValue + 1));
                 lastValue = value;
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
index a191166..2f0f9a8 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
@@ -175,7 +175,7 @@
                 "/lock",
                 QueueBuilder.NOT_SET,
                 true,
-                5000
+                timing.milliseconds()
             )
             {
                 @SuppressWarnings("SimplifiableConditionalExpression")
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
index 659154a..2bdd278 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/shared/TestSharedCount.java
@@ -23,6 +23,7 @@
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
@@ -147,7 +148,7 @@
             }
             for ( CuratorFramework client : clients )
             {
-                CloseableUtils.closeQuietly(client);
+                TestCleanState.closeAndTestClean(client);
             }
         }
     }
@@ -170,7 +171,7 @@
         finally
         {
             CloseableUtils.closeQuietly(count);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -215,7 +216,7 @@
         finally
         {
             CloseableUtils.closeQuietly(count);
-            CloseableUtils.closeQuietly(client);
+            TestCleanState.closeAndTestClean(client);
         }
     }
 
@@ -252,8 +253,8 @@
         {
             CloseableUtils.closeQuietly(count2);
             CloseableUtils.closeQuietly(count1);
-            CloseableUtils.closeQuietly(client2);
-            CloseableUtils.closeQuietly(client1);
+            TestCleanState.closeAndTestClean(client2);
+            TestCleanState.closeAndTestClean(client1);
         }
     }
 }
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 2f5455d..8fcffda 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-test</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
 
     <name>Curator Testing</name>
     <description>Unit testing utilities.</description>
@@ -41,11 +41,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.javassist</groupId>
-            <artifactId>javassist</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-math</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 13c3138..a5afaf2 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
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.test;
 
-import org.testng.IRetryAnalyzer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.IInvokedMethod;
+import org.testng.IInvokedMethodListener;
 import org.testng.ITestContext;
+import org.testng.ITestNGListener;
 import org.testng.ITestNGMethod;
 import org.testng.ITestResult;
 import org.testng.annotations.AfterMethod;
@@ -27,40 +32,82 @@
 import org.testng.annotations.BeforeSuite;
 import java.io.IOException;
 import java.net.BindException;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 public class BaseClassForTests
 {
     protected TestingServer server;
+    private final Logger log = LoggerFactory.getLogger(getClass());
 
-    private static final int    RETRY_WAIT_MS = 5000;
+    private static final int RETRY_WAIT_MS = 5000;
     private static final String INTERNAL_PROPERTY_DONT_LOG_CONNECTION_ISSUES;
-    private static final String INTERNAL_RETRY_FAILED_TESTS;
+    private static final String INTERNAL_PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND;
+    private static final String INTERNAL_PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY;
+
     static
     {
         String logConnectionIssues = null;
-        String retryFailedTests = null;
         try
         {
             // use reflection to avoid adding a circular dependency in the pom
             Class<?> debugUtilsClazz = Class.forName("org.apache.curator.utils.DebugUtils");
             logConnectionIssues = (String)debugUtilsClazz.getField("PROPERTY_DONT_LOG_CONNECTION_ISSUES").get(null);
-            retryFailedTests = (String)debugUtilsClazz.getField("PROPERTY_RETRY_FAILED_TESTS").get(null);
         }
         catch ( Exception e )
         {
             e.printStackTrace();
         }
         INTERNAL_PROPERTY_DONT_LOG_CONNECTION_ISSUES = logConnectionIssues;
-        INTERNAL_RETRY_FAILED_TESTS = retryFailedTests;
+        String s = null;
+        try
+        {
+            // use reflection to avoid adding a circular dependency in the pom
+            s = (String)Class.forName("org.apache.curator.utils.DebugUtils").getField("PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND").get(null);
+        }
+        catch ( Exception e )
+        {
+            e.printStackTrace();
+        }
+        INTERNAL_PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND = s;
+        s = null;
+        try
+        {
+            // use reflection to avoid adding a circular dependency in the pom
+            s = (String)Class.forName("org.apache.curator.utils.DebugUtils").getField("PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY").get(null);
+        }
+        catch ( Exception e )
+        {
+            e.printStackTrace();
+        }
+        INTERNAL_PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY = s;
     }
 
     @BeforeSuite(alwaysRun = true)
     public void beforeSuite(ITestContext context)
     {
+        if ( !enabledSessionExpiredStateAware() )
+        {
+            ITestNGListener listener = new IInvokedMethodListener()
+            {
+                @Override
+                public void beforeInvocation(IInvokedMethod method, ITestResult testResult)
+                {
+                    int invocationCount = method.getTestMethod().getCurrentInvocationCount();
+                    System.setProperty("curator-use-classic-connection-handling", Boolean.toString(invocationCount == 1));
+                    log.info("curator-use-classic-connection-handling: " + Boolean.toString(invocationCount == 1));
+                }
+
+                @Override
+                public void afterInvocation(IInvokedMethod method, ITestResult testResult)
+                {
+                    System.clearProperty("curator-use-classic-connection-handling");
+                }
+            };
+            context.getSuite().addListener(listener);
+        }
+
         for ( ITestNGMethod method : context.getAllTestMethods() )
         {
-            method.setRetryAnalyzer(new RetryTest());
+            method.setInvocationCount(enabledSessionExpiredStateAware() ? 1 : 2);
         }
     }
 
@@ -71,6 +118,8 @@
         {
             System.setProperty(INTERNAL_PROPERTY_DONT_LOG_CONNECTION_ISSUES, "true");
         }
+        System.setProperty(INTERNAL_PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND, "true");
+        System.setProperty(INTERNAL_PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY, "true");
 
         while ( server == null )
         {
@@ -89,6 +138,8 @@
     @AfterMethod
     public void teardown() throws Exception
     {
+        System.clearProperty(INTERNAL_PROPERTY_VALIDATE_NAMESPACE_WATCHER_MAP_EMPTY);
+        System.clearProperty(INTERNAL_PROPERTY_REMOVE_WATCHERS_IN_FOREGROUND);
         if ( server != null )
         {
             try
@@ -99,34 +150,15 @@
             {
                 e.printStackTrace();
             }
-            server = null;
-        }
-    }
-
-    private static class RetryTest implements IRetryAnalyzer
-    {
-        private final AtomicBoolean hasBeenRetried = new AtomicBoolean(!Boolean.getBoolean(INTERNAL_RETRY_FAILED_TESTS));
-
-        @Override
-        public boolean retry(ITestResult result)
-        {
-            boolean isRetrying = hasBeenRetried.compareAndSet(false, true);
-            if ( isRetrying )
+            finally
             {
-                System.err.println(String.format("Waiting " + RETRY_WAIT_MS + " ms and retrying test. Name: %s - TestName: %s ", result.getName(), result.getTestName()));
-                try
-                {
-                    Thread.sleep(RETRY_WAIT_MS);
-                }
-                catch ( InterruptedException e )
-                {
-                    System.err.println(String.format("Retry interrupted. Name: %s - TestName: %s ", result.getName(), result.getTestName()));
-                    Thread.currentThread().interrupt();
-                    isRetrying = false;
-                }
+                server = null;
             }
-            return isRetrying;
         }
     }
 
+    protected boolean enabledSessionExpiredStateAware()
+    {
+        return false;
+    }
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java b/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java
deleted file mode 100644
index eeca3d4..0000000
--- a/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java
+++ /dev/null
@@ -1,97 +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.test;
-
-import javassist.CannotCompileException;
-import javassist.ClassPool;
-import javassist.CtClass;
-import javassist.CtMethod;
-import javassist.NotFoundException;
-
-public class ByteCodeRewrite
-{
-    public static void apply()
-    {
-        // NOP - only needed so that static initializer is run
-    }
-
-    static
-    {
-        /*
-            This ugliness is necessary. There is no way to tell ZK to not register JMX beans. Something
-            in the shutdown of a QuorumPeer causes the state of the MBeanRegistry to get confused and
-            generates an assert Exception.
-         */
-        ClassPool pool = ClassPool.getDefault();
-        try
-        {
-            pool.appendClassPath(new javassist.LoaderClassPath(TestingCluster.class.getClassLoader()));     // re: https://github.com/Netflix/curator/issues/11
-
-            try
-            {
-                CtClass cc = pool.get("org.apache.zookeeper.server.ZooKeeperServer");
-                fixMethods(cc, "registerJMX", "unregisterJMX");
-            }
-            catch ( NotFoundException ignore )
-            {
-                // ignore
-            }
-
-            try
-            {
-                CtClass cc = pool.get("org.apache.zookeeper.server.quorum.LearnerZooKeeperServer");
-                fixMethods(cc, "registerJMX", "unregisterJMX");
-            }
-            catch ( NotFoundException ignore )
-            {
-                // ignore
-            }
-
-            try
-            {
-                CtClass cc = pool.get("org.apache.zookeeper.jmx.MBeanRegistry");
-                fixMethods(cc, "register", "unregister");
-            }
-            catch ( NotFoundException ignore )
-            {
-                // ignore
-            }
-        }
-        catch ( Exception e )
-        {
-            e.printStackTrace();
-        }
-    }
-
-    private static void fixMethods(CtClass cc, String... methodNames) throws CannotCompileException
-    {
-        for ( CtMethod method : cc.getDeclaredMethods() )
-        {
-            for ( String methodName : methodNames )
-            {
-                if ( method.getName().equals(methodName) )
-                {
-                    method.setBody(null);
-                }
-            }
-        }
-        cc.toClass();
-    }
-}
diff --git a/curator-test/src/main/java/org/apache/curator/test/InstanceSpec.java b/curator-test/src/main/java/org/apache/curator/test/InstanceSpec.java
index b39a949..6d495df 100644
--- a/curator-test/src/main/java/org/apache/curator/test/InstanceSpec.java
+++ b/curator-test/src/main/java/org/apache/curator/test/InstanceSpec.java
@@ -70,6 +70,10 @@
     private final int tickTime;
     private final int maxClientCnxns;
 
+    public static void reset() {
+        nextServerId.set(1);
+    }
+
     public static InstanceSpec newInstanceSpec()
     {
         return new InstanceSpec(null, -1, -1, -1, true, -1, -1, -1);
diff --git a/curator-test/src/main/java/org/apache/curator/test/KillSession.java b/curator-test/src/main/java/org/apache/curator/test/KillSession.java
index 63b7f2a..ce2b7e6 100644
--- a/curator-test/src/main/java/org/apache/curator/test/KillSession.java
+++ b/curator-test/src/main/java/org/apache/curator/test/KillSession.java
@@ -18,24 +18,12 @@
  */
 package org.apache.curator.test;
 
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 
 /**
  * <p>
- *     Utility to simulate a ZK session dying. See: <a href="http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A4">ZooKeeper FAQ</a>
+ *     Utility to simulate a ZK session dying.
  * </p>
- *
- * <blockquote>
- *     In the case of testing we want to cause a problem, so to explicitly expire a session an
- *     application connects to ZooKeeper, saves the session id and password, creates another
- *     ZooKeeper handle with that id and password, and then closes the new handle. Since both
- *     handles reference the same session, the close on second handle will invalidate the session
- *     causing a SESSION_EXPIRED on the first handle.
- * </blockquote>
  */
 public class KillSession
 {
@@ -43,12 +31,24 @@
      * Kill the given ZK session
      *
      * @param client the client to kill
+     * @since 3.0.0
+     */
+    public static void     kill(ZooKeeper client)
+    {
+        client.getTestable().injectSessionExpiration();
+    }
+
+    /**
+     * Kill the given ZK session
+     *
+     * @param client the client to kill
      * @param connectString server connection string
      * @throws Exception errors
+     * @deprecated use {@link #kill(ZooKeeper)} instead
      */
     public static void     kill(ZooKeeper client, String connectString) throws Exception
     {
-        kill(client, connectString, new Timing().forWaiting().milliseconds());
+        kill(client);
     }
 
     /**
@@ -58,65 +58,10 @@
      * @param connectString server connection string
      * @param maxMs max time ms to wait for kill
      * @throws Exception errors
+     * @deprecated use {@link #kill(ZooKeeper)} instead
      */
     public static void     kill(ZooKeeper client, String connectString, int maxMs) throws Exception
     {
-        long                    startTicks = System.currentTimeMillis();
-
-        final CountDownLatch    sessionLostLatch = new CountDownLatch(1);
-        Watcher                 sessionLostWatch = new Watcher()
-        {
-            @Override
-            public void process(WatchedEvent event)
-            {
-                sessionLostLatch.countDown();
-            }
-        };
-        client.exists("/___CURATOR_KILL_SESSION___" + System.nanoTime(), sessionLostWatch);
-
-        final CountDownLatch    connectionLatch = new CountDownLatch(1);
-        Watcher                 connectionWatcher = new Watcher()
-        {
-            @Override
-            public void process(WatchedEvent event)
-            {
-                if ( event.getState() == Event.KeeperState.SyncConnected )
-                {
-                    connectionLatch.countDown();
-                }
-            }
-        };
-        ZooKeeper zk = new ZooKeeper(connectString, maxMs, connectionWatcher, client.getSessionId(), client.getSessionPasswd());
-        try
-        {
-            if ( !connectionLatch.await(maxMs, TimeUnit.MILLISECONDS) )
-            {
-                throw new Exception("KillSession could not establish duplicate session");
-            }
-            try
-            {
-                zk.close();
-            }
-            finally
-            {
-                zk = null;
-            }
-
-            while ( client.getState().isConnected() && !sessionLostLatch.await(100, TimeUnit.MILLISECONDS) )
-            {
-                long        elapsed = System.currentTimeMillis() - startTicks;
-                if ( elapsed > maxMs )
-                {
-                    throw new Exception("KillSession timed out waiting for session to expire");
-                }
-            }
-        }
-        finally
-        {
-            if ( zk != null )
-            {
-                zk.close();
-            }
-        }
+        kill(client);
     }
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/QuorumConfigBuilder.java b/curator-test/src/main/java/org/apache/curator/test/QuorumConfigBuilder.java
index 8add08e..17bb185 100644
--- a/curator-test/src/main/java/org/apache/curator/test/QuorumConfigBuilder.java
+++ b/curator-test/src/main/java/org/apache/curator/test/QuorumConfigBuilder.java
@@ -19,19 +19,23 @@
 
 package org.apache.curator.test;
 
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.io.Files;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import java.io.Closeable;
 import java.io.File;
+import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 
 @SuppressWarnings("UnusedDeclaration")
-public class QuorumConfigBuilder
+public class QuorumConfigBuilder implements Closeable
 {
     private final ImmutableList<InstanceSpec> instanceSpecs;
     private final boolean fromRandom;
+    private final File fakeConfigFile;
 
     public QuorumConfigBuilder(Collection<InstanceSpec> specs)
     {
@@ -42,6 +46,16 @@
     {
         fromRandom = (specs == null) || (specs.length == 0);
         instanceSpecs = fromRandom ? ImmutableList.of(InstanceSpec.newInstanceSpec()) : ImmutableList.copyOf(specs);
+        File fakeConfigFile = null;
+        try
+        {
+            fakeConfigFile = File.createTempFile("temp", "temp");
+        }
+        catch ( IOException e )
+        {
+            Throwables.propagate(e);
+        }
+        this.fakeConfigFile = fakeConfigFile;
     }
 
     public boolean isFromRandom()
@@ -69,6 +83,16 @@
         return instanceSpecs.size();
     }
 
+    @Override
+    public void close()
+    {
+        if ( fakeConfigFile != null )
+        {
+            //noinspection ResultOfMethodCallIgnored
+            fakeConfigFile.delete();
+        }
+    }
+
     public QuorumPeerConfig buildConfig(int instanceIndex) throws Exception
     {
         boolean isCluster = (instanceSpecs.size() > 1);
@@ -99,11 +123,19 @@
         {
             for ( InstanceSpec thisSpec : instanceSpecs )
             {
-                properties.setProperty("server." + thisSpec.getServerId(), String.format("localhost:%d:%d", thisSpec.getQuorumPort(), thisSpec.getElectionPort()));
+                properties.setProperty("server." + thisSpec.getServerId(), String.format("localhost:%d:%d;localhost:%d", thisSpec.getQuorumPort(), thisSpec.getElectionPort(), thisSpec.getPort()));
             }
         }
 
-        QuorumPeerConfig config = new QuorumPeerConfig();
+        QuorumPeerConfig config = new QuorumPeerConfig()
+        {
+            {
+                if ( fakeConfigFile != null )
+                {
+                    configFileStr = fakeConfigFile.getPath();
+                }
+            }
+        };
         config.parseProperties(properties);
         return config;
     }
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 cd86b72..3d38fe1 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
@@ -38,11 +38,6 @@
  */
 public class TestingCluster implements Closeable
 {
-    static
-    {
-        ByteCodeRewrite.apply();
-    }
-
     private final List<TestingZooKeeperServer>  servers;
 
     /**
@@ -104,17 +99,17 @@
     public Collection<InstanceSpec> getInstances()
     {
         Iterable<InstanceSpec> transformed = Iterables.transform
-        (
-            servers,
-            new Function<TestingZooKeeperServer, InstanceSpec>()
-            {
-                @Override
-                public InstanceSpec apply(TestingZooKeeperServer server)
+            (
+                servers,
+                new Function<TestingZooKeeperServer, InstanceSpec>()
                 {
-                    return server.getInstanceSpec();
+                    @Override
+                    public InstanceSpec apply(TestingZooKeeperServer server)
+                    {
+                        return server.getInstanceSpec();
+                    }
                 }
-            }
-        );
+            );
         return Lists.newArrayList(transformed);
     }
 
@@ -247,8 +242,17 @@
         return null;
     }
 
-    private static Map<InstanceSpec, Collection<InstanceSpec>> makeSpecs(int instanceQty)
+    public static Map<InstanceSpec, Collection<InstanceSpec>> makeSpecs(int instanceQty)
     {
+        return makeSpecs(instanceQty, true);
+    }
+
+    public static Map<InstanceSpec, Collection<InstanceSpec>> makeSpecs(int instanceQty, boolean resetServerIds)
+    {
+        if ( resetServerIds )
+        {
+            InstanceSpec.reset();
+        }
         ImmutableList.Builder<InstanceSpec> builder = ImmutableList.builder();
         for ( int i = 0; i < instanceQty; ++i )
         {
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 9f9f302..e1c1f89 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
@@ -63,18 +63,26 @@
     {
         if ( quorumPeer != null )
         {
-            quorumPeer.shutdown();
+            try
+            {
+                quorumPeer.shutdown();
+            }
+            finally
+            {
+                quorumPeer = null;
+            }
         }
     }
 
     @Override
     public void blockUntilStarted() throws Exception
     {
-        while ( quorumPeer == null )
+        long startTime = System.currentTimeMillis();
+        while ( (quorumPeer == null) && ((System.currentTimeMillis() - startTime) <= TestingZooKeeperMain.MAX_WAIT_MS) )
         {
             try
             {
-                Thread.sleep(100);
+                Thread.sleep(10);
             }
             catch ( InterruptedException e )
             {
@@ -82,5 +90,9 @@
                 break;
             }
         }
+        if ( quorumPeer == null )
+        {
+            throw new Exception("quorumPeer never got set");
+        }
     }
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingServer.java b/curator-test/src/main/java/org/apache/curator/test/TestingServer.java
index 7ed8add..5228c8e 100644
--- a/curator-test/src/main/java/org/apache/curator/test/TestingServer.java
+++ b/curator-test/src/main/java/org/apache/curator/test/TestingServer.java
@@ -28,11 +28,6 @@
  */
 public class TestingServer implements Closeable
 {
-    static
-    {
-        ByteCodeRewrite.apply();
-    }
-
     private final TestingZooKeeperServer testingZooKeeperServer;
     private final InstanceSpec spec;
 
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 7487557..19535a6 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
@@ -19,28 +19,44 @@
 
 package org.apache.curator.test;
 
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.jmx.ZKMBeanInfo;
+import org.apache.zookeeper.server.ContainerManager;
+import org.apache.zookeeper.server.RequestProcessor;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ServerConfig;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZooKeeperServer;
-import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import javax.management.JMException;
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.channels.ServerSocketChannel;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class TestingZooKeeperMain extends ZooKeeperServerMain implements ZooKeeperMainFace
+public class TestingZooKeeperMain implements ZooKeeperMainFace
 {
+    private static final Logger log = LoggerFactory.getLogger(TestingZooKeeperMain.class);
+
     private final CountDownLatch latch = new CountDownLatch(1);
     private final AtomicReference<Exception> startingException = new AtomicReference<Exception>(null);
 
-    private static final int MAX_WAIT_MS;
+    private volatile ServerCnxnFactory cnxnFactory;
+    private volatile TestZooKeeperServer zkServer;
+    private volatile ContainerManager containerManager;
 
+    private static final Timing timing = new Timing();
+
+    static final int MAX_WAIT_MS;
     static
     {
         long startMs = System.currentTimeMillis();
@@ -64,15 +80,15 @@
     {
         try
         {
-            Field cnxnFactoryField = ZooKeeperServerMain.class.getDeclaredField("cnxnFactory");
-            cnxnFactoryField.setAccessible(true);
-            ServerCnxnFactory cnxnFactory = (ServerCnxnFactory)cnxnFactoryField.get(this);
-            cnxnFactory.closeAll();
+            if ( cnxnFactory != null )
+            {
+                cnxnFactory.closeAll();
 
-            Field ssField = cnxnFactory.getClass().getDeclaredField("ss");
-            ssField.setAccessible(true);
-            ServerSocketChannel ss = (ServerSocketChannel)ssField.get(cnxnFactory);
-            ss.close();
+                Field ssField = cnxnFactory.getClass().getDeclaredField("ss");
+                ssField.setAccessible(true);
+                ServerSocketChannel ss = (ServerSocketChannel)ssField.get(cnxnFactory);
+                ss.close();
+            }
 
             close();
         }
@@ -85,12 +101,36 @@
     @Override
     public void runFromConfig(QuorumPeerConfig config) throws Exception
     {
-        ServerConfig serverConfig = new ServerConfig();
-        serverConfig.readFrom(config);
-        latch.countDown();
         try
         {
-            super.runFromConfig(serverConfig);
+            Field instance = MBeanRegistry.class.getDeclaredField("instance");
+            instance.setAccessible(true);
+            MBeanRegistry nopMBeanRegistry = new MBeanRegistry()
+            {
+                @Override
+                public void register(ZKMBeanInfo bean, ZKMBeanInfo parent) throws JMException
+                {
+                    // NOP
+                }
+
+                @Override
+                public void unregister(ZKMBeanInfo bean)
+                {
+                    // NOP
+                }
+            };
+            instance.set(null, nopMBeanRegistry);
+        }
+        catch ( Exception e )
+        {
+            log.error("Could not fix MBeanRegistry");
+        }
+
+        ServerConfig serverConfig = new ServerConfig();
+        serverConfig.readFrom(config);
+        try
+        {
+            internalRunFromConfig(serverConfig);
         }
         catch ( IOException e )
         {
@@ -109,25 +149,24 @@
     @Override
     public void blockUntilStarted() throws Exception
     {
-        latch.await();
+        if(!timing.awaitLatch(latch))
+            throw new IllegalStateException("Timed out waiting for watch removal");
 
-        ServerCnxnFactory cnxnFactory = getServerConnectionFactory();
-        if ( cnxnFactory != null )
+        if ( zkServer != null )
         {
-            final ZooKeeperServer zkServer = getZooKeeperServer(cnxnFactory);
-            if ( zkServer != null )
+            //noinspection SynchronizeOnNonFinalField
+            synchronized(zkServer)
             {
-                synchronized(zkServer)
+                while ( !zkServer.isRunning() )
                 {
-                    if ( !zkServer.isRunning() )
-                    {
-                        zkServer.wait();
-                    }
+                    zkServer.wait();
                 }
             }
         }
-
-        Thread.sleep(1000);
+        else
+        {
+            throw new Exception("No zkServer.");
+        }
 
         Exception exception = startingException.get();
         if ( exception != null )
@@ -141,67 +180,128 @@
     {
         try
         {
-            shutdown();
+            cnxnFactory.shutdown();
         }
         catch ( Throwable e )
         {
             e.printStackTrace();    // just ignore - this class is only for testing
         }
+        finally
+        {
+            cnxnFactory = null;
+        }
+
+        if ( containerManager != null ) {
+            containerManager.stop();
+            containerManager = null;
+        }
 
         try
         {
-            ServerCnxnFactory cnxnFactory = getServerConnectionFactory();
-            if ( cnxnFactory != null )
+            if ( zkServer != null )
             {
-                ZooKeeperServer zkServer = getZooKeeperServer(cnxnFactory);
-                if ( zkServer != null )
+                zkServer.shutdown();
+                ZKDatabase zkDb = zkServer.getZKDatabase();
+                if ( zkDb != null )
                 {
-                    ZKDatabase zkDb = zkServer.getZKDatabase();
-                    if ( zkDb != null )
-                    {
-                        // make ZK server close its log files
-                        zkDb.close();
-                    }
+                    // make ZK server close its log files
+                    zkDb.close();
                 }
             }
         }
-        catch ( Exception e )
+        catch ( Throwable e )
         {
             e.printStackTrace();    // just ignore - this class is only for testing
         }
+        finally
+        {
+            zkServer = null;
+        }
     }
 
-    private ServerCnxnFactory getServerConnectionFactory() throws Exception
+    // copied from ZooKeeperServerMain.java
+    private void internalRunFromConfig(ServerConfig config) throws IOException
     {
-        Field cnxnFactoryField = ZooKeeperServerMain.class.getDeclaredField("cnxnFactory");
-        cnxnFactoryField.setAccessible(true);
-        ServerCnxnFactory cnxnFactory;
+        log.info("Starting server");
+        FileTxnSnapLog txnLog = null;
+        try {
+            // Note that this thread isn't going to be doing anything else,
+            // so rather than spawning another thread, we will just call
+            // run() in this thread.
+            // create a file logger url from the command line args
+            txnLog = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir());
+            zkServer = new TestZooKeeperServer(txnLog, config);
 
-        // Wait until the cnxnFactory field is non-null or up to 1s, whichever comes first.
-        long startTime = System.currentTimeMillis();
-        do
-        {
-            cnxnFactory = (ServerCnxnFactory)cnxnFactoryField.get(this);
+            try
+            {
+                cnxnFactory = ServerCnxnFactory.createFactory();
+                cnxnFactory.configure(config.getClientPortAddress(),
+                    config.getMaxClientCnxns());
+            }
+            catch ( IOException e )
+            {
+                log.info("Could not server. Waiting and trying one more time.", e);
+                timing.sleepABit();
+                cnxnFactory = ServerCnxnFactory.createFactory();
+                cnxnFactory.configure(config.getClientPortAddress(),
+                    config.getMaxClientCnxns());
+            }
+            cnxnFactory.startup(zkServer);
+            containerManager = new ContainerManager(zkServer.getZKDatabase(), zkServer.getFirstProcessor(), Integer.getInteger("znode.container.checkIntervalMs", (int)TimeUnit.MINUTES.toMillis(1L)), Integer.getInteger("znode.container.maxPerMinute", 10000));
+            containerManager.start();
+            latch.countDown();
+            cnxnFactory.join();
+            if ( (zkServer != null) && zkServer.isRunning()) {
+                zkServer.shutdown();
+            }
+        } catch (InterruptedException e) {
+            // warn, but generally this is ok
+            Thread.currentThread().interrupt();
+            log.warn("Server interrupted", e);
+        } finally {
+            if (txnLog != null) {
+                txnLog.close();
+            }
         }
-        while ( (cnxnFactory == null) && ((System.currentTimeMillis() - startTime) < MAX_WAIT_MS) );
-
-        return cnxnFactory;
     }
 
-    private ZooKeeperServer getZooKeeperServer(ServerCnxnFactory cnxnFactory) throws Exception
+    public static class TestZooKeeperServer extends ZooKeeperServer
     {
-        Field zkServerField = ServerCnxnFactory.class.getDeclaredField("zkServer");
-        zkServerField.setAccessible(true);
-        ZooKeeperServer zkServer;
-
-        // Wait until the zkServer field is non-null or up to 1s, whichever comes first.
-        long startTime = System.currentTimeMillis();
-        do
+        public TestZooKeeperServer(FileTxnSnapLog txnLog, ServerConfig config)
         {
-            zkServer = (ZooKeeperServer)zkServerField.get(cnxnFactory);
+            super(txnLog, config.getTickTime(), config.getMinSessionTimeout(), config.getMaxSessionTimeout(), null);
         }
-        while ( (zkServer == null) && ((System.currentTimeMillis() - startTime) < MAX_WAIT_MS) );
 
-        return zkServer;
+        private final AtomicBoolean isRunning = new AtomicBoolean(false);
+
+        public RequestProcessor getFirstProcessor()
+        {
+            return firstProcessor;
+        }
+
+        protected void registerJMX()
+        {
+            // NOP
+        }
+
+        @Override
+        protected void unregisterJMX()
+        {
+            // NOP
+        }
+
+        @Override
+        public boolean isRunning()
+        {
+            return isRunning.get() || super.isRunning();
+        }
+
+        public void noteStartup()
+        {
+            synchronized (this) {
+                isRunning.set(true);
+                this.notifyAll();
+            }
+        }
     }
 }
diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperServer.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperServer.java
index eeec877..c14e3e4 100644
--- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperServer.java
+++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperServer.java
@@ -123,6 +123,8 @@
 
         if ( state.compareAndSet(State.STOPPED, State.CLOSED) )
         {
+            configBuilder.close();
+
             InstanceSpec spec = getInstanceSpec();
             if ( spec.deleteDataDirectoryOnClose() )
             {
diff --git a/curator-test/src/main/java/org/apache/curator/test/Timing.java b/curator-test/src/main/java/org/apache/curator/test/Timing.java
index f29b1c5..27e4e53 100644
--- a/curator-test/src/main/java/org/apache/curator/test/Timing.java
+++ b/curator-test/src/main/java/org/apache/curator/test/Timing.java
@@ -34,7 +34,8 @@
 
     private static final int DEFAULT_SECONDS = 10;
     private static final int DEFAULT_WAITING_MULTIPLE = 5;
-    private static final double SESSION_MULTIPLE = .25;
+    private static final double SESSION_MULTIPLE = 1.5;
+    private static final double SESSION_SLEEP_MULTIPLE = SESSION_MULTIPLE * 1.75;  // has to be at least session + 2/3 of a session to account for missed heartbeat then session expiration
 
     /**
      * Use the default base time
@@ -179,6 +180,18 @@
     }
 
     /**
+     * Return a new timing that is a multiple of the this timing
+     *
+     * @param n the multiple
+     * @param waitingMultiple new waitingMultiple
+     * @return this timing times the multiple
+     */
+    public Timing multiple(double n, int waitingMultiple)
+    {
+        return new Timing((int)(value * n), unit, waitingMultiple);
+    }
+
+    /**
      * Return a new timing with the standard multiple for waiting on latches, etc.
      *
      * @return this timing multiplied
@@ -190,13 +203,43 @@
     }
 
     /**
+     * Return a new timing with a multiple that ensures a ZK session timeout
+     *
+     * @return this timing multiplied
+     */
+    public Timing forSessionSleep()
+    {
+        return multiple(SESSION_SLEEP_MULTIPLE, 1);
+    }
+
+    /**
+     * Return a new timing with a multiple for sleeping a smaller amount of time
+     *
+     * @return this timing multiplied
+     */
+    public Timing forSleepingABit()
+    {
+        return multiple(.25);
+    }
+
+    /**
      * Sleep for a small amount of time
      *
      * @throws InterruptedException if interrupted
      */
     public void sleepABit() throws InterruptedException
     {
-        unit.sleep(value / 4);
+        forSleepingABit().sleep();
+    }
+
+    /**
+     * Sleep for a the full amount of time
+     *
+     * @throws InterruptedException if interrupted
+     */
+    public void sleep() throws InterruptedException
+    {
+        unit.sleep(value);
     }
 
     /**
diff --git a/curator-test/src/main/java/org/apache/curator/test/WatchersDebug.java b/curator-test/src/main/java/org/apache/curator/test/WatchersDebug.java
new file mode 100644
index 0000000..e4c3b7e
--- /dev/null
+++ b/curator-test/src/main/java/org/apache/curator/test/WatchersDebug.java
@@ -0,0 +1,92 @@
+/**
+ * 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.test;
+
+import org.apache.zookeeper.ZooKeeper;
+import java.lang.reflect.Method;
+import java.util.List;
+
+public class WatchersDebug
+{
+    private static final Method getDataWatches;
+    private static final Method getExistWatches;
+    private static final Method getChildWatches;
+    static
+    {
+        Method localGetDataWatches = null;
+        Method localGetExistWatches = null;
+        Method localGetChildWatches = null;
+        try
+        {
+            localGetDataWatches = getMethod("getDataWatches");
+            localGetExistWatches = getMethod("getExistWatches");
+            localGetChildWatches = getMethod("getChildWatches");
+        }
+        catch ( NoSuchMethodException e )
+        {
+            e.printStackTrace();
+        }
+        getDataWatches = localGetDataWatches;
+        getExistWatches = localGetExistWatches;
+        getChildWatches = localGetChildWatches;
+    }
+
+    public static List<String> getDataWatches(ZooKeeper zooKeeper)
+    {
+        return callMethod(zooKeeper, WatchersDebug.getDataWatches);
+    }
+
+    public static List<String> getExistWatches(ZooKeeper zooKeeper)
+    {
+        return callMethod(zooKeeper, getExistWatches);
+    }
+
+    public static List<String> getChildWatches(ZooKeeper zooKeeper)
+    {
+        return callMethod(zooKeeper, getChildWatches);
+    }
+
+    private WatchersDebug()
+    {
+    }
+
+    private static Method getMethod(String name) throws NoSuchMethodException
+    {
+        Method m = ZooKeeper.class.getDeclaredMethod(name);
+        m.setAccessible(true);
+        return m;
+    }
+
+    private static List<String> callMethod(ZooKeeper zooKeeper, Method method)
+    {
+        if ( zooKeeper == null )
+        {
+            return null;
+        }
+        try
+        {
+            //noinspection unchecked
+            return (List<String>)method.invoke(zooKeeper);
+        }
+        catch ( Exception e )
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/curator-x-discovery-server/pom.xml b/curator-x-discovery-server/pom.xml
index da80059..5c56b45 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-x-discovery-server</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
     <packaging>bundle</packaging>
 
     <name>Curator Service Discovery Server</name>
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index bcb896f..82169cb 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>curator-x-discovery</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
     <packaging>bundle</packaging>
 
     <name>Curator Service Discovery</name>
diff --git a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
index 8b1e5fc..989edaf 100644
--- a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
+++ b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/details/TestServiceDiscovery.java
@@ -269,6 +269,7 @@
     @Test
     public void testNoServerOnStart() throws Exception
     {
+        Timing timing = new Timing();
         server.stop();
         List<Closeable> closeables = Lists.newArrayList();
         try
@@ -283,6 +284,7 @@
             discovery.start();
 
             server.restart();
+            timing.sleepABit();
             Assert.assertEquals(discovery.queryForNames(), Collections.singletonList("test"));
 
             List<ServiceInstance<String>> list = Lists.newArrayList();
diff --git a/curator-x-rpc/pom.xml b/curator-x-rpc/pom.xml
index 2b16996..50c5353 100644
--- a/curator-x-rpc/pom.xml
+++ b/curator-x-rpc/pom.xml
@@ -22,12 +22,12 @@
     <parent>
         <artifactId>apache-curator</artifactId>
         <groupId>org.apache.curator</groupId>
-        <version>2.9.1-SNAPSHOT</version>
+        <version>3.0.1-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>curator-x-rpc</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
 
     <name>Curator RPC Proxy</name>
     <description>A proxy that bridges non-java environments with the Curator framework and recipes</description>
diff --git a/curator-x-rpc/src/test/java/org/apache/curator/x/rpc/RpcTests.java b/curator-x-rpc/src/test/java/org/apache/curator/x/rpc/RpcTests.java
index a0e0a42..792eedd 100644
--- a/curator-x-rpc/src/test/java/org/apache/curator/x/rpc/RpcTests.java
+++ b/curator-x-rpc/src/test/java/org/apache/curator/x/rpc/RpcTests.java
@@ -100,9 +100,14 @@
     @Override
     public void teardown() throws Exception
     {
-        thriftServer.stop();
-
-        super.teardown();
+        try
+        {
+            thriftServer.stop();
+        }
+        finally
+        {
+            super.teardown();
+        }
     }
 
     @Test
diff --git a/pom.xml b/pom.xml
index c13a8b5..d899fd5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -23,12 +23,12 @@
     <parent>
         <groupId>org.apache</groupId>
         <artifactId>apache</artifactId>
-        <version>16</version>
+        <version>17</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>apache-curator</artifactId>
-    <version>2.9.1-SNAPSHOT</version>
+    <version>3.0.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <name>Apache Curator</name>
@@ -56,18 +56,15 @@
         <project.build.resourceEncoding>UTF-8</project.build.resourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
-        <jdk-version>1.6</jdk-version>
-
-        <surefire-forkcount>1</surefire-forkcount>
+        <jdk-version>1.7</jdk-version>
 
         <!-- versions -->
-        <zookeeper-version>3.4.6</zookeeper-version>
+        <zookeeper-version>3.5.1-alpha</zookeeper-version>
         <maven-project-info-reports-plugin-version>2.7</maven-project-info-reports-plugin-version>
         <maven-bundle-plugin-version>2.3.7</maven-bundle-plugin-version>
         <maven-javadoc-plugin-version>2.10.3</maven-javadoc-plugin-version>
         <doxia-module-confluence-version>1.6</doxia-module-confluence-version>
         <maven-license-plugin-version>1.9.0</maven-license-plugin-version>
-        <javassist-version>3.18.1-GA</javassist-version>
         <commons-math-version>2.2</commons-math-version>
         <jackson-mapper-asl-version>1.9.13</jackson-mapper-asl-version>
         <jersey-version>1.18.1</jersey-version>
@@ -98,7 +95,7 @@
         <connection>scm:git:https://git-wip-us.apache.org/repos/asf/curator.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/curator.git
         </developerConnection>
-        <tag>apache-curator-2.8.0</tag>
+        <tag>HEAD</tag>
     </scm>
 
     <issueManagement>
@@ -347,12 +344,6 @@
             </dependency>
 
             <dependency>
-                <groupId>org.javassist</groupId>
-                <artifactId>javassist</artifactId>
-                <version>${javassist-version}</version>
-            </dependency>
-
-            <dependency>
                 <groupId>org.apache.commons</groupId>
                 <artifactId>commons-math</artifactId>
                 <version>${commons-math-version}</version>
@@ -631,7 +622,7 @@
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-surefire-plugin</artifactId>
                 <configuration>
-                    <forkCount>${surefire-forkcount}</forkCount>
+                    <threadCount>1</threadCount>
                     <reuseForks>false</reuseForks>
                     <redirectTestOutputToFile>true</redirectTestOutputToFile>
                 </configuration>
diff --git a/src/site/confluence/errors.confluence b/src/site/confluence/errors.confluence
index 28805e0..f58c8d7 100644
--- a/src/site/confluence/errors.confluence
+++ b/src/site/confluence/errors.confluence
@@ -23,13 +23,20 @@
 appropriate action. These are the possible state changes:
 
 |CONNECTED|Sent for the first successful connection to the server. NOTE: You will only get one of these messages for any CuratorFramework instance.|
-|SUSPENDED|There has been a loss of connection. Leaders, locks, etc. should suspend until the connection is re\-established. If the connection times\-out you will receive a LOST notice.|
-|RECONNECTED|A suspended or lost connection has been re\-established.|
-|LOST|The connection is confirmed to be lost. Close any locks, leaders, etc. and attempt to re\-create them. NOTE: it is possible to get a RECONNECTED state after this but you should still consider any locks, etc. as dirty/unstable.|
 |READ_ONLY|The connection has gone into read\-only mode. This can only happen if you pass true for CuratorFrameworkFactory.Builder.canBeReadOnly(). See the ZooKeeper doc regarding read only connections: [[http://wiki.apache.org/hadoop/ZooKeeper/GSoCReadOnlyMode]]. The connection will remain in read only mode until another state change is sent.|
+|SUSPENDED|There has been a loss of connection. Leaders, locks, etc. should suspend until the connection is re\-established.|
+|RECONNECTED|A suspended or lost connection has been re\-established.|
+|LOST|Curator will set the LOST state when it believes that the ZooKeeper session has expired. ZooKeeper connections have a session. When the session expires, clients must take appropriate action. In Curator, this is complicated by the fact that Curator internally manages the ZooKeeper connection. Curator will set the LOST state when any of the following occurs: a) ZooKeeper returns a Watcher.Event.KeeperState.Expired or KeeperException.Code.SESSIONEXPIRED; b) Curator closes the internally managed ZooKeeper instance; c) The session timeout elapses during a network partition. It is possible to get a RECONNECTED state after this but you should still consider any locks, etc. as dirty/unstable. *NOTE*: The meaning of LOST has changed since Curator 3.0.0. Prior to 3.0.0 LOST only meant that the retry policy had expired.|
 
 {{UnhandledErrorListener}} is called when a background task, etc. catches an exception. In general, Curator users shouldn't care
 about these as they are logged. However, you can listen for them if you choose.
 
+h2. Error Policy
+
+Curator has a pluggable error policy. The default policy takes the conservative approach of treating connection states SUSPENDED and LOST the same way.
+i.e. when a recipe sees the state change to SUSPENDED it will assume that the ZooKeeper session is lost and will clean up any watchers, nodes, etc. You can choose,
+however, a more aggressive approach by setting the error policy to only treat LOST (i.e. true session loss) as an error state. Do this in the CuratorFrameworkFactory via:
+{{connectionStateErrorPolicy(new SessionConnectionStateErrorPolicy())}}.
+
 h2. Recipes
 In general, the recipes attempt to deal with errors and connection issues. See the doc for each recipe for details on how it deals with errors.
diff --git a/src/site/confluence/index.confluence b/src/site/confluence/index.confluence
index cd533cf..543e041 100644
--- a/src/site/confluence/index.confluence
+++ b/src/site/confluence/index.confluence
@@ -33,3 +33,13 @@
 |org.apache.curator|curator\-x\-discovery|A Service Discovery implementation built on the Curator Framework.|
 |org.apache.curator|curator\-x\-discovery-server|A RESTful server that can be used with Curator Discovery.|
 |org.apache.curator|curator\-x\-rpc|A proxy that bridges non\-java environments with the Curator framework and recipes.|
+
+h2. Versions
+
+The are currently two released versions of Curator, 2.x.x and 3.x.x:
+
+* Curator 2.x.x \- compatible with both ZooKeeper 3.4.x and ZooKeeper 3.5.x
+* Curator 3.x.x \- compatible only with ZooKeeper 3.5.x and includes support for new features such as dynamic reconfiguration, etc.
+
+ZooKeeper 3.5.x adds a number of new features including dynamic reconfiguration, watcher removal and others. The Curator 3.x.x Jira
+issues tracking these features are here: [[https://issues.apache.org/jira/browse/CURATOR-159]].
diff --git a/src/site/confluence/utilities.confluence b/src/site/confluence/utilities.confluence
index 16bbc13..3a62fa5 100644
--- a/src/site/confluence/utilities.confluence
+++ b/src/site/confluence/utilities.confluence
@@ -14,6 +14,18 @@
 * getSortedChildren: Return the children of the given path sorted by sequence number
 * makePath: Given a parent path and a child node, create a combined full path
 
+h2. Locker
+
+Curator's Locker uses Java 7's try\-with\-resources feature to making using Curator locks safer:
+
+{code}
+InterProcessMutex mutex = new InterProcessMutex(...) // or any InterProcessLock
+try ( Locker locker = new Locker(mutex, maxTimeout, unit) )
+{
+   // do work
+}
+{code}
+
 h2. BlockingQueueConsumer
 
 See: *[[DistributedQueue|curator-recipes/distributed-queue.html]]* and *[[DistributedPriorityQueue|curator-recipes/distributed-priority-queue.html]]*
@@ -25,3 +37,9 @@
 Due to limitations in ZooKeeper's transport layer, a single queue will break if it has more than 10K\-ish items in it. This class
 provides a facade over multiple distributed queues. It monitors the queues and if any one of them goes over a threshold, a new
 queue is added. Puts are distributed amongst the queues.
+
+h2. WatcherRemoveCuratorFramework
+
+Curator has a utility that makes it easy to set watchers and remove them at a later date. It is used for all Curator recipes.
+From your CuratorFramework instance, call newWatcherRemoveCuratorFramework(). When using this proxy instance any watchers that are
+set are recorded. You can then call removeWatchers() to remove those watchers. See the Curator source code for usage details.