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 2013/06/20 19:50:47 UTC

[01/14] git commit: If mutex. acquire() throws a KeeperException.SessionExpiredException in LeaderSelector.doWork(), the exception is not handled in LeaderSelector.doWorkLoop(), causing the loop to terminate even when autoRequeue is true.

Updated Branches:
  refs/heads/master b766eb76a -> a95842651


If mutex. acquire() throws a KeeperException.SessionExpiredException in LeaderSelector.doWork(), the exception is not handled in LeaderSelector.doWorkLoop(), causing the loop to terminate even when autoRequeue is true.


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

Branch: refs/heads/master
Commit: 0c1af3bdd974cfbfc308783aed1afe743a3c1f62
Parents: aec1cfd
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 07:58:48 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 07:58:48 2013 -0700

----------------------------------------------------------------------
 .../framework/recipes/leader/LeaderSelector.java      | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/0c1af3bd/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java
index 21d34c7..5c33622 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java
@@ -370,22 +370,28 @@ public class LeaderSelector implements Closeable
     {
         do
         {
+            KeeperException exception = null;
             try
             {
                 doWork();
             }
             catch ( KeeperException.ConnectionLossException e )
             {
-                if ( !autoRequeue.get() )   // autoRequeue should ignore connection loss and just keep trying
-                {
-                    throw e;
-                }
+                exception = e;
+            }
+            catch ( KeeperException.SessionExpiredException e )
+            {
+                exception = e;
             }
             catch ( InterruptedException ignore )
             {
                 Thread.currentThread().interrupt();
                 break;
             }
+            if ( (exception != null) && !autoRequeue.get() )   // autoRequeue should ignore connection loss or session expired and just keep trying
+            {
+                throw exception;
+            }
         } while ( autoRequeue.get() && (state.get() == State.STARTED) && !Thread.currentThread().isInterrupted() );
     }
 


[11/14] git commit: Added missing license headers

Posted by ra...@apache.org.
Added missing license headers


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

Branch: refs/heads/master
Commit: 133f846171dfd3fc967aaa98d5155725cd765675
Parents: d9d0dc2
Author: randgalt <ra...@apache.org>
Authored: Tue Jun 18 15:29:43 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Tue Jun 18 15:29:43 2013 -0700

----------------------------------------------------------------------
 .../curator/framework/imps/SyncBuilderImpl.java   | 18 ++++++++++++++++++
 .../recipes/nodes/PersistentEphemeralNode.java    | 18 ++++++++++++++++++
 .../nodes/TestPersistentEphemeralNode.java        | 18 ++++++++++++++++++
 3 files changed, 54 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/133f8461/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java
----------------------------------------------------------------------
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 7c74abc..e168d76 100644
--- 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
@@ -1,3 +1,21 @@
+/**
+ * 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.TimeTrace;

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/133f8461/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
index 7e92bdf..9366492 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
@@ -1,3 +1,21 @@
+/**
+ * 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.nodes;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/133f8461/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
----------------------------------------------------------------------
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 84c1cf7..865a43d 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
@@ -1,3 +1,21 @@
+/**
+ * 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.nodes;
 
 import com.google.common.base.Throwables;


[13/14] git commit: 2.1.0-incubating-SNAPSHOT

Posted by ra...@apache.org.
2.1.0-incubating-SNAPSHOT


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

Branch: refs/heads/master
Commit: 34372218a1f74c302960f6c41e0aa590207839ce
Parents: 2cb8b0e
Author: randgalt <ra...@apache.org>
Authored: Thu Jun 20 10:49:03 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Thu Jun 20 10:49:03 2013 -0700

----------------------------------------------------------------------
 curator-client/pom.xml             |  4 ++--
 curator-examples/pom.xml           |  4 ++--
 curator-framework/pom.xml          |  4 ++--
 curator-recipes/pom.xml            |  4 ++--
 curator-test/pom.xml               |  4 ++--
 curator-x-discovery-server/pom.xml |  4 ++--
 curator-x-discovery/pom.xml        |  4 ++--
 pom.xml                            | 14 +++++++-------
 8 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-client/pom.xml
----------------------------------------------------------------------
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 5d67db6..947f54e 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-client</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Client</name>
     <description>Low-level API</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-examples/pom.xml
----------------------------------------------------------------------
diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index 4728708..f541fcb 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-examples</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Examples</name>
     <description>Example usages of various Curator features.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-framework/pom.xml
----------------------------------------------------------------------
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index 4b8f9c6..83b3952 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-framework</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Framework</name>
     <description>High-level API that greatly simplifies using ZooKeeper.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-recipes/pom.xml
----------------------------------------------------------------------
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index b121931..0557e65 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-recipes</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Recipes</name>
     <description>All of the recipes listed on the ZooKeeper recipes doc (except two phase commit).</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 287a439..6d713ac 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-test</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Testing</name>
     <description>Unit testing utilities.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-x-discovery-server/pom.xml
----------------------------------------------------------------------
diff --git a/curator-x-discovery-server/pom.xml b/curator-x-discovery-server/pom.xml
index c29373b..00d1033 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-x-discovery-server</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Service Discovery Server</name>
     <description>Bridges non-Java or legacy applications with the Curator Service Discovery.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/curator-x-discovery/pom.xml
----------------------------------------------------------------------
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 2bf3b08..bee8ea6 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating</version>
+        <version>2.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-x-discovery</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
 
     <name>Curator Service Discovery</name>
     <description>A service discovery recipe.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/34372218/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5995bfe..d2fad2a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.curator</groupId>
     <artifactId>apache-curator</artifactId>
-    <version>2.1.0-incubating</version>
+    <version>2.1.0-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <name>Apache Curator</name>
@@ -229,37 +229,37 @@
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-client</artifactId>
-                <version>2.1.0-incubating</version>
+                <version>2.1.0-incubating-SNAPSHOT</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-framework</artifactId>
-                <version>2.1.0-incubating</version>
+                <version>2.1.0-incubating-SNAPSHOT</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-recipes</artifactId>
-                <version>2.1.0-incubating</version>
+                <version>2.1.0-incubating-SNAPSHOT</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-test</artifactId>
-                <version>2.1.0-incubating</version>
+                <version>2.1.0-incubating-SNAPSHOT</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-x-discovery</artifactId>
-                <version>2.1.0-incubating</version>
+                <version>2.1.0-incubating-SNAPSHOT</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-x-discovery-server</artifactId>
-                <version>2.1.0-incubating</version>
+                <version>2.1.0-incubating-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>


[02/14] git commit: PersistentEphemeralNode recipe

Posted by ra...@apache.org.
PersistentEphemeralNode recipe


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

Branch: refs/heads/master
Commit: f074683bc44b08dfa201e81977a9e736644bfae8
Parents: aec1cfd
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 08:42:25 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 08:42:25 2013 -0700

----------------------------------------------------------------------
 .../recipes/nodes/PersistentEphemeralNode.java  | 349 ++++++++++++++++++
 .../nodes/TestPersistentEphemeralNode.java      | 361 +++++++++++++++++++
 2 files changed, 710 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/f074683b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
new file mode 100644
index 0000000..d86af92
--- /dev/null
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
@@ -0,0 +1,349 @@
+package org.apache.curator.framework.recipes.nodes;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.ACLBackgroundPathAndBytesable;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CreateModable;
+import org.apache.curator.framework.api.CuratorEvent;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.utils.EnsurePath;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.Closeable;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * <p>
+ *     A persistent ephemeral node is an ephemeral node that attempts to stay present in
+ *     ZooKeeper, even through connection and session interruptions.
+ * </p>
+ *
+ * <p>
+ *     Thanks to bbeck (https://github.com/bbeck) for the initial coding and design
+ * </p>
+ */
+public class PersistentEphemeralNode implements Closeable
+{
+    @VisibleForTesting
+    volatile CountDownLatch         initialCreateLatch = new CountDownLatch(1);
+
+    private final Logger                    log = LoggerFactory.getLogger(getClass());
+    private final CuratorFramework          client;
+    private final EnsurePath                ensurePath;
+    private final CreateModable<ACLBackgroundPathAndBytesable<String>>  createMethod;
+    private final AtomicReference<String>   nodePath = new AtomicReference<String>(null);
+    private final String                    basePath;
+    private final Mode                      mode;
+    private final byte[]                    data;
+    private final AtomicReference<State>    state = new AtomicReference<State>(State.LATENT);
+    private final AtomicBoolean             isSuspended = new AtomicBoolean(false);
+    private final BackgroundCallback        backgroundCallback;
+    private final Watcher                   watcher = new Watcher()
+    {
+        @Override
+        public void process(WatchedEvent event)
+        {
+            if ( Objects.equal(nodePath.get(), event.getPath()) )
+            {
+                createNode();
+            }
+        }
+    };
+    private final ConnectionStateListener   listener = new ConnectionStateListener()
+    {
+        @Override
+        public void stateChanged(CuratorFramework client, ConnectionState newState)
+        {
+            isSuspended.set((newState != ConnectionState.RECONNECTED) && (newState != ConnectionState.CONNECTED));
+            if ( newState == ConnectionState.RECONNECTED )
+            {
+                createNode();
+            }
+        }
+    };
+    private final BackgroundCallback        checkExistsCallback = new BackgroundCallback()
+    {
+        @Override
+        public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+        {
+            if ( event.getResultCode() == KeeperException.Code.NONODE.intValue() )
+            {
+                createNode();
+            }
+        }
+    };
+
+    private enum State
+    {
+        LATENT,
+        STARTED,
+        CLOSED
+    }
+
+    /**
+     * The mode for node creation
+     */
+    public enum Mode
+    {
+        /**
+         * Same as {@link CreateMode#EPHEMERAL}
+         */
+        EPHEMERAL()
+        {
+            @Override
+            protected CreateMode getCreateMode(boolean pathIsSet)
+            {
+                return CreateMode.EPHEMERAL;
+            }
+
+            @Override
+            protected boolean isProtected()
+            {
+                return false;
+            }
+        },
+
+        /**
+         * Same as {@link CreateMode#EPHEMERAL_SEQUENTIAL}
+         */
+        EPHEMERAL_SEQUENTIAL()
+        {
+            @Override
+            protected CreateMode getCreateMode(boolean pathIsSet)
+            {
+                return pathIsSet ? CreateMode.EPHEMERAL : CreateMode.EPHEMERAL_SEQUENTIAL;
+            }
+
+            @Override
+            protected boolean isProtected()
+            {
+                return false;
+            }
+        },
+
+        /**
+         * Same as {@link CreateMode#EPHEMERAL} with protection
+         */
+        PROTECTED_EPHEMERAL()
+        {
+            @Override
+            protected CreateMode getCreateMode(boolean pathIsSet)
+            {
+                return CreateMode.EPHEMERAL;
+            }
+
+            @Override
+            protected boolean isProtected()
+            {
+                return true;
+            }
+        },
+
+        /**
+         * Same as {@link CreateMode#EPHEMERAL_SEQUENTIAL} with protection
+         */
+        PROTECTED_EPHEMERAL_SEQUENTIAL()
+        {
+            @Override
+            protected CreateMode getCreateMode(boolean pathIsSet)
+            {
+                return pathIsSet ? CreateMode.EPHEMERAL : CreateMode.EPHEMERAL_SEQUENTIAL;
+            }
+
+            @Override
+            protected boolean isProtected()
+            {
+                return true;
+            }
+        }
+
+        ;
+
+        protected abstract CreateMode getCreateMode(boolean pathIsSet);
+
+        protected abstract boolean isProtected();
+    }
+
+    /**
+     * @param client client instance
+     * @param mode creation/protection mode
+     * @param basePath the base path for the node
+     * @param data data for the node
+     */
+    public PersistentEphemeralNode(CuratorFramework client, Mode mode, String basePath, byte[] data)
+    {
+        this.client = Preconditions.checkNotNull(client, "client cannot be null");
+        this.basePath = Preconditions.checkNotNull(basePath, "basePath cannot be null");
+        this.mode = Preconditions.checkNotNull(mode, "mode cannot be null");
+        data = Preconditions.checkNotNull(data, "data cannot be null");
+
+        String parentDir = ZKPaths.getPathAndNode(basePath).getPath();
+        ensurePath = client.newNamespaceAwareEnsurePath(parentDir);
+
+        backgroundCallback = new BackgroundCallback()
+        {
+            @Override
+            public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+            {
+                String      path = null;
+                if ( event.getResultCode() == KeeperException.Code.NODEEXISTS.intValue() )
+                {
+                    path = event.getPath();
+                }
+                else if ( event.getResultCode() == KeeperException.Code.OK.intValue() )
+                {
+                    path = event.getName();
+                }
+                if ( path != null )
+                {
+                    nodePath.set(path);
+                    watchNode();
+
+                    CountDownLatch localLatch = initialCreateLatch;
+                    initialCreateLatch = null;
+                    if ( localLatch != null )
+                    {
+                        localLatch.countDown();
+                    }
+                }
+                else
+                {
+                    createNode();
+                }
+            }
+        };
+
+        createMethod = mode.isProtected() ? client.create().withProtection() : client.create();
+        this.data = Arrays.copyOf(data, data.length);
+    }
+
+    /**
+     * You must call start() to initiate the persistent ephemeral node. An attempt to create the node
+     * in the background will be started
+     */
+    public void start()
+    {
+        Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Already started");
+
+        client.getConnectionStateListenable().addListener(listener);
+        createNode();
+    }
+
+    /**
+     * Block until the either initial node creation initiated by {@link #start()} succeeds or
+     * the timeout elapses.
+     *
+     * @param timeout the maximum time to wait
+     * @param unit time unit
+     * @return if the node was created before timeout
+     * @throws InterruptedException if the thread is interrupted
+     */
+    public boolean waitForInitialCreate(long timeout, TimeUnit unit) throws InterruptedException
+    {
+        Preconditions.checkState(state.get() == State.STARTED, "Not started");
+
+        return initialCreateLatch.await(timeout, unit);
+    }
+
+    @Override
+    public void close()
+    {
+        if ( !state.compareAndSet(State.STARTED, State.CLOSED) )
+        {
+            return;
+        }
+
+        client.getConnectionStateListenable().removeListener(listener);
+
+        deleteNode();
+    }
+
+    /**
+     * Returns the currently set path or null if the node does not exist
+     *
+     * @return node path or null
+     */
+    public String getActualPath()
+    {
+        return nodePath.get();
+    }
+
+    private void deleteNode()
+    {
+        String          localNodePath = nodePath.getAndSet(null);
+        if ( localNodePath != null )
+        {
+            try
+            {
+                client.delete().guaranteed().forPath(localNodePath);
+            }
+            catch ( KeeperException.NoNodeException ignore )
+            {
+                // ignore
+            }
+            catch ( Exception e )
+            {
+                log.error("Deleting node: " + localNodePath, e);
+            }
+        }
+    }
+
+    private void createNode()
+    {
+        if ( !isActive() )
+        {
+            return;
+        }
+
+        try
+        {
+            String      existingPath = nodePath.get();
+            String      createPath = (existingPath != null) ? existingPath : basePath;
+            ensurePath.ensure(client.getZookeeperClient());
+            createMethod.withMode(mode.getCreateMode(existingPath != null)).inBackground(backgroundCallback).forPath(createPath, data);
+        }
+        catch ( Exception e )
+        {
+            log.error("Creating node. BasePath: " + basePath, e);
+        }
+    }
+
+    private void watchNode()
+    {
+        if ( !isActive() )
+        {
+            return;
+        }
+
+        String          localNodePath = nodePath.get();
+        if ( localNodePath != null )
+        {
+            try
+            {
+                client.checkExists().usingWatcher(watcher).inBackground(checkExistsCallback).forPath(localNodePath);
+            }
+            catch ( Exception e )
+            {
+                log.error("Watching node: " + localNodePath, e);
+            }
+        }
+    }
+
+    private boolean isActive()
+    {
+        return (state.get() == State.STARTED) && !isSuspended.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/f074683b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..033ec1f
--- /dev/null
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
@@ -0,0 +1,361 @@
+package org.apache.curator.framework.recipes.nodes;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.recipes.BaseClassForTests;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.KillSession;
+import org.apache.curator.test.Timing;
+import org.apache.curator.utils.ZKPaths;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.testng.Assert.*;
+
+public class TestPersistentEphemeralNode extends BaseClassForTests
+{
+    private static final String DIR = "/test";
+    private static final String PATH = ZKPaths.makePath(DIR, "/foo");
+
+    private final Collection<CuratorFramework> curatorInstances = Lists.newArrayList();
+    private final Collection<PersistentEphemeralNode> createdNodes = Lists.newArrayList();
+
+    @AfterMethod
+    public void teardown() throws Exception
+    {
+        for ( PersistentEphemeralNode node : createdNodes )
+        {
+            node.close();
+        }
+
+        for ( CuratorFramework curator : curatorInstances )
+        {
+            curator.close();
+        }
+
+        super.teardown();
+    }
+
+    @Test(expectedExceptions = NullPointerException.class)
+    public void testNullCurator() throws Exception
+    {
+        new PersistentEphemeralNode(null, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+    }
+
+    @Test(expectedExceptions = NullPointerException.class)
+    public void testNullPath() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, null, new byte[0]);
+    }
+
+    @Test(expectedExceptions = NullPointerException.class)
+    public void testNullData() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, null);
+    }
+
+    @Test(expectedExceptions = NullPointerException.class)
+    public void testNullMode() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        new PersistentEphemeralNode(curator, null, PATH, new byte[0]);
+    }
+
+    @Test
+    public void testDeletesNodeWhenClosed() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.start();
+        String path = null;
+        try
+        {
+            node.waitForInitialCreate(5, TimeUnit.SECONDS);
+            path = node.getActualPath();
+            assertNodeExists(curator, path);
+        }
+        finally
+        {
+            node.close();  // After closing the path is set to null...
+        }
+
+        assertNodeDoesNotExist(curator, path);
+    }
+
+    @Test
+    public void testClosingMultipleTimes() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.start();
+        node.waitForInitialCreate(5, TimeUnit.SECONDS);
+
+        String path = node.getActualPath();
+        node.close();
+        assertNodeDoesNotExist(curator, path);
+
+        node.close();
+        assertNodeDoesNotExist(curator, path);
+    }
+
+    @Test
+    public void testDeletesNodeWhenSessionDisconnects() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        CuratorFramework observer = newCurator();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.start();
+        try
+        {
+            node.waitForInitialCreate(5, TimeUnit.SECONDS);
+            assertNodeExists(observer, node.getActualPath());
+
+            // Register a watch that will fire when the node is deleted...
+            Trigger deletedTrigger = Trigger.deleted();
+            observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
+
+            killSession(curator);
+
+            // Make sure the node got deleted
+            assertTrue(deletedTrigger.firedWithin(10, TimeUnit.SECONDS));
+        }
+        finally
+        {
+            node.close();
+        }
+    }
+
+    @Test
+    public void testRecreatesNodeWhenSessionReconnects() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        CuratorFramework observer = newCurator();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.start();
+        try
+        {
+            node.waitForInitialCreate(5, TimeUnit.SECONDS);
+            assertNodeExists(observer, node.getActualPath());
+
+            Trigger deletedTrigger = Trigger.deleted();
+            observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath());
+
+            killSession(curator);
+
+            // Make sure the node got deleted...
+            assertTrue(deletedTrigger.firedWithin(10, TimeUnit.SECONDS));
+
+            // Check for it to be recreated...
+            Trigger createdTrigger = Trigger.created();
+            Stat stat = observer.checkExists().usingWatcher(createdTrigger).forPath(node.getActualPath());
+            assertTrue(stat != null || createdTrigger.firedWithin(10, TimeUnit.SECONDS));
+        }
+        finally
+        {
+            node.close();
+        }
+    }
+
+    @Test
+    public void testRecreatesNodeWhenSessionReconnectsMultipleTimes() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        CuratorFramework observer = newCurator();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.start();
+        try
+        {
+            node.waitForInitialCreate(5, TimeUnit.SECONDS);
+            String path = node.getActualPath();
+            assertNodeExists(observer, path);
+
+            // 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);
+
+                // Kill the session, thus cleaning up the node...
+                killSession(curator);
+
+                // Make sure the node ended up getting deleted...
+                assertTrue(deletionTrigger.firedWithin(10, TimeUnit.SECONDS));
+
+                // 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);
+                assertTrue(stat != null || creationTrigger.firedWithin(10, TimeUnit.SECONDS));
+            }
+        }
+        finally
+        {
+            node.close();
+        }
+    }
+
+    @Test
+    public void testRecreatesNodeWhenItGetsDeleted() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]);
+        node.start();
+        try
+        {
+            node.waitForInitialCreate(5, TimeUnit.SECONDS);
+            String originalNode = node.getActualPath();
+            assertNodeExists(curator, originalNode);
+
+            // 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(10, TimeUnit.SECONDS));
+        }
+        finally
+        {
+            node.close();
+        }
+    }
+
+    @Test
+    public void testNodesCreateUniquePaths() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+
+        PersistentEphemeralNode node1 = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL_SEQUENTIAL, PATH, new byte[0]);
+        node1.start();
+        try
+        {
+            node1.waitForInitialCreate(5, TimeUnit.SECONDS);
+            String path1 = node1.getActualPath();
+
+            PersistentEphemeralNode node2 = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL_SEQUENTIAL, PATH, new byte[0]);
+            node2.start();
+            try
+            {
+                node2.waitForInitialCreate(5, TimeUnit.SECONDS);
+                String path2 = node2.getActualPath();
+
+                assertFalse(path1.equals(path2));
+            }
+            finally
+            {
+                node2.close();
+            }
+        }
+        finally
+        {
+            node1.close();
+        }
+    }
+
+    @Test
+    public void testData() throws Exception
+    {
+        CuratorFramework curator = newCurator();
+        byte[] data = "Hello World".getBytes();
+
+        PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, data);
+        node.start();
+        try
+        {
+            node.waitForInitialCreate(5, TimeUnit.SECONDS);
+            assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), data));
+        }
+        finally
+        {
+            node.close();
+        }
+    }
+
+    private void assertNodeExists(CuratorFramework curator, String path) throws Exception
+    {
+        assertNotNull(path);
+        assertTrue(curator.checkExists().forPath(path) != null);
+    }
+
+    private void assertNodeDoesNotExist(CuratorFramework curator, String path) throws Exception
+    {
+        assertTrue(curator.checkExists().forPath(path) == null);
+    }
+
+    private CuratorFramework newCurator() throws IOException
+    {
+        Timing timing = new Timing();
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
+        client.start();
+
+        curatorInstances.add(client);
+        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 CountDownLatch latch;
+
+        public Trigger(Event.EventType type)
+        {
+            assertNotNull(type);
+
+            this.type = type;
+            this.latch = new CountDownLatch(1);
+        }
+
+        @Override
+        public void process(WatchedEvent event)
+        {
+            if ( type == event.getType() )
+            {
+                latch.countDown();
+            }
+        }
+
+        public boolean firedWithin(long duration, TimeUnit unit)
+        {
+            try
+            {
+                return latch.await(duration, unit);
+            } catch ( InterruptedException e )
+            {
+                throw Throwables.propagate(e);
+            }
+        }
+
+        private static Trigger created()
+        {
+            return new Trigger(Event.EventType.NodeCreated);
+        }
+
+        private static Trigger deleted()
+        {
+            return new Trigger(Event.EventType.NodeDeleted);
+        }
+    }
+}
\ No newline at end of file


[10/14] git commit: CURATOR-35 - added a full featured sync() method and deprecated the previous method

Posted by ra...@apache.org.
CURATOR-35 - added a full featured sync() method and deprecated the previous method


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

Branch: refs/heads/master
Commit: d9d0dc2fdb63ae0f2754144a250348774454d701
Parents: 6a26d4f
Author: randgalt <ra...@apache.org>
Authored: Tue Jun 18 14:11:42 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Tue Jun 18 14:11:42 2013 -0700

----------------------------------------------------------------------
 .../curator/framework/CuratorFramework.java     | 29 ++++--
 .../curator/framework/api/SyncBuilder.java      | 24 +++++
 .../framework/imps/CuratorFrameworkImpl.java    |  6 ++
 .../curator/framework/imps/SyncBuilderImpl.java | 92 ++++++++++++++++++++
 .../curator/framework/imps/TestFramework.java   | 31 +++++++
 5 files changed, 173 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/d9d0dc2f/curator-framework/src/main/java/org/apache/curator/framework/CuratorFramework.java
----------------------------------------------------------------------
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 0ca9b23..e49386e 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.framework;
 
 import org.apache.curator.CuratorZookeeperClient;
@@ -35,12 +36,12 @@ public interface CuratorFramework extends Closeable
     /**
      * Start the client. Most mutator methods will not work until the client is started
      */
-    public void     start();
+    public void start();
 
     /**
      * Stop the client
      */
-    public void     close();
+    public void close();
 
     /**
      * Returns the state of this instance
@@ -55,7 +56,7 @@ public interface CuratorFramework extends Closeable
      * @return true/false
      * @deprecated use {@link #getState()} instead
      */
-    public boolean  isStarted();
+    public boolean isStarted();
 
     /**
      * Start a create builder
@@ -73,7 +74,7 @@ public interface CuratorFramework extends Closeable
 
     /**
      * Start an exists builder
-     *
+     * <p/>
      * The builder will return a Stat object as if org.apache.zookeeper.ZooKeeper.exists() were called.  Thus, a null
      * means that it does not exist and an actual Stat object means it does exist.
      *
@@ -126,10 +127,19 @@ public interface CuratorFramework extends Closeable
     /**
      * Perform a sync on the given path - syncs are always in the background
      *
-     * @param path the path
+     * @param path                    the path
      * @param backgroundContextObject optional context
+     * @deprecated use {@link #sync()} instead
+     */
+    public void sync(String path, Object backgroundContextObject);
+
+    /**
+     * Start a sync builder. Note: sync is ALWAYS in the background even
+     * if you don't use one of the background() methods
+     *
+     * @return builder object
      */
-    public void     sync(String path, Object backgroundContextObject);
+    public SyncBuilder sync();
 
     /**
      * Returns the listenable interface for the Connect State
@@ -143,14 +153,14 @@ public interface CuratorFramework extends Closeable
      *
      * @return listenable
      */
-    public Listenable<CuratorListener>         getCuratorListenable();
+    public Listenable<CuratorListener> getCuratorListenable();
 
     /**
      * Returns the listenable interface for unhandled errors
      *
      * @return listenable
      */
-    public Listenable<UnhandledErrorListener>  getUnhandledErrorListenable();
+    public Listenable<UnhandledErrorListener> getUnhandledErrorListenable();
 
     /**
      * Returns a facade of the current instance that does _not_ automatically
@@ -172,6 +182,7 @@ public interface CuratorFramework extends Closeable
 
     /**
      * Return the current namespace or "" if none
+     *
      * @return namespace
      */
     public String getNamespace();
@@ -189,5 +200,5 @@ public interface CuratorFramework extends Closeable
      * @param path path to ensure
      * @return new EnsurePath instance
      */
-    public EnsurePath    newNamespaceAwareEnsurePath(String path);
+    public EnsurePath newNamespaceAwareEnsurePath(String path);
 }

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/d9d0dc2f/curator-framework/src/main/java/org/apache/curator/framework/api/SyncBuilder.java
----------------------------------------------------------------------
diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/SyncBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/SyncBuilder.java
new file mode 100644
index 0000000..6ae3586
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/api/SyncBuilder.java
@@ -0,0 +1,24 @@
+/**
+ * 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 SyncBuilder extends
+    BackgroundPathable<Void>
+{
+}

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/d9d0dc2f/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java
----------------------------------------------------------------------
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 28cad83..c3b0992 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
@@ -411,6 +411,12 @@ public class CuratorFrameworkImpl implements CuratorFramework
         internalSync(this, path, context);
     }
 
+    @Override
+    public SyncBuilder sync()
+    {
+        return new SyncBuilderImpl(this);
+    }
+
     protected void internalSync(CuratorFrameworkImpl impl, String path, Object context)
     {
         BackgroundOperation<String> operation = new BackgroundSyncImpl(impl, context);

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/d9d0dc2f/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7c74abc
--- /dev/null
+++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/SyncBuilderImpl.java
@@ -0,0 +1,92 @@
+package org.apache.curator.framework.imps;
+
+import org.apache.curator.TimeTrace;
+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.Pathable;
+import org.apache.curator.framework.api.SyncBuilder;
+import org.apache.zookeeper.AsyncCallback;
+import java.util.concurrent.Executor;
+
+public class SyncBuilderImpl implements SyncBuilder, BackgroundOperation<String>
+{
+    private final CuratorFrameworkImpl client;
+    private Backgrounding backgrounding = new Backgrounding();
+
+    public SyncBuilderImpl(CuratorFrameworkImpl client)
+    {
+        //To change body of created methods use File | Settings | File Templates.
+        this.client = client;
+    }
+
+    @Override
+    public Pathable<Void> inBackground()
+    {
+        // NOP always in background
+        return this;
+    }
+
+    @Override
+    public Pathable<Void> inBackground(Object context)
+    {
+        backgrounding = new Backgrounding(context);
+        return this;
+    }
+
+    @Override
+    public Pathable<Void> inBackground(BackgroundCallback callback)
+    {
+        backgrounding = new Backgrounding(callback);
+        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, Executor executor)
+    {
+        backgrounding = new Backgrounding(callback, executor);
+        return this;
+    }
+
+    @Override
+    public Pathable<Void> inBackground(BackgroundCallback callback, Object context, Executor executor)
+    {
+        backgrounding = new Backgrounding(client, callback, context, executor);
+        return this;
+    }
+
+    @Override
+    public void performBackgroundOperation(final OperationAndData<String> operationAndData) throws Exception
+    {
+        final TimeTrace trace = client.getZookeeperClient().startTracer("SyncBuilderImpl-Background");
+        final String path = operationAndData.getData();
+        String adjustedPath = client.fixForNamespace(path);
+
+        AsyncCallback.VoidCallback voidCallback = new AsyncCallback.VoidCallback()
+        {
+            @Override
+            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);
+                client.processBackgroundOperation(operationAndData, event);
+            }
+        };
+        client.getZooKeeper().sync(adjustedPath, voidCallback, backgrounding.getContext());
+    }
+
+    @Override
+    public Void forPath(String path) throws Exception
+    {
+        OperationAndData<String> operationAndData = new OperationAndData<String>(this, path, backgrounding.getCallback(), null);
+        client.processBackgroundOperation(operationAndData, null);
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/d9d0dc2f/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 a3bf73b..f7ef6e9 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
@@ -427,6 +427,37 @@ public class TestFramework extends BaseClassForTests
     }
 
     @Test
+    public void     testSyncNew() throws Exception
+    {
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));
+        client.start();
+        try
+        {
+            client.create().forPath("/head");
+            Assert.assertNotNull(client.checkExists().forPath("/head"));
+
+            final CountDownLatch      latch = new CountDownLatch(1);
+            BackgroundCallback callback = new BackgroundCallback()
+            {
+                @Override
+                public void processResult(CuratorFramework client, CuratorEvent event) throws Exception
+                {
+                    if ( event.getType() == CuratorEventType.SYNC )
+                    {
+                        latch.countDown();
+                    }
+                }
+            };
+            client.sync().inBackground(callback).forPath("/head");
+            Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
+        }
+        finally
+        {
+            client.close();
+        }
+    }
+
+    @Test
     public void     testBackgroundDelete() throws Exception
     {
         CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), new RetryOneTime(1));


[05/14] git commit: Merge branch 'master' into CURATOR-25

Posted by ra...@apache.org.
Merge branch 'master' into CURATOR-25


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

Branch: refs/heads/master
Commit: f7e75e00e28b24bb5cce8fb84b494afddb7560ac
Parents: 53fd857 b766eb7
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 13:26:26 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 13:26:26 2013 -0700

----------------------------------------------------------------------
 pom.xml | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------



[04/14] git commit: Merge branch 'CURATOR-25' into 2.0.2-incubating

Posted by ra...@apache.org.
Merge branch 'CURATOR-25' into 2.0.2-incubating


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

Branch: refs/heads/master
Commit: e5c7755c9f5d7e1cf0cb3f29545b875d6bd55263
Parents: 0c1af3b 53fd857
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 08:50:37 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 08:50:37 2013 -0700

----------------------------------------------------------------------
 .../recipes/nodes/PersistentEphemeralNode.java  | 349 ++++++++++++++++++
 .../src/site/confluence/index.confluence        |  17 +-
 .../persistent-ephemeral-node.confluence        |  37 ++
 .../nodes/TestPersistentEphemeralNode.java      | 361 +++++++++++++++++++
 4 files changed, 757 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[06/14] git commit: Added setData() method

Posted by ra...@apache.org.
Added setData() method


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

Branch: refs/heads/master
Commit: 415ec2e8500e3a47ef1232c927e17f3e5cf2ef9a
Parents: f7e75e0
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 13:56:43 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 13:56:43 2013 -0700

----------------------------------------------------------------------
 .../recipes/nodes/PersistentEphemeralNode.java  | 22 ++++++--
 .../nodes/TestPersistentEphemeralNode.java      | 53 +++++++++++++++++++-
 2 files changed, 71 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/415ec2e8/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
----------------------------------------------------------------------
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
index d86af92..7e92bdf 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java
@@ -47,7 +47,7 @@ public class PersistentEphemeralNode implements Closeable
     private final AtomicReference<String>   nodePath = new AtomicReference<String>(null);
     private final String                    basePath;
     private final Mode                      mode;
-    private final byte[]                    data;
+    private final AtomicReference<byte[]>   data = new AtomicReference<byte[]>();
     private final AtomicReference<State>    state = new AtomicReference<State>(State.LATENT);
     private final AtomicBoolean             isSuspended = new AtomicBoolean(false);
     private final BackgroundCallback        backgroundCallback;
@@ -227,7 +227,7 @@ public class PersistentEphemeralNode implements Closeable
         };
 
         createMethod = mode.isProtected() ? client.create().withProtection() : client.create();
-        this.data = Arrays.copyOf(data, data.length);
+        this.data.set(Arrays.copyOf(data, data.length));
     }
 
     /**
@@ -281,6 +281,22 @@ public class PersistentEphemeralNode implements Closeable
         return nodePath.get();
     }
 
+    /**
+     * Set data that ephemeral node should set in ZK also writes the data to the node
+     *
+     * @param data new data value
+     * @throws Exception errors
+     */
+    public void setData(byte[] data) throws Exception
+    {
+        data = Preconditions.checkNotNull(data, "data cannot be null");
+        this.data.set(Arrays.copyOf(data, data.length));
+        if ( isActive() )
+        {
+            client.setData().inBackground().forPath(basePath, this.data.get());
+        }
+    }
+
     private void deleteNode()
     {
         String          localNodePath = nodePath.getAndSet(null);
@@ -313,7 +329,7 @@ public class PersistentEphemeralNode implements Closeable
             String      existingPath = nodePath.get();
             String      createPath = (existingPath != null) ? existingPath : basePath;
             ensurePath.ensure(client.getZookeeperClient());
-            createMethod.withMode(mode.getCreateMode(existingPath != null)).inBackground(backgroundCallback).forPath(createPath, data);
+            createMethod.withMode(mode.getCreateMode(existingPath != null)).inBackground(backgroundCallback).forPath(createPath, data.get());
         }
         catch ( Exception e )
         {

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/415ec2e8/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java
----------------------------------------------------------------------
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 033ec1f..84c1cf7 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
@@ -12,12 +12,14 @@ import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
+import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
 import static org.testng.Assert.*;
@@ -74,6 +76,54 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
     }
 
     @Test
+    public void testSettingData() throws Exception
+    {
+        PersistentEphemeralNode node = null;
+        Timing timing = new Timing();
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1));
+        try
+        {
+            client.start();
+            node = new PersistentEphemeralNode(client, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, "a".getBytes());
+            node.start();
+            Assert.assertTrue(node.waitForInitialCreate(5, TimeUnit.SECONDS));
+
+            Assert.assertEquals(node.getActualPath(), PATH);
+            Assert.assertEquals(client.getData().forPath(PATH), "a".getBytes());
+
+            final Semaphore semaphore = new Semaphore(0);
+            Watcher watcher = new Watcher()
+            {
+                @Override
+                public void process(WatchedEvent arg0)
+                {
+                    semaphore.release();
+                }
+            };
+            client.checkExists().usingWatcher(watcher).forPath(PATH);
+            node.setData("b".getBytes());
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            Assert.assertEquals(node.getActualPath(), PATH);
+            Assert.assertEquals(client.getData().usingWatcher(watcher).forPath(PATH), "b".getBytes());
+            node.setData("c".getBytes());
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            Assert.assertEquals(node.getActualPath(), PATH);
+            Assert.assertEquals(client.getData().usingWatcher(watcher).forPath(PATH), "c".getBytes());
+            node.close();
+            Assert.assertTrue(timing.acquireSemaphore(semaphore));
+            Assert.assertTrue(client.checkExists().forPath(PATH) == null);
+        }
+        finally
+        {
+            if ( node != null )
+            {
+                node.close();
+            }
+            client.close();
+        }
+    }
+
+    @Test
     public void testDeletesNodeWhenClosed() throws Exception
     {
         CuratorFramework curator = newCurator();
@@ -342,7 +392,8 @@ public class TestPersistentEphemeralNode extends BaseClassForTests
             try
             {
                 return latch.await(duration, unit);
-            } catch ( InterruptedException e )
+            }
+            catch ( InterruptedException e )
             {
                 throw Throwables.propagate(e);
             }


[08/14] git commit: Stub out additional JMX methods to avoid test-time errors

Posted by ra...@apache.org.
Stub out additional JMX methods to avoid test-time errors


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

Branch: refs/heads/master
Commit: c676e3e2d0e9fdc83f5fde1be1baeb808d00f9d7
Parents: e213214
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 15:37:05 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 15:37:05 2013 -0700

----------------------------------------------------------------------
 .../apache/curator/test/ByteCodeRewrite.java    | 26 +++++++++++++++-----
 1 file changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/c676e3e2/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java
----------------------------------------------------------------------
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
index 19d069a..eeca3d4 100644
--- a/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java
+++ b/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.test;
 
 import javassist.CannotCompileException;
@@ -26,7 +27,7 @@ import javassist.NotFoundException;
 
 public class ByteCodeRewrite
 {
-    public static void      apply()
+    public static void apply()
     {
         // NOP - only needed so that static initializer is run
     }
@@ -46,7 +47,7 @@ public class ByteCodeRewrite
             try
             {
                 CtClass cc = pool.get("org.apache.zookeeper.server.ZooKeeperServer");
-                fixMethods(cc);
+                fixMethods(cc, "registerJMX", "unregisterJMX");
             }
             catch ( NotFoundException ignore )
             {
@@ -56,7 +57,17 @@ public class ByteCodeRewrite
             try
             {
                 CtClass cc = pool.get("org.apache.zookeeper.server.quorum.LearnerZooKeeperServer");
-                fixMethods(cc);
+                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 )
             {
@@ -69,13 +80,16 @@ public class ByteCodeRewrite
         }
     }
 
-    private static void fixMethods(CtClass cc) throws CannotCompileException
+    private static void fixMethods(CtClass cc, String... methodNames) throws CannotCompileException
     {
         for ( CtMethod method : cc.getDeclaredMethods() )
         {
-            if ( method.getName().equals("registerJMX") || method.getName().equals("unregisterJMX") )
+            for ( String methodName : methodNames )
             {
-                method.setBody(null);
+                if ( method.getName().equals(methodName) )
+                {
+                    method.setBody(null);
+                }
             }
         }
         cc.toClass();


[12/14] git commit: 2.1.0-incubating

Posted by ra...@apache.org.
2.1.0-incubating


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

Branch: refs/heads/master
Commit: 2cb8b0e8656638677cc64be5f0210bdd325a8db4
Parents: 133f846
Author: randgalt <ra...@apache.org>
Authored: Thu Jun 20 10:47:53 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Thu Jun 20 10:47:53 2013 -0700

----------------------------------------------------------------------
 curator-client/pom.xml             |  4 ++--
 curator-examples/pom.xml           |  4 ++--
 curator-framework/pom.xml          |  4 ++--
 curator-recipes/pom.xml            |  4 ++--
 curator-test/pom.xml               |  4 ++--
 curator-x-discovery-server/pom.xml |  4 ++--
 curator-x-discovery/pom.xml        |  4 ++--
 pom.xml                            | 14 +++++++-------
 8 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-client/pom.xml
----------------------------------------------------------------------
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 3c1aa75..5d67db6 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-client</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Client</name>
     <description>Low-level API</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-examples/pom.xml
----------------------------------------------------------------------
diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index 674409e..4728708 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-examples</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Examples</name>
     <description>Example usages of various Curator features.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-framework/pom.xml
----------------------------------------------------------------------
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index 74569a8..4b8f9c6 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-framework</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Framework</name>
     <description>High-level API that greatly simplifies using ZooKeeper.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-recipes/pom.xml
----------------------------------------------------------------------
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index 35e57ce..b121931 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-recipes</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Recipes</name>
     <description>All of the recipes listed on the ZooKeeper recipes doc (except two phase commit).</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 4a781ad..287a439 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-test</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Testing</name>
     <description>Unit testing utilities.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-x-discovery-server/pom.xml
----------------------------------------------------------------------
diff --git a/curator-x-discovery-server/pom.xml b/curator-x-discovery-server/pom.xml
index f943ead..c29373b 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-x-discovery-server</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Service Discovery Server</name>
     <description>Bridges non-Java or legacy applications with the Curator Service Discovery.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/curator-x-discovery/pom.xml
----------------------------------------------------------------------
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 2a3b33b..2bf3b08 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.0.2-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-x-discovery</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Service Discovery</name>
     <description>A service discovery recipe.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/2cb8b0e8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 29770b5..5995bfe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.curator</groupId>
     <artifactId>apache-curator</artifactId>
-    <version>2.0.2-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
     <packaging>pom</packaging>
 
     <name>Apache Curator</name>
@@ -229,37 +229,37 @@
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-client</artifactId>
-                <version>2.0.2-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-framework</artifactId>
-                <version>2.0.2-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-recipes</artifactId>
-                <version>2.0.2-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-test</artifactId>
-                <version>2.0.2-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-x-discovery</artifactId>
-                <version>2.0.2-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-x-discovery-server</artifactId>
-                <version>2.0.2-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
         </dependencies>
     </dependencyManagement>


[14/14] git commit: [maven-release-plugin] prepare release apache-curator-2.1.0-incubating

Posted by ra...@apache.org.
[maven-release-plugin] prepare release apache-curator-2.1.0-incubating


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

Branch: refs/heads/master
Commit: a95842651cf1fadcf8612601a138ac8247a14ad6
Parents: 3437221
Author: randgalt <ra...@apache.org>
Authored: Thu Jun 20 10:50:42 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Thu Jun 20 10:50:42 2013 -0700

----------------------------------------------------------------------
 curator-client/pom.xml             |  4 ++--
 curator-examples/pom.xml           |  4 ++--
 curator-framework/pom.xml          |  4 ++--
 curator-recipes/pom.xml            |  4 ++--
 curator-test/pom.xml               |  4 ++--
 curator-x-discovery-server/pom.xml |  4 ++--
 curator-x-discovery/pom.xml        |  4 ++--
 pom.xml                            | 16 ++++++++--------
 8 files changed, 22 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-client/pom.xml
----------------------------------------------------------------------
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 947f54e..5d67db6 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-client</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Client</name>
     <description>Low-level API</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-examples/pom.xml
----------------------------------------------------------------------
diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index f541fcb..4728708 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-examples</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Examples</name>
     <description>Example usages of various Curator features.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-framework/pom.xml
----------------------------------------------------------------------
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index 83b3952..4b8f9c6 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-framework</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Framework</name>
     <description>High-level API that greatly simplifies using ZooKeeper.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-recipes/pom.xml
----------------------------------------------------------------------
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index 0557e65..b121931 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-recipes</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Recipes</name>
     <description>All of the recipes listed on the ZooKeeper recipes doc (except two phase commit).</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-test/pom.xml
----------------------------------------------------------------------
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 6d713ac..287a439 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-test</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Testing</name>
     <description>Unit testing utilities.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-x-discovery-server/pom.xml
----------------------------------------------------------------------
diff --git a/curator-x-discovery-server/pom.xml b/curator-x-discovery-server/pom.xml
index 00d1033..c29373b 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-x-discovery-server</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Service Discovery Server</name>
     <description>Bridges non-Java or legacy applications with the Curator Service Discovery.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/curator-x-discovery/pom.xml
----------------------------------------------------------------------
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index bee8ea6..2bf3b08 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -24,12 +24,12 @@
     <parent>
         <groupId>org.apache.curator</groupId>
         <artifactId>apache-curator</artifactId>
-        <version>2.1.0-incubating-SNAPSHOT</version>
+        <version>2.1.0-incubating</version>
     </parent>
 
     <groupId>org.apache.curator</groupId>
     <artifactId>curator-x-discovery</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
 
     <name>Curator Service Discovery</name>
     <description>A service discovery recipe.</description>

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/a9584265/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d2fad2a..6717817 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.curator</groupId>
     <artifactId>apache-curator</artifactId>
-    <version>2.1.0-incubating-SNAPSHOT</version>
+    <version>2.1.0-incubating</version>
     <packaging>pom</packaging>
 
     <name>Apache Curator</name>
@@ -62,7 +62,7 @@
         <url>https://git-wip-us.apache.org/repos/asf?p=incubator-curator.git</url>
         <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-curator.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-curator.git</developerConnection>
-        <tag>HEAD</tag>
+        <tag>apache-curator-2.1.0-incubating</tag>
     </scm>
 
     <issueManagement>
@@ -229,37 +229,37 @@
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-client</artifactId>
-                <version>2.1.0-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-framework</artifactId>
-                <version>2.1.0-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-recipes</artifactId>
-                <version>2.1.0-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-test</artifactId>
-                <version>2.1.0-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-x-discovery</artifactId>
-                <version>2.1.0-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-x-discovery-server</artifactId>
-                <version>2.1.0-incubating-SNAPSHOT</version>
+                <version>2.1.0-incubating</version>
             </dependency>
         </dependencies>
     </dependencyManagement>


[07/14] git commit: Merge branch 'CURATOR-25' into 2.0.2-incubating

Posted by ra...@apache.org.
Merge branch 'CURATOR-25' into 2.0.2-incubating


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

Branch: refs/heads/master
Commit: e2132143432c089681e962413d1afc56eaada87a
Parents: e5c7755 415ec2e
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 13:57:03 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 13:57:03 2013 -0700

----------------------------------------------------------------------
 .../recipes/nodes/PersistentEphemeralNode.java  | 22 ++++++--
 .../nodes/TestPersistentEphemeralNode.java      | 53 +++++++++++++++++++-
 pom.xml                                         | 11 ++++
 3 files changed, 82 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[03/14] git commit: Added doc

Posted by ra...@apache.org.
Added doc


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

Branch: refs/heads/master
Commit: 53fd85714d3d6070bc1febb4ce1c318de72755b2
Parents: f074683
Author: randgalt <ra...@apache.org>
Authored: Mon Jun 10 08:50:26 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Mon Jun 10 08:50:26 2013 -0700

----------------------------------------------------------------------
 .../src/site/confluence/index.confluence        | 17 +++++----
 .../persistent-ephemeral-node.confluence        | 37 ++++++++++++++++++++
 2 files changed, 47 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/53fd8571/curator-recipes/src/site/confluence/index.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/index.confluence b/curator-recipes/src/site/confluence/index.confluence
index b5c5435..c37b0de 100644
--- a/curator-recipes/src/site/confluence/index.confluence
+++ b/curator-recipes/src/site/confluence/index.confluence
@@ -13,13 +13,6 @@ Curator implements all of the recipes listed on the ZooKeeper recipes doc (excep
 |[[Shared Semaphore|shared-semaphore.html]] - A counting semaphore that works across JVMs. All processes in all JVMs that use the same lock path will achieve an inter-process limited set of leases. Further, this semaphore is mostly "fair" - each user will get a lease in the order requested (from ZK's point of view).|
 |[[Multi Shared Lock|multi-shared-lock.html]] - A container that manages multiple locks as a single entity. When acquire() is called, all the locks are acquired. If that fails, any paths that were acquired are released. Similarly, when release() is called, all locks are released (failures are ignored).|
 
-||Queues||
-|[[Distributed Queue|distributed-queue.html]] - An implementation of the Distributed Queue ZK recipe. Items put into the queue are guaranteed to be ordered (by means of ZK's PERSISTENT_SEQUENTIAL node). If a single consumer takes items out of the queue, they will be ordered FIFO. If ordering is important, use a LeaderSelector to nominate a single consumer.|
-|[[Distributed Id Queue|distributed-id-queue.html]] - A version of DistributedQueue that allows IDs to be associated with queue items. Items can then be removed from the queue if needed.|
-|[[Distributed Priority Queue|distributed-priority-queue.html]] - An implementation of the Distributed Priority Queue ZK recipe.|
-|[[Distributed Delay Queue|distributed-delay-queue.html]] - An implementation of a Distributed Delay Queue.|
-|[[Simple Distributed Queue|simple-distributed-queue.html]] - A drop-in replacement for the DistributedQueue that comes with the ZK distribution.|
-
 ||Barriers||
 |[[Barrier|barrier.html]] - Distributed systems use barriers to block processing of a set of nodes until a condition is met at which time all the nodes are allowed to proceed.|
 |[[Double Barrier|double-barrier.html]] - Double barriers enable clients to synchronize the beginning and the end of a computation. When enough processes have joined the barrier, processes start their computation and leave the barrier once they have finished.|
@@ -31,3 +24,13 @@ Curator implements all of the recipes listed on the ZooKeeper recipes doc (excep
 ||Caches||
 |[[Path Cache|path-cache.html]] - A Path Cache is used to watch a ZNode. Whenever a child is added, updated or removed, the Path Cache will change its state to contain the current set of children, the children's data and the children's state. Path caches in the Curator Framework are provided by the PathChildrenCache class. Changes to the path are passed to registered PathChildrenCacheListener instances.|
 |[[Node Cache|node-cache.html]] - A utility that attempts to keep the data from a node locally cached. This class will watch the node, respond to update/create/delete events, pull down the data, etc. You can register a listener that will get notified when changes occur.|
+
+||Nodes||
+|[[Persistent Ephemeral Node|persistent-ephemeral-node.html]] - An ephemeral node that attempts to stay present in ZooKeeper, even through connection and session interruptions..|
+
+||Queues||
+|[[Distributed Queue|distributed-queue.html]] - An implementation of the Distributed Queue ZK recipe. Items put into the queue are guaranteed to be ordered (by means of ZK's PERSISTENT_SEQUENTIAL node). If a single consumer takes items out of the queue, they will be ordered FIFO. If ordering is important, use a LeaderSelector to nominate a single consumer.|
+|[[Distributed Id Queue|distributed-id-queue.html]] - A version of DistributedQueue that allows IDs to be associated with queue items. Items can then be removed from the queue if needed.|
+|[[Distributed Priority Queue|distributed-priority-queue.html]] - An implementation of the Distributed Priority Queue ZK recipe.|
+|[[Distributed Delay Queue|distributed-delay-queue.html]] - An implementation of a Distributed Delay Queue.|
+|[[Simple Distributed Queue|simple-distributed-queue.html]] - A drop-in replacement for the DistributedQueue that comes with the ZK distribution.|

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/53fd8571/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
----------------------------------------------------------------------
diff --git a/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence b/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
new file mode 100644
index 0000000..0023f57
--- /dev/null
+++ b/curator-recipes/src/site/confluence/persistent-ephemeral-node.confluence
@@ -0,0 +1,37 @@
+h1. Persistent Ephemeral Node
+
+h2. Description
+A persistent ephemeral node is an ephemeral node that attempts to stay present in ZooKeeper, even through connection and session interruptions.
+
+h2. Participating Classes
+* PersistentEphemeralNode
+
+h2. Usage
+h3. Creating a PersistentEphemeralNode
+{code}
+public PersistentEphemeralNode(CuratorFramework client,
+                               PersistentEphemeralNode.Mode mode,
+                               String basePath,
+                               byte[] data)
+Parameters:
+client - client instance
+mode - creation/protection mode
+basePath - the base path for the node
+data - data for the node
+{code}
+
+h3. General Usage
+PersistentEphemeralNodes must be started:
+{code}
+node.start();
+{code}
+
+When you are through with the PersistentEphemeralNode instance, you should call close:
+{code}
+node.close();
+{code}
+
+NOTE: this will delete the node
+
+h2. Error Handling
+PersistentEphemeralNode instances internally handle all error states recreating the node as necessary.


[09/14] git commit: some additional functionality in testing code

Posted by ra...@apache.org.
some additional functionality in testing code


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

Branch: refs/heads/master
Commit: 6a26d4fa3ff431bbc0a5a166e4f92e8b3691f0f7
Parents: c676e3e
Author: randgalt <ra...@apache.org>
Authored: Sat Jun 15 16:39:14 2013 -0700
Committer: randgalt <ra...@apache.org>
Committed: Sat Jun 15 16:39:14 2013 -0700

----------------------------------------------------------------------
 .../org/apache/curator/test/TestingCluster.java | 66 +++++++++++++++++---
 .../curator/test/TestingQuorumPeerMain.java     |  6 ++
 .../curator/test/TestingZooKeeperMain.java      |  7 +++
 .../curator/test/TestingZooKeeperServer.java    | 12 ++++
 .../apache/curator/test/ZooKeeperMainFace.java  |  3 +
 5 files changed, 85 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/6a26d4fa/curator-test/src/main/java/org/apache/curator/test/TestingCluster.java
----------------------------------------------------------------------
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 442cb66..cd86b72 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
@@ -18,7 +18,12 @@
  */
 package org.apache.curator.test;
 
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 import org.apache.zookeeper.ZooKeeper;
 import java.io.Closeable;
 import java.io.IOException;
@@ -26,6 +31,7 @@ import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 
 /**
  * manages an internally running ensemble of ZooKeeper servers. FOR TESTING PURPOSES ONLY
@@ -37,7 +43,6 @@ public class TestingCluster implements Closeable
         ByteCodeRewrite.apply();
     }
 
-    private final QuorumConfigBuilder           builder;
     private final List<TestingZooKeeperServer>  servers;
 
     /**
@@ -58,7 +63,7 @@ public class TestingCluster implements Closeable
      */
     public TestingCluster(InstanceSpec... specs)
     {
-        this(ImmutableList.copyOf(specs));
+        this(listToMap(ImmutableList.copyOf(specs)));
     }
 
     /**
@@ -68,11 +73,25 @@ public class TestingCluster implements Closeable
      */
     public TestingCluster(Collection<InstanceSpec> specs)
     {
-        builder = new QuorumConfigBuilder(specs);
+        this(listToMap(specs));
+    }
+
+    /**
+     * Creates an ensemble using the given server specs
+     *
+     * @param specs map of an instance spec to its set of quorum instances. Allows simulation of an ensemble with instances
+     *              having different config peers
+     */
+    public TestingCluster(Map<InstanceSpec, Collection<InstanceSpec>> specs)
+    {
         ImmutableList.Builder<TestingZooKeeperServer> serverBuilder = ImmutableList.builder();
-        for ( int i = 0; i < specs.size(); ++i )
+        for ( Map.Entry<InstanceSpec, Collection<InstanceSpec>> entry : specs.entrySet() )
         {
-            serverBuilder.add(new TestingZooKeeperServer(builder, i));
+            List<InstanceSpec> instanceSpecs = Lists.newArrayList(entry.getValue());
+            int index = instanceSpecs.indexOf(entry.getKey());
+            Preconditions.checkState(index >= 0, entry.getKey() + " not found in specs");
+            QuorumConfigBuilder builder = new QuorumConfigBuilder(instanceSpecs);
+            serverBuilder.add(new TestingZooKeeperServer(builder, index));
         }
         servers = serverBuilder.build();
     }
@@ -84,7 +103,24 @@ public class TestingCluster implements Closeable
      */
     public Collection<InstanceSpec> getInstances()
     {
-        return builder.getInstanceSpecs();
+        Iterable<InstanceSpec> transformed = Iterables.transform
+        (
+            servers,
+            new Function<TestingZooKeeperServer, InstanceSpec>()
+            {
+                @Override
+                public InstanceSpec apply(TestingZooKeeperServer server)
+                {
+                    return server.getInstanceSpec();
+                }
+            }
+        );
+        return Lists.newArrayList(transformed);
+    }
+
+    public List<TestingZooKeeperServer> getServers()
+    {
+        return Lists.newArrayList(servers);
     }
 
     /**
@@ -95,7 +131,7 @@ public class TestingCluster implements Closeable
     public String   getConnectString()
     {
         StringBuilder       str = new StringBuilder();
-        for ( InstanceSpec spec : builder.getInstanceSpecs() )
+        for ( InstanceSpec spec : getInstances() )
         {
             if ( str.length() > 0 )
             {
@@ -211,13 +247,25 @@ public class TestingCluster implements Closeable
         return null;
     }
 
-    private static Collection<InstanceSpec> makeSpecs(int instanceQty)
+    private static Map<InstanceSpec, Collection<InstanceSpec>> makeSpecs(int instanceQty)
     {
         ImmutableList.Builder<InstanceSpec> builder = ImmutableList.builder();
         for ( int i = 0; i < instanceQty; ++i )
         {
             builder.add(InstanceSpec.newInstanceSpec());
         }
-        return builder.build();
+
+        return listToMap(builder.build());
+    }
+
+    private static Map<InstanceSpec, Collection<InstanceSpec>> listToMap(Collection<InstanceSpec> list)
+    {
+        ImmutableMap.Builder<InstanceSpec, Collection<InstanceSpec>> mapBuilder = ImmutableMap.builder();
+        for ( InstanceSpec spec : list )
+        {
+            mapBuilder.put(spec, list);
+        }
+
+        return mapBuilder.build();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/6a26d4fa/curator-test/src/main/java/org/apache/curator/test/TestingQuorumPeerMain.java
----------------------------------------------------------------------
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 0048099..9f9f302 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
@@ -53,6 +53,12 @@ class TestingQuorumPeerMain extends QuorumPeerMain implements ZooKeeperMainFace
     }
 
     @Override
+    public QuorumPeer getQuorumPeer()
+    {
+        return quorumPeer;
+    }
+
+    @Override
     public void close() throws IOException
     {
         if ( quorumPeer != null )

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/6a26d4fa/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java
----------------------------------------------------------------------
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 21bc57f..2193acb 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
@@ -23,6 +23,7 @@ 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.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -67,6 +68,12 @@ public class TestingZooKeeperMain extends ZooKeeperServerMain implements ZooKeep
         super.runFromConfig(serverConfig);
     }
 
+    @Override
+    public QuorumPeer getQuorumPeer()
+    {
+        throw new UnsupportedOperationException();
+    }
+
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
     @Override
     public void blockUntilStarted() throws Exception

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/6a26d4fa/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperServer.java
----------------------------------------------------------------------
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 b9e7dc1..be1a1b6 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
@@ -18,12 +18,14 @@
  */
 package org.apache.curator.test;
 
+import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 import org.apache.zookeeper.server.quorum.QuorumPeerMain;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
@@ -58,6 +60,16 @@ public class TestingZooKeeperServer extends QuorumPeerMain implements Closeable
         main = (configBuilder.size() > 1) ? new TestingQuorumPeerMain() : new TestingZooKeeperMain();
     }
 
+    public QuorumPeer getQuorumPeer()
+    {
+        return main.getQuorumPeer();
+    }
+
+    public Collection<InstanceSpec> getInstanceSpecs()
+    {
+        return configBuilder.getInstanceSpecs();
+    }
+
     public void     kill()
     {
         main.kill();

http://git-wip-us.apache.org/repos/asf/incubator-curator/blob/6a26d4fa/curator-test/src/main/java/org/apache/curator/test/ZooKeeperMainFace.java
----------------------------------------------------------------------
diff --git a/curator-test/src/main/java/org/apache/curator/test/ZooKeeperMainFace.java b/curator-test/src/main/java/org/apache/curator/test/ZooKeeperMainFace.java
index 946ac1a..6da29a4 100644
--- a/curator-test/src/main/java/org/apache/curator/test/ZooKeeperMainFace.java
+++ b/curator-test/src/main/java/org/apache/curator/test/ZooKeeperMainFace.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.test;
 
+import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 import java.io.Closeable;
 
@@ -28,4 +29,6 @@ public interface ZooKeeperMainFace extends Closeable
     public void blockUntilStarted() throws Exception;
 
     public void kill();
+
+    public QuorumPeer getQuorumPeer();
 }