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/22 02:05:15 UTC

curator git commit: Previous change to create containers broke some old behavior. Old EnsurePath class only created parents once. So, introduce new EnsureContainers class to do similar but for containers. Also, make usage in a protected method so that it

Repository: curator
Updated Branches:
  refs/heads/CURATOR-274 [created] 32588986e


Previous change to create containers broke some old behavior. Old EnsurePath class only created parents once. So, introduce new EnsureContainers class to do similar but for containers. Also, make usage in a protected method so that it can be turned off by users


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

Branch: refs/heads/CURATOR-274
Commit: 32588986ea141a8396a14842cf4b10f0bbb0a78f
Parents: 722d0ce
Author: randgalt <ra...@apache.org>
Authored: Wed Oct 21 19:05:02 2015 -0500
Committer: randgalt <ra...@apache.org>
Committed: Wed Oct 21 19:05:02 2015 -0500

----------------------------------------------------------------------
 .../curator/framework/EnsureContainers.java     | 65 ++++++++++++++++++++
 .../recipes/cache/PathChildrenCache.java        | 13 ++--
 .../recipes/queue/SimpleDistributedQueue.java   | 13 ++--
 3 files changed, 81 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/curator/blob/32588986/curator-framework/src/main/java/org/apache/curator/framework/EnsureContainers.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/EnsureContainers.java b/curator-framework/src/main/java/org/apache/curator/framework/EnsureContainers.java
new file mode 100644
index 0000000..83e0692
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/EnsureContainers.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;
+
+import org.apache.curator.utils.EnsurePath;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Similar to {@link EnsurePath} but creates containers.
+ *
+ */
+public class EnsureContainers
+{
+    private final CuratorFramework client;
+    private final String path;
+    private final AtomicBoolean ensureNeeded = new AtomicBoolean(true);
+
+    /**
+     * @param client the client
+     * @param path path to ensure is containers
+     */
+    public EnsureContainers(CuratorFramework client, String path)
+    {
+        this.client = client;
+        this.path = path;
+    }
+
+    /**
+     * The first time this method is called, all nodes in the
+     * path will be created as containers if needed
+     *
+     * @throws Exception errors
+     */
+    public void ensure() throws Exception
+    {
+        if ( ensureNeeded.get() )
+        {
+            internalEnsure();
+        }
+    }
+
+    private synchronized void internalEnsure() throws Exception
+    {
+        if ( ensureNeeded.compareAndSet(true, false) )
+        {
+            client.createContainers(path);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/curator/blob/32588986/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 b5d912c..3ee6b6d 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.ImmutableList;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.EnsureContainers;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.listen.ListenerContainer;
@@ -77,6 +78,7 @@ public class PathChildrenCache implements Closeable
     private final AtomicReference<Map<String, ChildData>> initialSet = new AtomicReference<Map<String, ChildData>>();
     private final Set<Operation> operationsQuantizer = Sets.newSetFromMap(Maps.<Operation, Boolean>newConcurrentMap());
     private final AtomicReference<State> state = new AtomicReference<State>(State.LATENT);
+    private final EnsureContainers ensureContainers;
 
     private enum State
     {
@@ -220,6 +222,7 @@ public class PathChildrenCache implements Closeable
         this.cacheData = cacheData;
         this.dataIsCompressed = dataIsCompressed;
         this.executorService = executorService;
+        ensureContainers = new EnsureContainers(client, path);
     }
 
     /**
@@ -563,6 +566,11 @@ public class PathChildrenCache implements Closeable
         log.error("", e);
     }
 
+    protected void ensurePath() throws Exception
+    {
+        ensureContainers.ensure();
+    }
+
     @VisibleForTesting
     protected void remove(String fullPath)
     {
@@ -611,11 +619,6 @@ public class PathChildrenCache implements Closeable
         }
     }
 
-    private void ensurePath() throws Exception
-    {
-        client.createContainers(path);
-    }
-
     private void handleStateChange(ConnectionState newState)
     {
         switch ( newState )

http://git-wip-us.apache.org/repos/asf/curator/blob/32588986/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 9650ffb..35afb53 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
@@ -19,6 +19,7 @@
 package org.apache.curator.framework.recipes.queue;
 
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.EnsureContainers;
 import org.apache.curator.utils.PathUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.CreateMode;
@@ -49,6 +50,7 @@ public class SimpleDistributedQueue
     private final Logger log = LoggerFactory.getLogger(getClass());
     private final CuratorFramework client;
     private final String path;
+    private final EnsureContainers ensureContainers;
 
     private final String PREFIX = "qn-";
 
@@ -60,6 +62,7 @@ public class SimpleDistributedQueue
     {
         this.client = client;
         this.path = PathUtils.validatePath(path);
+        ensureContainers = new EnsureContainers(client, path);
     }
 
     /**
@@ -174,6 +177,11 @@ public class SimpleDistributedQueue
         }
     }
 
+    protected void ensurePath() throws Exception
+    {
+        ensureContainers.ensure();
+    }
+
     private byte[] internalPoll(long timeout, TimeUnit unit) throws Exception
     {
         ensurePath();
@@ -215,11 +223,6 @@ public class SimpleDistributedQueue
         }
     }
 
-    private void ensurePath() throws Exception
-    {
-        client.createContainers(path);
-    }
-
     private byte[] internalElement(boolean removeIt, Watcher watcher) throws Exception
     {
         ensurePath();