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/06/24 23:25:42 UTC

curator git commit: added more usages of exists().creatingParents some more tests and added this behavior optionally to TreeCache

Repository: curator
Updated Branches:
  refs/heads/CURATOR-222 7ad12754a -> b4ff09517


added more usages of exists().creatingParents some more tests and added this behavior optionally to TreeCache


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

Branch: refs/heads/CURATOR-222
Commit: b4ff09517a9f429fac46c24eaf5f79857b4a35ec
Parents: 7ad1275
Author: randgalt <ra...@apache.org>
Authored: Wed Jun 24 16:25:33 2015 -0500
Committer: randgalt <ra...@apache.org>
Committed: Wed Jun 24 16:25:33 2015 -0500

----------------------------------------------------------------------
 .../curator/framework/imps/TestFramework.java   | 27 ++++++++++++++++++++
 .../recipes/cache/PathChildrenCache.java        |  9 +------
 .../framework/recipes/cache/TreeCache.java      | 27 +++++++++++++++++---
 .../framework/recipes/queue/QueueSharder.java   |  2 +-
 .../recipes/queue/SimpleDistributedQueue.java   |  9 +------
 .../framework/recipes/cache/TestTreeCache.java  | 13 ++++++++++
 6 files changed, 67 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/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 528b4a5..bb8747e 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
@@ -493,6 +493,32 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
+    public void testCreatingParentsTheSame() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+
+            Assert.assertNull(client.checkExists().forPath("/one/two"));
+            client.create().creatingParentContainersIfNeeded().forPath("/one/two/three");
+            Assert.assertNotNull(client.checkExists().forPath("/one/two"));
+
+            client.delete().deletingChildrenIfNeeded().forPath("/one");
+            Assert.assertNull(client.checkExists().forPath("/one"));
+
+            Assert.assertNull(client.checkExists().forPath("/one/two"));
+            client.checkExists().creatingParentContainersIfNeeded().forPath("/one/two/three");
+            Assert.assertNotNull(client.checkExists().forPath("/one/two"));
+            Assert.assertNull(client.checkExists().forPath("/one/two/three"));
+        }
+        finally
+        {
+            CloseableUtils.closeQuietly(client);
+        }
+    }
+
+    @Test
     public void testExistsCreatingParents() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
@@ -501,6 +527,7 @@ public class TestFramework extends BaseClassForTests
             client.start();
 
             Assert.assertNull(client.checkExists().forPath("/one/two"));
+            client.create().creatingParentContainersIfNeeded().forPath("/one/two/three");
             client.checkExists().creatingParentContainersIfNeeded().forPath("/one/two/three");
             Assert.assertNotNull(client.checkExists().forPath("/one/two"));
             Assert.assertNull(client.checkExists().forPath("/one/two/three"));

http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
----------------------------------------------------------------------
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 d2e3ddf..2010008 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
@@ -613,14 +613,7 @@ public class PathChildrenCache implements Closeable
 
     private void ensurePath() throws Exception
     {
-        try
-        {
-            client.create().creatingParentContainersIfNeeded().forPath(path);
-        }
-        catch ( KeeperException.NodeExistsException ignore )
-        {
-            // ignore
-        }
+        client.checkExists().creatingParentContainersIfNeeded().forPath(path);
     }
 
     private void handleStateChange(ConnectionState newState)

http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
----------------------------------------------------------------------
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 41fc574..16da736 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
@@ -69,6 +69,7 @@ import static org.apache.curator.utils.PathUtils.validatePath;
 public class TreeCache implements Closeable
 {
     private static final Logger LOG = LoggerFactory.getLogger(TreeCache.class);
+    private final boolean createParentNodes;
 
     public static final class Builder
     {
@@ -78,6 +79,7 @@ public class TreeCache implements Closeable
         private boolean dataIsCompressed = false;
         private CloseableExecutorService executorService = null;
         private int maxDepth = Integer.MAX_VALUE;
+        private boolean createParentNodes = false;
 
         private Builder(CuratorFramework client, String path)
         {
@@ -95,7 +97,7 @@ public class TreeCache implements Closeable
             {
                 executor = new CloseableExecutorService(Executors.newSingleThreadExecutor(defaultThreadFactory));
             }
-            return new TreeCache(client, path, cacheData, dataIsCompressed, maxDepth, executor);
+            return new TreeCache(client, path, cacheData, dataIsCompressed, maxDepth, executor, createParentNodes);
         }
 
         /**
@@ -159,6 +161,19 @@ public class TreeCache implements Closeable
             this.maxDepth = maxDepth;
             return this;
         }
+
+        /**
+         * By default, TreeCache does not auto-create parent nodes for the cached path. Change
+         * this behavior with this method. NOTE: parent nodes are created as containers
+         *
+         * @param createParentNodes true to create parent nodes
+         * @return this for chaining
+         */
+        public Builder setCreateParentNodes(boolean createParentNodes)
+        {
+            this.createParentNodes = createParentNodes;
+            return this;
+        }
     }
 
     /**
@@ -500,7 +515,7 @@ public class TreeCache implements Closeable
      */
     public TreeCache(CuratorFramework client, String path)
     {
-        this(client, path, true, false, Integer.MAX_VALUE, new CloseableExecutorService(Executors.newSingleThreadExecutor(defaultThreadFactory), true));
+        this(client, path, true, false, Integer.MAX_VALUE, new CloseableExecutorService(Executors.newSingleThreadExecutor(defaultThreadFactory), true), false);
     }
 
     /**
@@ -509,9 +524,11 @@ public class TreeCache implements Closeable
      * @param cacheData        if true, node contents are cached in addition to the stat
      * @param dataIsCompressed if true, data in the path is compressed
      * @param executorService  Closeable ExecutorService to use for the TreeCache's background thread
+     * @param createParentNodes true to create parent nodes as containers
      */
-    TreeCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, int maxDepth, final CloseableExecutorService executorService)
+    TreeCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, int maxDepth, final CloseableExecutorService executorService, boolean createParentNodes)
     {
+        this.createParentNodes = createParentNodes;
         this.root = new TreeNode(validatePath(path), null);
         this.client = client;
         this.cacheData = cacheData;
@@ -529,6 +546,10 @@ public class TreeCache implements Closeable
     public TreeCache start() throws Exception
     {
         Preconditions.checkState(treeState.compareAndSet(TreeState.LATENT, TreeState.STARTED), "already started");
+        if ( createParentNodes )
+        {
+            client.checkExists().creatingParentContainersIfNeeded().forPath(root.path);
+        }
         client.getConnectionStateListenable().addListener(connectionStateListener);
         if ( client.getZookeeperClient().isConnected() )
         {

http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSharder.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSharder.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSharder.java
index 2dbd484..c54e513 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSharder.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSharder.java
@@ -111,7 +111,7 @@ public class QueueSharder<U, T extends QueueBase<U>> implements Closeable
     {
         Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Cannot be started more than once");
 
-        client.newNamespaceAwareEnsurePath(queuePath).ensure(client.getZookeeperClient());
+        client.checkExists().creatingParentContainersIfNeeded().forPath(ZKPaths.makePath(queuePath, "foo"));
 
         getInitialQueues();
         leaderLatch.start();

http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java
index 1d71c64..dd07d1c 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java
@@ -217,14 +217,7 @@ public class SimpleDistributedQueue
 
     private void ensurePath() throws Exception
     {
-        try
-        {
-            client.create().creatingParentContainersIfNeeded().forPath(path);
-        }
-        catch ( KeeperException.NodeExistsException ignore )
-        {
-            // ignore
-        }
+        client.checkExists().creatingParentContainersIfNeeded().forPath(path);
     }
 
     private byte[] internalElement(boolean removeIt, Watcher watcher) throws Exception

http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
----------------------------------------------------------------------
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 467f32b..e98c21f 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
@@ -56,6 +56,19 @@ public class TestTreeCache extends BaseTestTreeCache
     }
 
     @Test
+    public void testCreateParents() throws Exception
+    {
+        cache = newTreeCacheWithListeners(client, "/one/two/three");
+        cache.start();
+        Assert.assertNull(client.checkExists().forPath("/one/two/three"));
+        cache.close();
+
+        cache = TreeCache.newBuilder(client, "/one/two/three").setCreateParentNodes(true).build();
+        cache.start();
+        Assert.assertNotNull(client.checkExists().forPath("/one/two"));
+    }
+
+    @Test
     public void testStartEmpty() throws Exception
     {
         cache = newTreeCacheWithListeners(client, "/test");