You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by ra...@apache.org on 2015/10/06 03:42:38 UTC

[1/2] curator git commit: Support delete().quietly() and create().orSetData()

Repository: curator
Updated Branches:
  refs/heads/CURATOR-3.0 afa8f7a45 -> 537156db4


Support delete().quietly() and create().orSetData()


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

Branch: refs/heads/CURATOR-3.0
Commit: add56dc92e4564ce5d146a8f3c0a8ae51c6dc46f
Parents: afa8f7a
Author: randgalt <ra...@apache.org>
Authored: Fri Oct 2 17:22:48 2015 -0500
Committer: randgalt <ra...@apache.org>
Committed: Fri Oct 2 17:22:48 2015 -0500

----------------------------------------------------------------------
 .../curator/framework/api/CreateBuilder.java    | 65 +--------------
 .../framework/api/CreateBuilderMain.java        | 86 ++++++++++++++++++++
 .../curator/framework/api/DeleteBuilder.java    |  4 +-
 .../framework/api/DeleteBuilderMain.java        | 23 ++++++
 .../framework/imps/CreateBuilderImpl.java       | 61 ++++++++++++++
 .../framework/imps/DeleteBuilderImpl.java       | 21 +++++
 .../curator/framework/imps/TestFramework.java   | 75 +++++++++++++++++
 7 files changed, 271 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
index fa1a68e..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,69 +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<CreateBackgroundModeStatACLable>,
-    Statable<CreateProtectACLCreateModePathAndBytesable<String>>
+    CreateBuilderMain
 {
-    /**
-     * 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();
+    CreateBuilderMain orSetData();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilderMain.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilder.java
----------------------------------------------------------------------
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 2da1843..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 GuaranteeableDeletable, ChildrenDeletable
+public interface DeleteBuilder extends
+    Quietly<DeleteBuilderMain>,
+    DeleteBuilderMain
 {
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java b/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java
new file mode 100644
index 0000000..8c68607
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java
@@ -0,0 +1,23 @@
+/**
+ * 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 DeleteBuilderMain extends GuaranteeableDeletable, ChildrenDeletable
+{
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index d44998d..8a8787a 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
@@ -51,6 +51,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
     private boolean createParentsAsContainers;
     private boolean doProtected;
     private boolean compress;
+    private boolean setDataIfExists;
     private String protectedId;
     private ACLing acling;
     private Stat storingStat;
@@ -71,10 +72,18 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         createParentsAsContainers = false;
         compress = false;
         doProtected = false;
+        setDataIfExists = false;
         protectedId = null;
         storingStat = null;
     }
 
+    @Override
+    public CreateBuilderMain orSetData()
+    {
+        setDataIfExists = true;
+        return this;
+    }
+
     <T> TransactionCreateBuilder<T> asTransactionCreateBuilder(final T context, final CuratorMultiTransactionRecord transaction)
     {
         return new TransactionCreateBuilder<T>()
@@ -537,6 +546,10 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                         {
                             backgroundCreateParentsThenNode(client, operationAndData, operationAndData.getData().getPath(), backgrounding, createParentsAsContainers);
                         }
+                        else if ( (rc == KeeperException.Code.NODEEXISTS.intValue()) && setDataIfExists )
+                        {
+                            backgroundSetData(client, operationAndData, operationAndData.getData().getPath(), backgrounding);
+                        }
                         else
                         {
                             sendBackgroundResponse(rc, path, ctx, name, null, operationAndData);
@@ -689,6 +702,42 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
         client.queueOperation(parentOperation);
     }
 
+    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);
@@ -999,6 +1048,18 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperation<PathAndByt
                                     throw e;
                                 }
                             }
+                            catch ( KeeperException.NodeExistsException e )
+                            {
+                                if ( setDataIfExists )
+                                {
+                                    client.getZooKeeper().setData(path, data, -1);
+                                    createdPath = path;
+                                }
+                                else
+                                {
+                                    throw e;
+                                }
+                            }
                         }
 
                         if ( failNextCreateForTesting )

http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/imps/DeleteBuilderImpl.java
----------------------------------------------------------------------
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 2a98f56..10bed18 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,6 +27,7 @@ import org.apache.curator.framework.api.ChildrenDeletable;
 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.OperationType;
 import org.apache.curator.framework.api.transaction.TransactionDeleteBuilder;
@@ -44,6 +45,7 @@ class DeleteBuilderImpl implements DeleteBuilder, BackgroundOperation<String>
     private Backgrounding backgrounding;
     private boolean deletingChildrenIfNeeded;
     private boolean guaranteed;
+    private boolean quietly;
 
     DeleteBuilderImpl(CuratorFrameworkImpl client)
     {
@@ -52,6 +54,7 @@ class DeleteBuilderImpl implements DeleteBuilder, BackgroundOperation<String>
         backgrounding = new Backgrounding();
         deletingChildrenIfNeeded = false;
         guaranteed = false;
+        quietly = false;
     }
 
     <T> TransactionDeleteBuilder<T> asTransactionDeleteBuilder(final T context, final CuratorMultiTransactionRecord transaction)
@@ -76,6 +79,13 @@ class DeleteBuilderImpl implements DeleteBuilder, BackgroundOperation<String>
     }
 
     @Override
+    public DeleteBuilderMain quietly()
+    {
+        quietly = true;
+        return this;
+    }
+
+    @Override
     public ChildrenDeletable guaranteed()
     {
         guaranteed = true;
@@ -158,6 +168,10 @@ class DeleteBuilderImpl implements DeleteBuilder, BackgroundOperation<String>
                         }
                         else
                         {
+                            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);
                         }
@@ -240,6 +254,13 @@ class DeleteBuilderImpl implements DeleteBuilder, BackgroundOperation<String>
                             {
                                 client.getZooKeeper().delete(path, version);
                             }
+                            catch ( KeeperException.NoNodeException e )
+                            {
+                                if ( !quietly )
+                                {
+                                    throw e;
+                                }
+                            }
                             catch ( KeeperException.NotEmptyException e )
                             {
                                 if ( deletingChildrenIfNeeded )

http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java
----------------------------------------------------------------------
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 472e0fc..3f4462d 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
@@ -102,6 +102,81 @@ public class TestFramework extends BaseClassForTests
     }
 
     @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();


[2/2] curator git commit: for some reason, one of the constructors got messed up

Posted by ra...@apache.org.
for some reason, one of the constructors got messed up


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

Branch: refs/heads/CURATOR-3.0
Commit: 537156db46becd67542a653d6ab055168ed507d0
Parents: add56dc
Author: randgalt <ra...@apache.org>
Authored: Mon Oct 5 20:27:46 2015 -0500
Committer: randgalt <ra...@apache.org>
Committed: Mon Oct 5 20:27:46 2015 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/curator/CuratorZookeeperClient.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/537156db/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java
----------------------------------------------------------------------
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 471adf0..18d8572 100644
--- a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java
+++ b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java
@@ -93,7 +93,7 @@ public class CuratorZookeeperClient implements Closeable
      */
     public CuratorZookeeperClient(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher watcher, RetryPolicy retryPolicy, boolean canBeReadOnly)
     {
-        this(new DefaultZookeeperFactory(), ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, canBeReadOnly, new ClassicConnectionHandlingPolicy());
+        this(zookeeperFactory, ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, canBeReadOnly, new ClassicConnectionHandlingPolicy());
     }
 
     /**