[3/3] curator git commit: Merge branch 'master' into CURATOR-3.0

2015-08-24 Thread mdrob
Merge branch 'master' into CURATOR-3.0


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

Branch: refs/heads/CURATOR-3.0
Commit: f9af0ce81408f7a4cb4343fb0770e9e6ef545fda
Parents: 2fc9e37 061ed0a
Author: Mike Drob 
Authored: Mon Aug 24 07:10:20 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 07:10:20 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 26 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 ++
 2 files changed, 36 insertions(+), 14 deletions(-)
--




[2/3] curator git commit: CURATOR-218 Reorder ConnectionState process event

2015-08-24 Thread mdrob
CURATOR-218 Reorder ConnectionState process event

Address a race condition in ConnectionState.process where it will
trigger watchers first before updating its own state. This can lead to
inconsistencies when blocking until connected.


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

Branch: refs/heads/master
Commit: 061ed0a6d4630fd166df7ba3d16acde3a231c716
Parents: 2266ca1
Author: Mike Drob 
Authored: Wed Jul 8 11:04:30 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 07:08:16 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 26 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 ++
 2 files changed, 36 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-client/src/main/java/org/apache/curator/ConnectionState.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/ConnectionState.java 
b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
index d3900a1..46ae9fd 100644
--- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
+++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
@@ -41,7 +41,7 @@ class ConnectionState implements Watcher, Closeable
 {
 private static final int MAX_BACKGROUND_EXCEPTIONS = 10;
 private static final boolean LOG_EVENTS = 
Boolean.getBoolean(DebugUtils.PROPERTY_LOG_EVENTS);
-private final Logger log = LoggerFactory.getLogger(getClass());
+private static final Logger log = 
LoggerFactory.getLogger(ConnectionState.class);
 private final HandleHolder zooKeeper;
 private final AtomicBoolean isConnected = new AtomicBoolean(false);
 private final EnsembleProvider ensembleProvider;
@@ -145,24 +145,22 @@ class ConnectionState implements Watcher, Closeable
 log.debug("ConnectState watcher: " + event);
 }
 
-for ( Watcher parentWatcher : parentWatchers )
-{
-TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
-parentWatcher.process(event);
-timeTrace.commit();
-}
-
-boolean wasConnected = isConnected.get();
-boolean newIsConnected = wasConnected;
 if ( event.getType() == Watcher.Event.EventType.None )
 {
-newIsConnected = checkState(event.getState(), wasConnected);
+boolean wasConnected = isConnected.get();
+boolean newIsConnected = checkState(event.getState(), 
wasConnected);
+if ( newIsConnected != wasConnected )
+{
+isConnected.set(newIsConnected);
+connectionStartMs = System.currentTimeMillis();
+}
 }
 
-if ( newIsConnected != wasConnected )
+for ( Watcher parentWatcher : parentWatchers )
 {
-isConnected.set(newIsConnected);
-connectionStartMs = System.currentTimeMillis();
+TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
+parentWatcher.process(event);
+timeTrace.commit();
 }
 }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
--
diff --git 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
index f649afb..a6dc7ab 100644
--- 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
+++ 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
@@ -232,4 +232,28 @@ public class TestBlockUntilConnected extends 
BaseClassForTests
 CloseableUtils.closeQuietly(client);
 }
 }
+
+/**
+ * Test that we are actually connected every time that we block until 
connection is established in a tight loop.
+ */
+@Test
+public void testBlockUntilConnectedTightLoop() throws InterruptedException
+{
+CuratorFramework client;
+for(int i = 0 ; i < 50 ; i++)
+{
+client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTime(100));
+try
+{
+client.start();
+client.blockUntilConnected();
+
+Assert.assertTrue(client.getZookeeperCli

[1/3] curator git commit: CURATOR-218 Reorder ConnectionState process event

2015-08-24 Thread mdrob
Repository: curator
Updated Branches:
  refs/heads/CURATOR-218 [created] 061ed0a6d
  refs/heads/CURATOR-3.0 2fc9e37f6 -> f9af0ce81
  refs/heads/master 2266ca1fb -> 061ed0a6d


CURATOR-218 Reorder ConnectionState process event

Address a race condition in ConnectionState.process where it will
trigger watchers first before updating its own state. This can lead to
inconsistencies when blocking until connected.


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

Branch: refs/heads/CURATOR-3.0
Commit: 061ed0a6d4630fd166df7ba3d16acde3a231c716
Parents: 2266ca1
Author: Mike Drob 
Authored: Wed Jul 8 11:04:30 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 07:08:16 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 26 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 ++
 2 files changed, 36 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-client/src/main/java/org/apache/curator/ConnectionState.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/ConnectionState.java 
b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
index d3900a1..46ae9fd 100644
--- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
+++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
@@ -41,7 +41,7 @@ class ConnectionState implements Watcher, Closeable
 {
 private static final int MAX_BACKGROUND_EXCEPTIONS = 10;
 private static final boolean LOG_EVENTS = 
Boolean.getBoolean(DebugUtils.PROPERTY_LOG_EVENTS);
-private final Logger log = LoggerFactory.getLogger(getClass());
+private static final Logger log = 
LoggerFactory.getLogger(ConnectionState.class);
 private final HandleHolder zooKeeper;
 private final AtomicBoolean isConnected = new AtomicBoolean(false);
 private final EnsembleProvider ensembleProvider;
@@ -145,24 +145,22 @@ class ConnectionState implements Watcher, Closeable
 log.debug("ConnectState watcher: " + event);
 }
 
-for ( Watcher parentWatcher : parentWatchers )
-{
-TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
-parentWatcher.process(event);
-timeTrace.commit();
-}
-
-boolean wasConnected = isConnected.get();
-boolean newIsConnected = wasConnected;
 if ( event.getType() == Watcher.Event.EventType.None )
 {
-newIsConnected = checkState(event.getState(), wasConnected);
+boolean wasConnected = isConnected.get();
+boolean newIsConnected = checkState(event.getState(), 
wasConnected);
+if ( newIsConnected != wasConnected )
+{
+isConnected.set(newIsConnected);
+connectionStartMs = System.currentTimeMillis();
+}
 }
 
-if ( newIsConnected != wasConnected )
+for ( Watcher parentWatcher : parentWatchers )
 {
-isConnected.set(newIsConnected);
-connectionStartMs = System.currentTimeMillis();
+TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
+parentWatcher.process(event);
+timeTrace.commit();
 }
 }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
--
diff --git 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
index f649afb..a6dc7ab 100644
--- 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
+++ 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
@@ -232,4 +232,28 @@ public class TestBlockUntilConnected extends 
BaseClassForTests
 CloseableUtils.closeQuietly(client);
 }
 }
+
+/**
+ * Test that we are actually connected every time that we block until 
connection is established in a tight loop.
+ */
+@Test
+public void testBlockUntilConnectedTightLoop() throws InterruptedException
+{
+CuratorFramework client;
+for(int i = 0 ; i < 50 ; i++)
+{
+client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTi

[3/3] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-247

2015-08-24 Thread randgalt
Merge branch 'CURATOR-3.0' into CURATOR-247

Conflicts:
curator-client/src/main/java/org/apache/curator/ConnectionState.java


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

Branch: refs/heads/CURATOR-247
Commit: 1bbf301ab75debf8866c78f13a32e65049a20772
Parents: 81bab45 f9af0ce
Author: randgalt 
Authored: Mon Aug 24 08:02:08 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 08:02:08 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 36 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 +
 2 files changed, 41 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/1bbf301a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
--
diff --cc curator-client/src/main/java/org/apache/curator/ConnectionState.java
index 555a52d,46ae9fd..f0996d8
--- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
+++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
@@@ -44,10 -41,9 +44,10 @@@ class ConnectionState implements Watche
  {
  private static final int MAX_BACKGROUND_EXCEPTIONS = 10;
  private static final boolean LOG_EVENTS = 
Boolean.getBoolean(DebugUtils.PROPERTY_LOG_EVENTS);
- private final Logger log = LoggerFactory.getLogger(getClass());
+ private static final Logger log = 
LoggerFactory.getLogger(ConnectionState.class);
  private final HandleHolder zooKeeper;
  private final AtomicBoolean isConnected = new AtomicBoolean(false);
 +private final AtomicInteger lastNegotiatedSessionTimeoutMs = new 
AtomicInteger(0);
  private final EnsembleProvider ensembleProvider;
  private final int sessionTimeoutMs;
  private final int connectionTimeoutMs;
@@@ -156,29 -145,22 +156,27 @@@
  log.debug("ConnectState watcher: " + event);
  }
  
- for ( Watcher parentWatcher : parentWatchers )
- {
- TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
- parentWatcher.process(event);
- timeTrace.commit();
- }
- 
- boolean wasConnected = isConnected.get();
- boolean newIsConnected = wasConnected;
  if ( event.getType() == Watcher.Event.EventType.None )
  {
- newIsConnected = checkState(event.getState(), wasConnected);
+ boolean wasConnected = isConnected.get();
+ boolean newIsConnected = checkState(event.getState(), 
wasConnected);
+ if ( newIsConnected != wasConnected )
+ {
+ isConnected.set(newIsConnected);
+ connectionStartMs = System.currentTimeMillis();
++if ( newIsConnected )
++{
++
lastNegotiatedSessionTimeoutMs.set(zooKeeper.getNegotiatedSessionTimeoutMs());
++log.debug("Negotiated session timeout: " + 
lastNegotiatedSessionTimeoutMs.get());
++}
+ }
  }
  
- if ( newIsConnected != wasConnected )
+ for ( Watcher parentWatcher : parentWatchers )
  {
- isConnected.set(newIsConnected);
- connectionStartMs = System.currentTimeMillis();
- if ( newIsConnected )
- {
- 
lastNegotiatedSessionTimeoutMs.set(zooKeeper.getNegotiatedSessionTimeoutMs());
- log.debug("Negotiated session timeout: " + 
lastNegotiatedSessionTimeoutMs.get());
- }
+ TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
+ parentWatcher.process(event);
+ timeTrace.commit();
  }
  }
  

http://git-wip-us.apache.org/repos/asf/curator/blob/1bbf301a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
--



[1/3] curator git commit: CURATOR-218 Reorder ConnectionState process event

2015-08-24 Thread randgalt
Repository: curator
Updated Branches:
  refs/heads/CURATOR-247 81bab455c -> 1bbf301ab


CURATOR-218 Reorder ConnectionState process event

Address a race condition in ConnectionState.process where it will
trigger watchers first before updating its own state. This can lead to
inconsistencies when blocking until connected.


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

Branch: refs/heads/CURATOR-247
Commit: 061ed0a6d4630fd166df7ba3d16acde3a231c716
Parents: 2266ca1
Author: Mike Drob 
Authored: Wed Jul 8 11:04:30 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 07:08:16 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 26 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 ++
 2 files changed, 36 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-client/src/main/java/org/apache/curator/ConnectionState.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/ConnectionState.java 
b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
index d3900a1..46ae9fd 100644
--- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
+++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
@@ -41,7 +41,7 @@ class ConnectionState implements Watcher, Closeable
 {
 private static final int MAX_BACKGROUND_EXCEPTIONS = 10;
 private static final boolean LOG_EVENTS = 
Boolean.getBoolean(DebugUtils.PROPERTY_LOG_EVENTS);
-private final Logger log = LoggerFactory.getLogger(getClass());
+private static final Logger log = 
LoggerFactory.getLogger(ConnectionState.class);
 private final HandleHolder zooKeeper;
 private final AtomicBoolean isConnected = new AtomicBoolean(false);
 private final EnsembleProvider ensembleProvider;
@@ -145,24 +145,22 @@ class ConnectionState implements Watcher, Closeable
 log.debug("ConnectState watcher: " + event);
 }
 
-for ( Watcher parentWatcher : parentWatchers )
-{
-TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
-parentWatcher.process(event);
-timeTrace.commit();
-}
-
-boolean wasConnected = isConnected.get();
-boolean newIsConnected = wasConnected;
 if ( event.getType() == Watcher.Event.EventType.None )
 {
-newIsConnected = checkState(event.getState(), wasConnected);
+boolean wasConnected = isConnected.get();
+boolean newIsConnected = checkState(event.getState(), 
wasConnected);
+if ( newIsConnected != wasConnected )
+{
+isConnected.set(newIsConnected);
+connectionStartMs = System.currentTimeMillis();
+}
 }
 
-if ( newIsConnected != wasConnected )
+for ( Watcher parentWatcher : parentWatchers )
 {
-isConnected.set(newIsConnected);
-connectionStartMs = System.currentTimeMillis();
+TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
+parentWatcher.process(event);
+timeTrace.commit();
 }
 }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
--
diff --git 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
index f649afb..a6dc7ab 100644
--- 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
+++ 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
@@ -232,4 +232,28 @@ public class TestBlockUntilConnected extends 
BaseClassForTests
 CloseableUtils.closeQuietly(client);
 }
 }
+
+/**
+ * Test that we are actually connected every time that we block until 
connection is established in a tight loop.
+ */
+@Test
+public void testBlockUntilConnectedTightLoop() throws InterruptedException
+{
+CuratorFramework client;
+for(int i = 0 ; i < 50 ; i++)
+{
+client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTime(100));
+try
+{
+client.start();
+

[2/3] curator git commit: Merge branch 'master' into CURATOR-3.0

2015-08-24 Thread randgalt
Merge branch 'master' into CURATOR-3.0


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

Branch: refs/heads/CURATOR-247
Commit: f9af0ce81408f7a4cb4343fb0770e9e6ef545fda
Parents: 2fc9e37 061ed0a
Author: Mike Drob 
Authored: Mon Aug 24 07:10:20 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 07:10:20 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 26 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 ++
 2 files changed, 36 insertions(+), 14 deletions(-)
--




curator git commit: updated doc

2015-08-24 Thread randgalt
Repository: curator
Updated Branches:
  refs/heads/CURATOR-247 1bbf301ab -> b8240a1df


updated doc


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

Branch: refs/heads/CURATOR-247
Commit: b8240a1dfcaf291196dc68fbfd1f2204c50951fc
Parents: 1bbf301
Author: randgalt 
Authored: Mon Aug 24 09:28:53 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 09:28:53 2015 -0500

--
 .../curator/framework/CuratorFrameworkFactory.java   |  2 +-
 .../apache/curator/framework/state/ConnectionState.java  |  7 ---
 .../framework/imps/TestEnabledSessionExpiredState.java   | 11 +++
 src/site/confluence/errors.confluence|  6 +++---
 4 files changed, 19 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/b8240a1d/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index c59d78f..daffa13 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -381,7 +381,7 @@ public class CuratorFrameworkFactory
  * action. In Curator, this is complicated by the fact that 
Curator internally manages the ZooKeeper
  * connection. Now, Curator will set the LOST state when any 
of the following occurs:
  * a) ZooKeeper returns a {@link 
Watcher.Event.KeeperState#Expired} or {@link 
KeeperException.Code#SESSIONEXPIRED};
- * b) Curator closes the internally managed ZooKeeper 
instance; c) The configured session timeout
+ * b) Curator closes the internally managed ZooKeeper 
instance; c) The session timeout
  * elapses during a network partition.
  * 
  * 

http://git-wip-us.apache.org/repos/asf/curator/blob/b8240a1d/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
index fe40abf..f9f245a 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionState.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.framework.state;
 
+import org.apache.curator.connection.ConnectionHandlingPolicy;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
@@ -64,17 +65,17 @@ public enum ConnectionState
 
 /**
  * 
- * Curator will set the LOST state only when it believes that the 
ZooKeeper session
+ * Curator will set the LOST state when it believes that the ZooKeeper 
session
  * has expired. ZooKeeper connections have a session. When the session 
expires, clients must take appropriate
  * action. In Curator, this is complicated by the fact that Curator 
internally manages the ZooKeeper
  * connection. Curator will set the LOST state when any of the 
following occurs:
  * a) ZooKeeper returns a {@link Watcher.Event.KeeperState#Expired} or 
{@link KeeperException.Code#SESSIONEXPIRED};
- * b) Curator closes the internally managed ZooKeeper instance; c) The 
configured session timeout
+ * b) Curator closes the internally managed ZooKeeper instance; c) The 
session timeout
  * elapses during a network partition.
  * 
  *
  * 
- * NOTE: see {@link 
CuratorFrameworkFactory.Builder#connectionHandlingPolicy} for an important note 
about a
+ * NOTE: see {@link 
CuratorFrameworkFactory.Builder#connectionHandlingPolicy(ConnectionHandlingPolicy)}
 for an important note about a
  * change in meaning to LOST since 3.0.0
  * 
  */

http://git-wip-us.apache.org/repos/asf/curator/blob/b8240a1d/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java
--
diff --git 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredSt

[2/3] curator git commit: CURATOR-234 Add remaining slf4j-log4j12 test deps

2015-08-24 Thread mdrob
CURATOR-234 Add remaining slf4j-log4j12 test deps


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

Branch: refs/heads/master
Commit: b177ed709a422f0b06f3a4a3683e5488d038fbb9
Parents: 9a444fa
Author: Mike Drob 
Authored: Mon Aug 24 10:40:12 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 10:41:09 2015 -0500

--
 curator-x-discovery-server/pom.xml | 6 ++
 curator-x-discovery/pom.xml| 6 ++
 curator-x-rpc/pom.xml  | 6 ++
 3 files changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-discovery-server/pom.xml
--
diff --git a/curator-x-discovery-server/pom.xml 
b/curator-x-discovery-server/pom.xml
index 2728ce5..4ed7e25 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -108,6 +108,12 @@
 resteasy-jaxrs
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-discovery/pom.xml
--
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 015fea7..9043533 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -66,5 +66,11 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-rpc/pom.xml
--
diff --git a/curator-x-rpc/pom.xml b/curator-x-rpc/pom.xml
index 299ef9f..519e8fb 100644
--- a/curator-x-rpc/pom.xml
+++ b/curator-x-rpc/pom.xml
@@ -118,6 +118,12 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 



[1/3] curator git commit: CURATOR-234 Add remaining slf4j-log4j12 test deps

2015-08-24 Thread mdrob
Repository: curator
Updated Branches:
  refs/heads/CURATOR-234 9a444faca -> b177ed709
  refs/heads/master 061ed0a6d -> 5dc27c1fa


CURATOR-234 Add remaining slf4j-log4j12 test deps


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

Branch: refs/heads/CURATOR-234
Commit: b177ed709a422f0b06f3a4a3683e5488d038fbb9
Parents: 9a444fa
Author: Mike Drob 
Authored: Mon Aug 24 10:40:12 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 10:41:09 2015 -0500

--
 curator-x-discovery-server/pom.xml | 6 ++
 curator-x-discovery/pom.xml| 6 ++
 curator-x-rpc/pom.xml  | 6 ++
 3 files changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-discovery-server/pom.xml
--
diff --git a/curator-x-discovery-server/pom.xml 
b/curator-x-discovery-server/pom.xml
index 2728ce5..4ed7e25 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -108,6 +108,12 @@
 resteasy-jaxrs
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-discovery/pom.xml
--
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 015fea7..9043533 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -66,5 +66,11 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-rpc/pom.xml
--
diff --git a/curator-x-rpc/pom.xml b/curator-x-rpc/pom.xml
index 299ef9f..519e8fb 100644
--- a/curator-x-rpc/pom.xml
+++ b/curator-x-rpc/pom.xml
@@ -118,6 +118,12 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 



[3/3] curator git commit: Merge branch 'CURATOR-234'

2015-08-24 Thread mdrob
Merge branch 'CURATOR-234'


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

Branch: refs/heads/master
Commit: 5dc27c1fa9b8e3bc6ed3e1b198bb1eca6b378c9a
Parents: 061ed0a b177ed7
Author: Mike Drob 
Authored: Mon Aug 24 10:41:39 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 10:41:39 2015 -0500

--
 curator-x-discovery-server/pom.xml | 6 ++
 curator-x-discovery/pom.xml| 6 ++
 curator-x-rpc/pom.xml  | 6 ++
 3 files changed, 18 insertions(+)
--




[1/3] curator git commit: Initial error policy with two implementations. Also, applied it to LeaderSelector as a test

2015-08-24 Thread randgalt
Repository: curator
Updated Branches:
  refs/heads/CURATOR-248 [created] 94dff8a5a


Initial error policy with two implementations. Also, applied it to 
LeaderSelector as a test


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

Branch: refs/heads/CURATOR-248
Commit: 45df7ba71f14a5f9751061a7dff956312bfdd421
Parents: f9af0ce
Author: randgalt 
Authored: Mon Aug 24 12:24:06 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 12:24:06 2015 -0500

--
 .../curator/framework/CuratorFramework.java |  8 ++
 .../framework/CuratorFrameworkFactory.java  | 20 +
 .../framework/imps/CuratorFrameworkImpl.java| 10 +++
 .../curator/framework/state/ErrorPolicy.java| 18 
 .../framework/state/SessionErrorPolicy.java | 13 +++
 .../framework/state/StandardErrorPolicy.java| 14 +++
 .../leader/LeaderSelectorListenerAdapter.java   |  2 +-
 .../recipes/leader/TestLeaderSelector.java  | 90 
 8 files changed, 174 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/45df7ba7/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 58c5bf5..d755d28 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
@@ -28,6 +28,7 @@ import 
org.apache.curator.framework.api.transaction.TransactionOp;
 import org.apache.curator.framework.imps.CuratorFrameworkState;
 import org.apache.curator.framework.listen.Listenable;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.framework.state.ErrorPolicy;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.zookeeper.Watcher;
 
@@ -297,4 +298,11 @@ public interface CuratorFramework extends Closeable
  * @return facade
  */
 public WatcherRemoveCuratorFramework newWatcherRemoveCuratorFramework();
+
+/**
+ * Return the configured error policy
+ *
+ * @return error policy
+ */
+public ErrorPolicy getErrorPolicy();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/45df7ba7/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index dcb2ee6..aa5181d 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -31,6 +31,8 @@ import org.apache.curator.framework.imps.CuratorFrameworkImpl;
 import org.apache.curator.framework.imps.CuratorTempFrameworkImpl;
 import org.apache.curator.framework.imps.DefaultACLProvider;
 import org.apache.curator.framework.imps.GzipCompressionProvider;
+import org.apache.curator.framework.state.ErrorPolicy;
+import org.apache.curator.framework.state.StandardErrorPolicy;
 import org.apache.curator.utils.DefaultZookeeperFactory;
 import org.apache.curator.utils.ZookeeperFactory;
 import org.apache.zookeeper.CreateMode;
@@ -116,6 +118,7 @@ public class CuratorFrameworkFactory
 private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER;
 private boolean canBeReadOnly = false;
 private boolean useContainerParentsIfAvailable = true;
+private ErrorPolicy errorPolicy = new StandardErrorPolicy();
 
 /**
  * Apply the current values and build a new CuratorFramework
@@ -343,6 +346,18 @@ public class CuratorFrameworkFactory
 return this;
 }
 
+/**
+ * Set the error policy to use. The default is {@link 
StandardErrorPolicy}
+ *
+ * @param errorPolicy new error policy
+ * @return this
+ */
+public Builder errorPolicy(ErrorPolicy errorPolicy)
+{
+this.errorPolicy = errorPolicy;
+return this;
+}
+
 public ACLProvider getAclProvider()
 {
 return aclProvider;
@@ -398,6 +413,11 @@ public class CuratorFrameworkFactory
 return useContainerParentsIfAvailable;
 }
 
+public ErrorPolicy getErrorPolicy()
+{
+  

[3/3] curator git commit: Added since tags

2015-08-24 Thread randgalt
Added since tags


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

Branch: refs/heads/CURATOR-248
Commit: 94dff8a5a2ec336a23c05ebe5cdf7e4b117d3925
Parents: 2e1e92e
Author: randgalt 
Authored: Mon Aug 24 12:31:52 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 12:31:52 2015 -0500

--
 .../java/org/apache/curator/framework/CuratorFrameworkFactory.java | 1 +
 .../main/java/org/apache/curator/framework/state/ErrorPolicy.java  | 2 ++
 2 files changed, 3 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/94dff8a5/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index aa5181d..9a67684 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -349,6 +349,7 @@ public class CuratorFrameworkFactory
 /**
  * Set the error policy to use. The default is {@link 
StandardErrorPolicy}
  *
+ * @since 3.0.0
  * @param errorPolicy new error policy
  * @return this
  */

http://git-wip-us.apache.org/repos/asf/curator/blob/94dff8a5/curator-framework/src/main/java/org/apache/curator/framework/state/ErrorPolicy.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/state/ErrorPolicy.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/state/ErrorPolicy.java
index 0e1bfb5..73fc99d 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/state/ErrorPolicy.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/state/ErrorPolicy.java
@@ -3,6 +3,8 @@ package org.apache.curator.framework.state;
 /**
  * Recipes should use the configured error policy to decide how to handle
  * errors such as {@link ConnectionState} changes.
+ *
+ * @since 3.0.0
  */
 public interface ErrorPolicy
 {



[2/3] curator git commit: doc

2015-08-24 Thread randgalt
doc


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

Branch: refs/heads/CURATOR-248
Commit: 2e1e92e162ef812a9d076695500f0bda8b15b6c5
Parents: 45df7ba
Author: randgalt 
Authored: Mon Aug 24 12:30:33 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 12:30:33 2015 -0500

--
 src/site/confluence/errors.confluence | 7 +++
 1 file changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/2e1e92e1/src/site/confluence/errors.confluence
--
diff --git a/src/site/confluence/errors.confluence 
b/src/site/confluence/errors.confluence
index 28805e0..c9545ac 100644
--- a/src/site/confluence/errors.confluence
+++ b/src/site/confluence/errors.confluence
@@ -31,5 +31,12 @@ appropriate action. These are the possible state changes:
 {{UnhandledErrorListener}} is called when a background task, etc. catches an 
exception. In general, Curator users shouldn't care
 about these as they are logged. However, you can listen for them if you choose.
 
+h2. Error Policy
+
+Curator has a pluggable error policy. The default policy takes the 
conservative approach of treating connection states SUSPENDED and LOST the same 
way.
+i.e. when a recipe sees the state change to SUSPENDED it will assume that the 
ZooKeeper session is lost and will clean up any watchers, nodes, etc. You can 
choose,
+however, a more aggressive approach by setting the error policy to only treat 
LOST (i.e. true session loss) as an error state. Do this in the 
CuratorFrameworkFactory via:
+{{errorPolicy(new SessionErrorPolicy())}}.
+
 h2. Recipes
 In general, the recipes attempt to deal with errors and connection issues. See 
the doc for each recipe for details on how it deals with errors.



curator git commit: Updated LeaderLatch for error policy

2015-08-24 Thread randgalt
Repository: curator
Updated Branches:
  refs/heads/CURATOR-248 94dff8a5a -> 5429a217b


Updated LeaderLatch for error policy


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

Branch: refs/heads/CURATOR-248
Commit: 5429a217bb23901aaf2b187bb8c1d760d0a76bcc
Parents: 94dff8a
Author: randgalt 
Authored: Mon Aug 24 17:39:41 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 17:39:41 2015 -0500

--
 .../framework/recipes/leader/LeaderLatch.java   |  39 +++--
 .../recipes/leader/TestLeaderLatch.java | 162 +++
 2 files changed, 187 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/5429a217/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
index da9b8b2..aa4dd9f 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
@@ -160,20 +160,20 @@ public class LeaderLatch implements Closeable
 Preconditions.checkState(state.compareAndSet(State.LATENT, 
State.STARTED), "Cannot be started more than once");
 
 startTask.set(AfterConnectionEstablished.execute(client, new Runnable()
+{
+@Override
+public void run()
+{
+try
 {
-@Override
-public void run()
-{
-try
-{
-internalStart();
-}
-finally
-{
-startTask.set(null);
-}
-}
-}));
+internalStart();
+}
+finally
+{
+startTask.set(null);
+}
+}
+}));
 }
 
 /**
@@ -604,7 +604,10 @@ public class LeaderLatch implements Closeable
 {
 try
 {
-reset();
+if ( 
client.getErrorPolicy().isErrorState(ConnectionState.SUSPENDED) || 
!hasLeadership.get() )
+{
+reset();
+}
 }
 catch ( Exception e )
 {
@@ -615,6 +618,14 @@ public class LeaderLatch implements Closeable
 }
 
 case SUSPENDED:
+{
+if ( 
client.getErrorPolicy().isErrorState(ConnectionState.SUSPENDED) )
+{
+setLeadership(false);
+}
+break;
+}
+
 case LOST:
 {
 setLeadership(false);

http://git-wip-us.apache.org/repos/asf/curator/blob/5429a217/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
index 3742fb7..bd73e9d 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
@@ -21,12 +21,15 @@ package org.apache.curator.framework.recipes.leader;
 
 import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Queues;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.imps.TestCleanState;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.framework.state.SessionErrorPolicy;
+import org.apache.curator.framework.state.StandardErrorPolicy;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
@@ -37,11 +40,13 @@ import org.testng.Assert

curator git commit: removed import

2015-08-24 Thread randgalt
Repository: curator
Updated Branches:
  refs/heads/CURATOR-248 5429a217b -> d412f2320


removed import


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

Branch: refs/heads/CURATOR-248
Commit: d412f2320d571d23c8960214482e84a7911bec16
Parents: 5429a21
Author: randgalt 
Authored: Mon Aug 24 22:03:13 2015 -0500
Committer: randgalt 
Committed: Mon Aug 24 22:03:13 2015 -0500

--
 .../apache/curator/framework/recipes/leader/TestLeaderLatch.java| 1 -
 1 file changed, 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/d412f232/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
index bd73e9d..41b53fd 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
@@ -46,7 +46,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;



[43/50] curator git commit: Merge branch 'CURATOR-231'

2015-08-24 Thread cammckenzie
Merge branch 'CURATOR-231'


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

Branch: refs/heads/CURATOR-167
Commit: e8aff9a6092689662f33efac5cd9601fba5c2c64
Parents: 41f49ec 9a444fa
Author: Mike Drob 
Authored: Fri Aug 14 14:27:10 2015 -0500
Committer: Mike Drob 
Committed: Fri Aug 14 14:27:10 2015 -0500

--
 curator-client/pom.xml| 6 ++
 curator-examples/pom.xml  | 6 ++
 curator-framework/pom.xml | 6 ++
 curator-recipes/pom.xml   | 6 ++
 curator-test/pom.xml  | 6 ++
 5 files changed, 30 insertions(+)
--




[15/50] curator git commit: [CURATOR-223] Add executorService methods to ServiceCacheBuilder

2015-08-24 Thread cammckenzie
[CURATOR-223] Add executorService methods to ServiceCacheBuilder

Add executorService methods to ServiceCacheBuilder to allow the caller to 
specify
an ExecutorService or a CloseableExecutorService to be used by the 
PathChildrenCache
embedded in ServiceCacheImpl.

Extracts ExecuteCalledWatchingExecutorService (and DelegatingExecutorService) 
into
the curator-test module for use by TestServiceCache.


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

Branch: refs/heads/CURATOR-167
Commit: 6ca6d3d2c71b1e541c0edd60d2c17efe9c66
Parents: 20e92a5
Author: Tom Dyas 
Authored: Tue Jun 16 17:38:18 2015 -0400
Committer: Tom Dyas 
Committed: Wed Jun 17 13:03:17 2015 -0400

--
 .../recipes/cache/TestPathChildrenCache.java| 124 +--
 .../curator/test/DelegatingExecutorService.java | 119 ++
 .../ExecuteCalledWatchingExecutorService.java   |  48 +++
 .../x/discovery/ServiceCacheBuilder.java|  24 +++-
 .../details/ServiceCacheBuilderImpl.java|  39 +-
 .../x/discovery/details/ServiceCacheImpl.java   |  17 ++-
 .../curator/x/discovery/TestServiceCache.java   |  53 
 7 files changed, 297 insertions(+), 127 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/6ca6/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
index b904bdc..216660f 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
@@ -31,6 +31,7 @@ import 
org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.framework.imps.CuratorFrameworkImpl;
 import org.apache.curator.retry.RetryOneTime;
 import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
 import org.apache.curator.test.KillSession;
 import org.apache.curator.test.Timing;
 import org.apache.curator.utils.CloseableUtils;
@@ -1039,127 +1040,4 @@ public class TestPathChildrenCache extends 
BaseClassForTests
 CloseableUtils.closeQuietly(client);
 }
 }
-
-public static class ExecuteCalledWatchingExecutorService extends 
DelegatingExecutorService
-{
-boolean executeCalled = false;
-
-public ExecuteCalledWatchingExecutorService(ExecutorService delegate)
-{
-super(delegate);
-}
-
-@Override
-public synchronized void execute(Runnable command)
-{
-executeCalled = true;
-super.execute(command);
-}
-
-public synchronized boolean isExecuteCalled()
-{
-return executeCalled;
-}
-
-public synchronized void setExecuteCalled(boolean executeCalled)
-{
-this.executeCalled = executeCalled;
-}
-}
-
-public static class DelegatingExecutorService implements ExecutorService
-{
-private final ExecutorService delegate;
-
-public DelegatingExecutorService(
-ExecutorService delegate
-)
-{
-this.delegate = delegate;
-}
-
-
-@Override
-public void shutdown()
-{
-delegate.shutdown();
-}
-
-@Override
-public List shutdownNow()
-{
-return delegate.shutdownNow();
-}
-
-@Override
-public boolean isShutdown()
-{
-return delegate.isShutdown();
-}
-
-@Override
-public boolean isTerminated()
-{
-return delegate.isTerminated();
-}
-
-@Override
-public boolean awaitTermination(long timeout, TimeUnit unit)
-throws InterruptedException
-{
-return delegate.awaitTermination(timeout, unit);
-}
-
-@Override
-public  Future submit(Callable task)
-{
-return delegate.submit(task);
-}
-
-@Override
-public  Future submit(Runnable task, T result)
-{
-return delegate.submit(task, result);
-}
-
-@Override
-public Future submit(Runnable task)
-{
-return delegate.submit(task);
-}
-
-@Override
-

[34/50] curator git commit: Merge branch 'CURATOR-235' of github.com:oza/curator into CURATOR-235

2015-08-24 Thread cammckenzie
Merge branch 'CURATOR-235' of github.com:oza/curator into CURATOR-235


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

Branch: refs/heads/CURATOR-167
Commit: 870b4d51968756aa6cbf82444bcd721c005db884
Parents: a63bb03 0bec8a0
Author: randgalt 
Authored: Sat Jul 18 12:40:58 2015 -0500
Committer: randgalt 
Committed: Sat Jul 18 12:40:58 2015 -0500

--
 .../apache/curator/framework/recipes/leader/LeaderSelector.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--




[42/50] curator git commit: CURATOR-234 Add slf4j-log4j12 test dependency

2015-08-24 Thread cammckenzie
CURATOR-234 Add slf4j-log4j12 test dependency


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

Branch: refs/heads/CURATOR-167
Commit: 9a444faca2c3e799d2494cb9968118775a9a1b0b
Parents: e71da8f
Author: Mike Drob 
Authored: Mon Jul 13 08:02:48 2015 -0500
Committer: Mike Drob 
Committed: Fri Aug 14 14:26:55 2015 -0500

--
 curator-client/pom.xml| 6 ++
 curator-examples/pom.xml  | 6 ++
 curator-framework/pom.xml | 6 ++
 curator-recipes/pom.xml   | 6 ++
 curator-test/pom.xml  | 6 ++
 5 files changed, 30 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/9a444fac/curator-client/pom.xml
--
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 22ccf28..64a3393 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -77,5 +77,11 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/9a444fac/curator-examples/pom.xml
--
diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index 85eaaf7..406cbac 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -48,5 +48,11 @@
 org.apache.curator
 curator-x-discovery
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/9a444fac/curator-framework/pom.xml
--
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index 7d9bd03..212678d 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -61,6 +61,12 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/9a444fac/curator-recipes/pom.xml
--
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index 8c257b7..de45404 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -67,5 +67,11 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/9a444fac/curator-test/pom.xml
--
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index 59abda2..96809d8 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -60,5 +60,11 @@
 testng
 provided
 
+
+   
+org.slf4j
+slf4j-log4j12
+test
+
 
 



[36/50] curator git commit: CURATOR-240: fix pathological "root doesn't exist" infinite refresh cycle

2015-08-24 Thread cammckenzie
CURATOR-240: fix pathological "root doesn't exist" infinite refresh cycle


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

Branch: refs/heads/CURATOR-167
Commit: 3a4d541a383a1a42c1f605056d75d36da26a76c5
Parents: c3426ba
Author: Scott Blum 
Authored: Fri Jul 31 17:19:14 2015 -0400
Committer: Scott Blum 
Committed: Fri Jul 31 17:19:14 2015 -0400

--
 .../apache/curator/framework/recipes/cache/TreeCache.java | 10 --
 1 file changed, 4 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/3a4d541a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
index 7cf2267..4f3ffb6 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
@@ -301,7 +301,8 @@ public class TreeCache implements Closeable
 return;
 }
 
-if ( nodeState.compareAndSet(NodeState.LIVE, NodeState.DEAD) )
+NodeState oldState = nodeState.getAndSet(NodeState.DEAD);
+if ( oldState == NodeState.LIVE )
 {
 publishEvent(TreeCacheEvent.Type.NODE_REMOVED, path);
 }
@@ -363,10 +364,6 @@ public class TreeCache implements Closeable
 nodeState.compareAndSet(NodeState.DEAD, NodeState.PENDING);
 wasCreated();
 }
-else if ( event.getResultCode() == 
KeeperException.Code.NONODE.intValue() )
-{
-wasDeleted();
-}
 break;
 case CHILDREN:
 if ( event.getResultCode() == 
KeeperException.Code.OK.intValue() )
@@ -429,7 +426,8 @@ public class TreeCache implements Closeable
 }
 
 Stat oldStat = stat.getAndSet(newStat);
-if ( nodeState.compareAndSet(NodeState.PENDING, 
NodeState.LIVE) )
+NodeState oldState = nodeState.getAndSet(NodeState.LIVE);
+if ( oldState != NodeState.LIVE )
 {
 publishEvent(TreeCacheEvent.Type.NODE_ADDED, new 
ChildData(event.getPath(), newStat, event.getData()));
 }



[16/50] curator git commit: finalized usage and APIs. Made sure is backward compatible to 3.4.6

2015-08-24 Thread cammckenzie
finalized usage and APIs. Made sure is backward compatible to 3.4.6


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

Branch: refs/heads/CURATOR-167
Commit: 759ae68274ddfc38f18994021d31a5a08dac1066
Parents: 25dcef9
Author: randgalt 
Authored: Wed Jun 17 20:40:15 2015 -0500
Committer: randgalt 
Committed: Wed Jun 17 20:40:15 2015 -0500

--
 .../org/apache/curator/utils/DebugUtils.java|  1 +
 .../org/apache/curator/utils/EnsurePath.java|  2 +
 .../java/org/apache/curator/utils/ZKPaths.java  | 14 +-
 .../curator/framework/CuratorFramework.java |  1 +
 .../api/CreateBackgroundModeACLable.java|  7 ++-
 .../curator/framework/api/CreateBuilder.java|  7 ++-
 .../curator/framework/api/ExistsBuilder.java| 15 --
 .../framework/api/ExistsBuilderMain.java| 27 ++
 .../framework/imps/CreateBuilderImpl.java   | 12 ++---
 .../framework/imps/ExistsBuilderImpl.java   | 53 +---
 .../curator/framework/imps/TestFramework.java   | 21 
 .../recipes/atomic/DistributedAtomicValue.java  |  2 +-
 .../framework/recipes/cache/NodeCache.java  |  7 +--
 .../recipes/cache/PathChildrenCache.java| 21 +---
 .../recipes/queue/SimpleDistributedQueue.java   | 16 ++
 .../recipes/cache/TestPathChildrenCache.java| 10 ++--
 .../locks/TestInterProcessMutexBase.java|  8 +++
 .../apache/curator/test/BaseClassForTests.java  | 28 ---
 pom.xml |  2 +-
 19 files changed, 212 insertions(+), 42 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/759ae682/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java 
b/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
index ce751ec..b098989 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java
@@ -23,6 +23,7 @@ public class DebugUtils
 public static final String  PROPERTY_LOG_EVENTS = 
"curator-log-events";
 public static final String  PROPERTY_DONT_LOG_CONNECTION_ISSUES = 
"curator-dont-log-connection-problems";
 public static final String  
PROPERTY_LOG_ONLY_FIRST_CONNECTION_ISSUE_AS_ERROR_LEVEL = 
"curator-log-only-first-connection-issue-as-error-level";
+public static final String  PROPERTY_RETRY_FAILED_TESTS = 
"curator-retry-failed-tests";
 
 private DebugUtils()
 {

http://git-wip-us.apache.org/repos/asf/curator/blob/759ae682/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java 
b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
index 3abb618..a4a8528 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
@@ -47,6 +47,8 @@ import java.util.concurrent.atomic.AtomicReference;
  * ensurePath.ensure(zk);   // subsequent times are NOPs
  * zk.create(nodePath, ...);
  * 
+ *
+ * @deprecated Prefer 
CuratorFramework.create().creatingParentContainersIfNeeded() or 
CuratorFramework.exists().creatingParentContainersIfNeeded()
  */
 public class EnsurePath
 {

http://git-wip-us.apache.org/repos/asf/curator/blob/759ae682/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
--
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java 
b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
index f025fa6..75e1171 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
@@ -38,6 +38,8 @@ public class ZKPaths
  */
 public static final String PATH_SEPARATOR = "/";
 
+private static final CreateMode NON_CONTAINER_MODE = CreateMode.PERSISTENT;
+
 /**
  * @return {@link CreateMode#CONTAINER} if the ZK JAR supports it. 
Otherwise {@link CreateMode#PERSISTENT}
  */
@@ -46,6 +48,16 @@ public class ZKPaths
 return CreateModeHolder.containerCreateMode;
 }
 
+/**
+ * Returns true if the version of ZooKeeper client in use supports 
containers
+ *
+ * @return true/false
+ */
+public static 

[37/50] curator git commit: CURATOR-241: Write updated data on reconnect

2015-08-24 Thread cammckenzie
CURATOR-241: Write updated data on reconnect

PersistentEphemeralNode can be initialised with certain data, then
later updated. If the client reconnects, the replacing ephemeral
should write the updated data.


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

Branch: refs/heads/CURATOR-167
Commit: 7fd4034e30f2ed3690226d2d78bf9f139df8b491
Parents: 3a4d541
Author: Alex Brasetvik 
Authored: Mon Aug 3 00:57:51 2015 +0200
Committer: Alex Brasetvik 
Committed: Mon Aug 3 01:02:19 2015 +0200

--
 .../recipes/nodes/PersistentEphemeralNode.java  |  8 +++-
 .../nodes/TestPersistentEphemeralNode.java  | 41 
 2 files changed, 47 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/7fd4034e/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 f50dca4..1011ad5 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
@@ -240,7 +240,7 @@ public class PersistentEphemeralNode implements Closeable
 
 if(nodeExists)
 {
-   
client.setData().inBackground(setDataCallback).forPath(getActualPath(), data);
+   
client.setData().inBackground(setDataCallback).forPath(getActualPath(), 
getData());
 }
 else
 {
@@ -338,10 +338,14 @@ public class PersistentEphemeralNode implements Closeable
 this.data.set(Arrays.copyOf(data, data.length));
 if ( isActive() )
 {
-client.setData().inBackground().forPath(getActualPath(), 
this.data.get());
+client.setData().inBackground().forPath(getActualPath(), 
getData());
 }
 }
 
+byte[] getData() {
+return this.data.get();
+}
+
 private void deleteNode() throws Exception
 {
 String localNodePath = nodePath.getAndSet(null);

http://git-wip-us.apache.org/repos/asf/curator/blob/7fd4034e/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 34620ff..9f5907a 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
@@ -23,6 +23,8 @@ import com.google.common.collect.Lists;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.BackgroundCallback;
+import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.retry.RetryOneTime;
@@ -536,6 +538,45 @@ public class TestPersistentEphemeralNode extends 
BaseClassForTests
 node.close();
 }  
 }
+
+@Test
+public void testSetUpdatedDataWhenReconnected() throws Exception
+{
+CuratorFramework curator = newCurator();
+
+byte[] initialData = "Hello World".getBytes();
+byte[] updatedData = "Updated".getBytes();
+
+PersistentEphemeralNode node = new PersistentEphemeralNode(curator, 
PersistentEphemeralNode.Mode.EPHEMERAL, PATH, initialData);
+node.start();
+try
+{
+node.waitForInitialCreate(timing.forWaiting().seconds(), 
TimeUnit.SECONDS);
+
assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), 
initialData));
+
+node.setData(updatedData);
+
assertTrue(Arrays.equals(curator.getData().forPath(node.getActualPath()), 
updatedData));
+
+server.restart();
+
+final CountDownLatch dataUpdateLatch = new CountDownLatch(1);
+curator.getData().inBackground(new BackgroundCallback

[28/50] curator git commit: Use new APIs to ensure path

2015-08-24 Thread cammckenzie
Use new APIs to ensure path


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

Branch: refs/heads/CURATOR-167
Commit: df8ec866d5caf01a16e97ffaca26c35220baa019
Parents: f18f876
Author: randgalt 
Authored: Wed Jul 1 17:50:00 2015 -0500
Committer: randgalt 
Committed: Wed Jul 1 17:50:00 2015 -0500

--
 .../org/apache/curator/framework/imps/NamespaceImpl.java  | 10 ++
 1 file changed, 6 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/df8ec866/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
index 03411a2..b0dda5c 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
@@ -21,12 +21,13 @@ package org.apache.curator.framework.imps;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.curator.utils.PathUtils;
 import org.apache.curator.utils.ZKPaths;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 class NamespaceImpl
 {
 private final CuratorFrameworkImpl client;
 private final String namespace;
-private final EnsurePath ensurePath;
+private final AtomicBoolean ensurePathNeeded;
 
 NamespaceImpl(CuratorFrameworkImpl client, String namespace)
 {
@@ -44,7 +45,7 @@ class NamespaceImpl
 
 this.client = client;
 this.namespace = namespace;
-ensurePath = (namespace != null) ? new 
EnsurePath(ZKPaths.makePath("/", namespace)) : null;
+ensurePathNeeded = new AtomicBoolean(namespace != null);
 }
 
 String getNamespace()
@@ -67,11 +68,12 @@ class NamespaceImpl
 
 StringfixForNamespace(String path, boolean isSequential)
 {
-if ( ensurePath != null )
+if ( ensurePathNeeded.get() )
 {
 try
 {
-ensurePath.ensure(client.getZookeeperClient());
+client.createContainers(ZKPaths.makePath("/", namespace));
+ensurePathNeeded.set(false);
 }
 catch ( Exception e )
 {



[23/50] curator git commit: creatingParentContainersIfNeeded for checkExists() was broken. Fixed and added a test

2015-08-24 Thread cammckenzie
creatingParentContainersIfNeeded for checkExists() was broken. Fixed and added 
a test


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

Branch: refs/heads/CURATOR-167
Commit: 7ad12754a9f1bd4ac9242886c245e3e2d2fa7dc4
Parents: d678de0
Author: randgalt 
Authored: Tue Jun 23 17:58:28 2015 -0500
Committer: randgalt 
Committed: Tue Jun 23 17:58:28 2015 -0500

--
 .../framework/imps/ExistsBuilderImpl.java   | 85 +
 .../curator/framework/imps/TestFramework.java   | 97 +++-
 2 files changed, 123 insertions(+), 59 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/7ad12754/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
index db7df9e..d4a059d 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/ExistsBuilderImpl.java
@@ -131,15 +131,8 @@ class ExistsBuilderImpl implements ExistsBuilder, 
BackgroundOperation
 public void processResult(int rc, String path, Object ctx, Stat 
stat)
 {
 trace.commit();
-if ( (rc == KeeperException.Code.NONODE.intValue()) && 
createParentContainersIfNeeded )
-{
-CreateBuilderImpl.backgroundCreateParentsThenNode(client, 
operationAndData, operationAndData.getData(), backgrounding, true);
-}
-else
-{
-CuratorEvent event = new CuratorEventImpl(client, 
CuratorEventType.EXISTS, rc, path, null, ctx, stat, null, null, null, null);
-client.processBackgroundOperation(operationAndData, event);
-}
+CuratorEvent event = new CuratorEventImpl(client, 
CuratorEventType.EXISTS, rc, path, null, ctx, stat, null, null, null, null);
+client.processBackgroundOperation(operationAndData, event);
 }
 };
 if ( watching.isWatched() )
@@ -160,7 +153,15 @@ class ExistsBuilderImpl implements ExistsBuilder, 
BackgroundOperation
 StatreturnStat = null;
 if ( backgrounding.inBackground() )
 {
-client.processBackgroundOperation(new 
OperationAndData(this, path, backgrounding.getCallback(), null, 
backgrounding.getContext()), null);
+OperationAndData operationAndData = new 
OperationAndData(this, path, backgrounding.getCallback(), null, 
backgrounding.getContext());
+if ( createParentContainersIfNeeded )
+{
+CreateBuilderImpl.backgroundCreateParentsThenNode(client, 
operationAndData, operationAndData.getData(), backgrounding, true);
+}
+else
+{
+client.processBackgroundOperation(operationAndData, null);
+}
 }
 else
 {
@@ -172,6 +173,40 @@ class ExistsBuilderImpl implements ExistsBuilder, 
BackgroundOperation
 
 private Stat pathInForeground(final String path) throws Exception
 {
+if ( createParentContainersIfNeeded )
+{
+final String parent = ZKPaths.getPathAndNode(path).getPath();
+if ( !parent.equals(ZKPaths.PATH_SEPARATOR) )
+{
+TimeTrace   trace = 
client.getZookeeperClient().startTracer("ExistsBuilderImpl-Foreground-CreateParents");
+RetryLoop.callWithRetry
+(
+client.getZookeeperClient(),
+new Callable()
+{
+@Override
+public Void call() throws Exception
+{
+try
+{
+ZKPaths.mkdirs(client.getZooKeeper(), parent, 
true, client.getAclProvider(), true);
+}
+catch ( KeeperException e )
+{
+// ignore
+}
+return null;
+}
+}
+);
+trace.commit();
+}
+}
+return pathInForegroundStandard(path);
+}
+
+private Stat pathInForegroundStandard(final String path) throws Except

[02/50] curator git commit: Don't show RPC report

2015-08-24 Thread cammckenzie
Don't show RPC report


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

Branch: refs/heads/CURATOR-167
Commit: 8d8fdf61e92781806d4633607e75f54223dff8fa
Parents: 121efdb
Author: randgalt 
Authored: Fri May 8 14:17:16 2015 -0500
Committer: randgalt 
Committed: Fri May 8 14:17:16 2015 -0500

--
 src/site/confluence/compatibility.confluence | 1 -
 1 file changed, 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/8d8fdf61/src/site/confluence/compatibility.confluence
--
diff --git a/src/site/confluence/compatibility.confluence 
b/src/site/confluence/compatibility.confluence
index 4ffa7bb..ef3324f 100644
--- a/src/site/confluence/compatibility.confluence
+++ b/src/site/confluence/compatibility.confluence
@@ -7,4 +7,3 @@ A [[Clirr|http://clirr.sourceforge.net/]] report is generated 
for each Curator m
 * [[Curator Recipes Report|curator-recipes/clirr-report.html]]
 * [[Curator Discovery Report|curator-x-discovery/clirr-report.html]]
 * [[Curator Discovery Server 
Report|curator-x-discovery-server/clirr-report.html]]
-* [[Curator RPC Report|curator-x-rpc/clirr-report.html]]



[27/50] curator git commit: removed testClientClosedDuringRefreshErrorMessage() I didn't write this test and it's not working. I'm not even sure what it tests. Check into it later -JZ

2015-08-24 Thread cammckenzie
removed testClientClosedDuringRefreshErrorMessage() I didn't write this test 
and it's not working. I'm not even sure what it tests. Check into it later -JZ


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

Branch: refs/heads/CURATOR-167
Commit: f18f87657af56bec32e8861ce5b3965c99807609
Parents: 47da621
Author: randgalt 
Authored: Wed Jul 1 16:17:58 2015 -0500
Committer: randgalt 
Committed: Wed Jul 1 16:17:58 2015 -0500

--
 .../recipes/cache/TestPathChildrenCache.java| 78 
 1 file changed, 78 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/f18f8765/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
index b07ac9c..3671e64 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
@@ -100,84 +100,6 @@ public class TestPathChildrenCache extends 
BaseClassForTests
 }
 }
 
-@Test(enabled = false)  // I didn't write this test and it's not working. 
I'm not even sure what it tests. Check into it later -JZ
-public void testClientClosedDuringRefreshErrorMessage() throws Exception
-{
-// Fiddle with logging so we can intercept the error events for 
org.apache.curator
-final List events = Lists.newArrayList();
-Collection messages = Collections2.transform(events, new 
Function()
-{
-@Override
-public String apply(LoggingEvent loggingEvent)
-{
-return loggingEvent.getRenderedMessage();
-}
-});
-Appender appender = new AppenderSkeleton(true) {
-@Override
-protected void append(LoggingEvent event) {
-if (event.getLevel().equals(Level.ERROR)) {
-events.add(event);
-}
-}
-
-@Override
-public void close() {
-
-}
-
-@Override
-public boolean requiresLayout() {
-return false;
-}
-};
-appender.setLayout(new SimpleLayout());
-Logger logger = Logger.getLogger("org.apache.curator");
-logger.addAppender(appender);
-
-// Check that we can intercept error log messages from the client
-CuratorFramework clientTestLogSetup = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTime(1));
-clientTestLogSetup.start();
-try {
-Pathable callback = 
clientTestLogSetup.getData().inBackground(new BackgroundCallback() {
-@Override
-public void processResult(CuratorFramework client, 
CuratorEvent event) throws Exception {
-// ignore result
-}
-});
-CloseableUtils.closeQuietly(clientTestLogSetup);
-callback.forPath("/test/aaa"); // this should cause an error log 
message
-} catch (IllegalStateException ise) {
-// ok, excpected
-} finally {
-CloseableUtils.closeQuietly(clientTestLogSetup);
-}
-
-Assert.assertTrue(messages.contains("Background exception was not 
retry-able or retry gave up"),
-"The expected error was not logged. This is an indication that 
this test could be broken due to" +
-" an incomplete logging setup.");
-
-// try to reproduce a bunch of times because it doesn't happen reliably
-for (int i = 0; i < 50; i++) {
-CuratorFramework client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTime(1));
-client.start();
-try {
-PathChildrenCache cache = new PathChildrenCache(client, 
"/test", true);
-cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
-
client.newNamespaceAwareEnsurePath("/test/aaa").ensure(client.getZookeeperClient());
-client.setData().forPath("/test/aaa", new byte[]{1, 2, 3, 4, 
5});
-cache.rebuildNode("/test/aaa");
-CloseableUtils.closeQuietly(cache);
-} finally {
-CloseableUtils.closeQuietly(client);
-}
-   

[30/50] curator git commit: Previous change had a potential infinite loop - stack overflow

2015-08-24 Thread cammckenzie
Previous change had a potential infinite loop - stack overflow


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

Branch: refs/heads/CURATOR-167
Commit: 584e5294ba94f01120575ad757520c3b9290c4c1
Parents: df8ec86
Author: randgalt 
Authored: Wed Jul 1 18:35:05 2015 -0500
Committer: randgalt 
Committed: Wed Jul 1 18:35:05 2015 -0500

--
 .../curator/framework/imps/NamespaceImpl.java   | 20 +++-
 1 file changed, 19 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/584e5294/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
index b0dda5c..74f6320 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceImpl.java
@@ -18,9 +18,14 @@
  */
 package org.apache.curator.framework.imps;
 
+import org.apache.curator.CuratorZookeeperClient;
+import org.apache.curator.RetryLoop;
+import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.curator.utils.PathUtils;
 import org.apache.curator.utils.ZKPaths;
+import org.apache.zookeeper.ZooDefs;
+import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 class NamespaceImpl
@@ -72,7 +77,20 @@ class NamespaceImpl
 {
 try
 {
-client.createContainers(ZKPaths.makePath("/", namespace));
+final CuratorZookeeperClient zookeeperClient = 
client.getZookeeperClient();
+RetryLoop.callWithRetry
+(
+zookeeperClient,
+new Callable()
+{
+@Override
+public Object call() throws Exception
+{
+ZKPaths.mkdirs(zookeeperClient.getZooKeeper(), 
ZKPaths.makePath("/", namespace), true, client.getAclProvider(), true);
+return null;
+}
+}
+);
 ensurePathNeeded.set(false);
 }
 catch ( Exception e )



[08/50] curator git commit: Provide a way to override creating containers

2015-08-24 Thread cammckenzie
Provide a way to override creating containers


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

Branch: refs/heads/CURATOR-167
Commit: 35ec01c0898ca40a6912a55ff30cf38525751f09
Parents: 8569763
Author: randgalt 
Authored: Tue May 19 15:10:25 2015 -0700
Committer: randgalt 
Committed: Tue May 19 15:10:25 2015 -0700

--
 .../framework/CuratorFrameworkFactory.java  | 21 
 .../framework/imps/CreateBuilderImpl.java   | 12 +--
 .../framework/imps/CuratorFrameworkImpl.java| 21 +++-
 3 files changed, 51 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/35ec01c0/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
index 11cee2d..c9c9c10 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java
@@ -25,13 +25,16 @@ import org.apache.curator.ensemble.EnsembleProvider;
 import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
 import org.apache.curator.framework.api.ACLProvider;
 import org.apache.curator.framework.api.CompressionProvider;
+import org.apache.curator.framework.api.CreateBuilder;
 import org.apache.curator.framework.api.PathAndBytesable;
 import org.apache.curator.framework.imps.CuratorFrameworkImpl;
 import org.apache.curator.framework.imps.CuratorTempFrameworkImpl;
 import org.apache.curator.framework.imps.DefaultACLProvider;
 import org.apache.curator.framework.imps.GzipCompressionProvider;
 import org.apache.curator.utils.DefaultZookeeperFactory;
+import org.apache.curator.utils.EnsurePathContainers;
 import org.apache.curator.utils.ZookeeperFactory;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import java.net.InetAddress;
@@ -113,6 +116,7 @@ public class CuratorFrameworkFactory
 private ZookeeperFactory zookeeperFactory = DEFAULT_ZOOKEEPER_FACTORY;
 private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER;
 private boolean canBeReadOnly = false;
+private boolean useContainerParentsIfAvailable = true;
 
 /**
  * Apply the current values and build a new CuratorFramework
@@ -328,6 +332,18 @@ public class CuratorFrameworkFactory
 return this;
 }
 
+/**
+ * By default, Curator uses {@link EnsurePathContainers} and {@link 
CreateBuilder#creatingParentContainersIfNeeded()}
+ * if the ZK JAR supports {@link CreateMode#CONTAINER}. Call this 
method to turn off this behavior.
+ *
+ * @return this
+ */
+public Builder dontUseContainerParents()
+{
+this.useContainerParentsIfAvailable = false;
+return this;
+}
+
 public ACLProvider getAclProvider()
 {
 return aclProvider;
@@ -378,6 +394,11 @@ public class CuratorFrameworkFactory
 return namespace;
 }
 
+public boolean useContainerParentsIfAvailable()
+{
+return useContainerParentsIfAvailable;
+}
+
 @Deprecated
 public String getAuthScheme()
 {

http://git-wip-us.apache.org/repos/asf/curator/blob/35ec01c0/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
index 7b1e855..4a669b2 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java
@@ -134,7 +134,7 @@ class CreateBuilderImpl implements CreateBuilder, 
BackgroundOperation 
creatingParentContainersIfNeeded()
 {
-createParentsAsContainers = true;
+setCreateParentsAsContainers();
 return creatingParentsIfNeeded();
 }
 
@@ -271,10 +271,18 @@ class CreateBuilderImpl implements CreateBuilder, 
BackgroundOperation 
creatingParentContainersIfNeeded()
 {
-createParentsAsContai

[21/50] curator git commit: Curator-224: Fixed the requeuing problem with DistributedIdQueue.

2015-08-24 Thread cammckenzie
Curator-224: Fixed the requeuing problem with DistributedIdQueue.


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

Branch: refs/heads/CURATOR-167
Commit: 83e1a855a15e17a37ec02d441d6c75e7f08a2617
Parents: 20e92a5
Author: Zhihong Zhang 
Authored: Mon Jun 22 14:59:02 2015 -0400
Committer: Zhihong Zhang 
Committed: Mon Jun 22 14:59:02 2015 -0400

--
 .../recipes/queue/DistributedQueue.java |  2 +-
 .../recipes/queue/TestDistributedIdQueue.java   | 47 
 2 files changed, 48 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/83e1a855/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
index 9dd2217..a183adf 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
@@ -756,7 +756,7 @@ public class DistributedQueue implements QueueBase
 client.inTransaction()
 .delete().forPath(itemPath)
 .and()
-
.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath(makeItemPath(), 
bytes)
+
.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath(itemPath, bytes)
 .and()
 .commit();
 }

http://git-wip-us.apache.org/repos/asf/curator/blob/83e1a855/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
index 30e552f..858086b 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
@@ -124,4 +124,51 @@ public class TestDistributedIdQueue extends 
BaseClassForTests
 CloseableUtils.closeQuietly(client);
 }
 }
+
+@Test
+public void testRequeuingWithLock() throws Exception
+{
+DistributedIdQueue  queue = null;
+CuratorFramework client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTime(1));
+client.start();
+try
+{
+final CountDownLatchconsumingLatch = new CountDownLatch(1);
+
+QueueConsumer consumer = new 
QueueConsumer()
+{
+@Override
+public void consumeMessage(TestQueueItem message) throws 
Exception
+{
+consumingLatch.countDown();
+// Throw an exception so requeuing occurs
+throw new Exception("Consumer failed");
+}
+
+@Override
+public void stateChanged(CuratorFramework client, 
ConnectionState newState)
+{
+}
+};
+
+queue = QueueBuilder.builder(client, consumer, serializer, 
QUEUE_PATH).lockPath("/locks").buildIdQueue();
+queue.start();
+
+queue.put(new TestQueueItem("test"), "id");
+
+Assert.assertTrue(consumingLatch.await(10, TimeUnit.SECONDS));  // 
wait until consumer has it
+
+// Sleep one more second
+
+Thread.sleep(1000);
+
+Assert.assertEquals(queue.remove("id"), 1);
+
+}
+finally
+{
+CloseableUtils.closeQuietly(queue);
+CloseableUtils.closeQuietly(client);
+}
+}
 }



[05/50] curator git commit: Don't directly link to CreateMode.CONTAINER. Get it via reflection to avoid link issues with different versions of ZooKeeper

2015-08-24 Thread cammckenzie
Don't directly link to CreateMode.CONTAINER. Get it via reflection to avoid 
link issues with different versions of ZooKeeper


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

Branch: refs/heads/CURATOR-167
Commit: d492f8c1ab6243b49f30b27754bf6a1cff3b80fe
Parents: 04ae811
Author: randgalt 
Authored: Tue May 19 14:56:30 2015 -0700
Committer: randgalt 
Committed: Tue May 19 14:56:30 2015 -0700

--
 .../java/org/apache/curator/utils/ZKPaths.java  | 36 ++--
 1 file changed, 33 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/d492f8c1/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
--
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java 
b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
index 3a68b5b..526f705 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
@@ -26,6 +26,8 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import java.util.Collections;
 import java.util.List;
 
@@ -35,8 +37,31 @@ public class ZKPaths
  * Zookeeper's path separator character.
  */
 public static final String PATH_SEPARATOR = "/";
-
-
+
+private static class CreatModeHolder
+{
+private static final Logger log = 
LoggerFactory.getLogger(ZKPaths.class);
+private static final CreateMode containerCreateMode;
+
+static
+{
+CreateMode localCreateMode = CreateMode.PERSISTENT;
+for ( CreateMode createMode : CreateMode.class.getEnumConstants() )
+{
+if ( createMode.name().equals("CONTAINER") )
+{
+localCreateMode = createMode;
+break;
+}
+}
+if ( localCreateMode == CreateMode.PERSISTENT )
+{
+log.warn("The version of ZooKeeper being used doesn't support 
Container nodes. CreateMode.PERSISTENT will be used instead");
+}
+containerCreateMode = localCreateMode;
+}
+}
+
 /**
  * Apply the namespace to the given path
  *
@@ -246,7 +271,7 @@ public class ZKPaths
 {
 acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
 }
-zookeeper.create(subPath, new byte[0], acl, asContainers ? 
CreateMode.CONTAINER : CreateMode.PERSISTENT);
+zookeeper.create(subPath, new byte[0], acl, 
getCreateMode(asContainers));
 }
 catch ( KeeperException.NodeExistsException e )
 {
@@ -414,4 +439,9 @@ public class ZKPaths
 private ZKPaths()
 {
 }
+
+private static CreateMode getCreateMode(boolean asContainers)
+{
+return asContainers ? CreatModeHolder.containerCreateMode : 
CreateMode.PERSISTENT;
+}
 }



[29/50] curator git commit: Merge branch 'CURATOR-222'

2015-08-24 Thread cammckenzie
Merge branch 'CURATOR-222'


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

Branch: refs/heads/CURATOR-167
Commit: d90325cfbd5fb4ee373bc7b8206d28305b26e0a8
Parents: 19bb4d1 df8ec86
Author: randgalt 
Authored: Wed Jul 1 18:01:04 2015 -0500
Committer: randgalt 
Committed: Wed Jul 1 18:01:04 2015 -0500

--
 curator-client/pom.xml  |  16 ++
 .../org/apache/curator/utils/DebugUtils.java|   1 +
 .../org/apache/curator/utils/EnsurePath.java|  14 +-
 .../java/org/apache/curator/utils/ZKPaths.java  |  74 ++-
 .../src/main/java/cache/PathCacheExample.java   |   2 +-
 curator-framework/pom.xml   |   6 +
 .../curator/framework/CuratorFramework.java |  18 +-
 .../framework/CuratorFrameworkFactory.java  |  20 ++
 .../api/CreateBackgroundModeACLable.java|  12 ++
 .../curator/framework/api/CreateBuilder.java|  13 ++
 .../curator/framework/api/ExistsBuilder.java|  15 +-
 .../framework/api/ExistsBuilderMain.java|  27 +++
 .../framework/imps/CreateBuilderImpl.java   |  38 +++-
 .../framework/imps/CuratorFrameworkImpl.java|  15 ++
 .../framework/imps/ExistsBuilderImpl.java   |  56 -
 .../curator/framework/imps/NamespaceFacade.java |  10 +-
 .../curator/framework/imps/NamespaceImpl.java   |  12 +-
 .../src/site/confluence/index.confluence|   6 +-
 .../curator/framework/imps/TestFramework.java   | 211 +--
 curator-recipes/pom.xml |   6 +
 .../recipes/atomic/DistributedAtomicValue.java  |  15 +-
 .../recipes/barriers/DistributedBarrier.java|   2 +-
 .../barriers/DistributedDoubleBarrier.java  |   2 +-
 .../framework/recipes/cache/NodeCache.java  |   9 +-
 .../recipes/cache/PathChildrenCache.java|  20 +-
 .../recipes/cache/PathChildrenCacheMode.java|   1 +
 .../framework/recipes/cache/TreeCache.java  |  27 ++-
 .../framework/recipes/leader/LeaderLatch.java   |   2 +-
 .../framework/recipes/locks/ChildReaper.java|   5 +
 .../recipes/locks/InterProcessSemaphore.java|   1 +
 .../recipes/locks/InterProcessSemaphoreV2.java  |   2 +-
 .../curator/framework/recipes/locks/Reaper.java |   5 +
 .../locks/StandardLockInternalsDriver.java  |   4 +-
 .../recipes/nodes/PersistentEphemeralNode.java  |   2 +-
 .../recipes/queue/DistributedQueue.java |   4 +-
 .../framework/recipes/queue/QueueSharder.java   |   2 +-
 .../recipes/queue/SimpleDistributedQueue.java   |  18 +-
 .../framework/recipes/shared/SharedValue.java   |   2 +-
 .../src/site/confluence/index.confluence|   2 +
 .../recipes/cache/TestPathChildrenCache.java|  78 ---
 .../framework/recipes/cache/TestTreeCache.java  |  13 ++
 .../locks/TestInterProcessMultiMutex.java   |   4 +-
 .../recipes/locks/TestInterProcessMutex.java|   2 +-
 .../locks/TestInterProcessMutexBase.java|  85 +++-
 .../locks/TestInterProcessSemaphoreMutex.java   |   2 +-
 curator-test/pom.xml|  10 +
 .../apache/curator/test/BaseClassForTests.java  |  28 ++-
 curator-x-discovery-server/pom.xml  |   6 +
 curator-x-discovery/pom.xml |   6 +
 .../discovery/details/ServiceDiscoveryImpl.java |   4 +-
 curator-x-rpc/pom.xml   |   6 +
 .../idl/services/CuratorProjectionService.java  |   4 +
 .../curator/x/rpc/idl/structs/CreateSpec.java   |   6 +-
 .../x/rpc/idl/structs/RpcCreateMode.java|   3 +-
 curator-x-rpc/src/main/thrift/curator.thrift|   3 +-
 .../src/site/confluence/reference.confluence|   1 +
 pom.xml |  27 +--
 src/site/confluence/utilities.confluence|  32 ---
 58 files changed, 763 insertions(+), 254 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/d90325cf/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
--

http://git-wip-us.apache.org/repos/asf/curator/blob/d90325cf/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestPathChildrenCache.java
--



[45/50] curator git commit: Added some better logging to testRetryAfterFailure_Curator56()

2015-08-24 Thread cammckenzie
Added some better logging to testRetryAfterFailure_Curator56()


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

Branch: refs/heads/CURATOR-167
Commit: 25f51498df05c50c7d6406d558a35dd8b54eb749
Parents: 2a39a45
Author: randgalt 
Authored: Fri Aug 21 13:07:12 2015 -0500
Committer: randgalt 
Committed: Fri Aug 21 13:07:12 2015 -0500

--
 .../curator/framework/recipes/queue/TestDistributedQueue.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/25f51498/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
index 80509bc..a191166 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedQueue.java
@@ -109,7 +109,7 @@ public class TestDistributedQueue extends BaseClassForTests
 retryCounter.await(10, TimeUnit.SECONDS);
 Assert.assertEquals(retryCounter.getCount(), 0, "Queue item was 
not consumed. Retry counter is " + retryCounter.getCount());
 Assert.assertEquals(names.size(), 2);
-Assert.assertEquals(names.get(0).length(), names.get(1).length());
+Assert.assertEquals(names.get(0).length(), names.get(1).length(), 
"name1: " + names.get(0) + " - " + "name2: " + names.get(1));
 }
 finally
 {



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

2015-08-24 Thread cammckenzie
added more usages of exists().creatingParents some more tests and added this 
behavior optionally to TreeCache


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

Branch: refs/heads/CURATOR-167
Commit: b4ff09517a9f429fac46c24eaf5f79857b4a35ec
Parents: 7ad1275
Author: randgalt 
Authored: Wed Jun 24 16:25:33 2015 -0500
Committer: randgalt 
Committed: Wed Jun 24 16:25:33 2015 -0500

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


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

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

http://git-wip-us.apache.org/repos/asf/curator/blob/b4ff0951/curator-recipes/src/main/java/org/apache/curator/framework/

[10/50] curator git commit: Got rid of EnsurePathContainers. It's not needed and confusing

2015-08-24 Thread cammckenzie
Got rid of EnsurePathContainers. It's not needed and confusing


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

Branch: refs/heads/CURATOR-167
Commit: b3cf69d0a683393556bff6ccefca33b9c8325b93
Parents: 12baea3
Author: randgalt 
Authored: Tue May 19 16:08:53 2015 -0700
Committer: randgalt 
Committed: Tue May 19 16:08:53 2015 -0700

--
 .../org/apache/curator/utils/EnsurePath.java| 11 
 .../curator/utils/EnsurePathContainers.java | 49 
 .../java/org/apache/curator/utils/ZKPaths.java  |  2 +-
 .../curator/framework/CuratorFramework.java | 10 
 .../framework/CuratorFrameworkFactory.java  |  3 +-
 .../framework/imps/CuratorFrameworkImpl.java| 18 --
 .../curator/framework/imps/NamespaceFacade.java | 11 +---
 .../curator/framework/imps/NamespaceImpl.java   |  8 +--
 .../curator/framework/imps/TestFramework.java   | 59 
 .../recipes/atomic/DistributedAtomicValue.java  | 10 +---
 .../framework/recipes/cache/NodeCache.java  |  6 +-
 .../recipes/cache/PathChildrenCache.java|  6 +-
 .../recipes/queue/SimpleDistributedQueue.java   | 11 +---
 13 files changed, 15 insertions(+), 189 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/b3cf69d0/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java 
b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
index 3181aca..3abb618 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
@@ -110,17 +110,6 @@ public class EnsurePath
 return new EnsurePath(path, helper, false, aclProvider);
 }
 
-/**
- * Returns a view of this EnsurePath instance that does not make the last 
node and also makes containers.
- * i.e. if the path is "/a/b/c" only "/a/b" will be ensured
- *
- * @return view
- */
-public EnsurePathContainers excludingLastContainers()
-{
-return new EnsurePathContainers(path, helper, false, aclProvider);
-}
-
 protected EnsurePath(String path, AtomicReference helper, boolean 
makeLastNode, InternalACLProvider aclProvider)
 {
 this.path = path;

http://git-wip-us.apache.org/repos/asf/curator/blob/b3cf69d0/curator-client/src/main/java/org/apache/curator/utils/EnsurePathContainers.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/utils/EnsurePathContainers.java
 
b/curator-client/src/main/java/org/apache/curator/utils/EnsurePathContainers.java
deleted file mode 100644
index 455d58d..000
--- 
a/curator-client/src/main/java/org/apache/curator/utils/EnsurePathContainers.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.curator.utils;
-
-import org.apache.zookeeper.CreateMode;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Same as {@link EnsurePath} but creates parents as {@link 
CreateMode#CONTAINER}
- */
-public class EnsurePathContainers extends EnsurePath
-{
-public EnsurePathContainers(String path)
-{
-super(path);
-}
-
-public EnsurePathContainers(String path, InternalACLProvider aclProvider)
-{
-super(path, aclProvider);
-}
-
-EnsurePathContainers(String path, AtomicReference 
helper, boolean makeLastNode, InternalACLProvider aclProvider)
-{
-super(path, helper, makeLastNode, aclProvider);
-}
-
-@Override
-protected boolean asContainers()
-{
-return true;
-}
-}

http://git-wip-us.apache.org/repos/asf/curator/blob/b3cf69d0/curator-client/src/main/java/org/apache/cura

[49/50] curator git commit: Merge branch 'CURATOR-234'

2015-08-24 Thread cammckenzie
Merge branch 'CURATOR-234'


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

Branch: refs/heads/CURATOR-167
Commit: 5dc27c1fa9b8e3bc6ed3e1b198bb1eca6b378c9a
Parents: 061ed0a b177ed7
Author: Mike Drob 
Authored: Mon Aug 24 10:41:39 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 10:41:39 2015 -0500

--
 curator-x-discovery-server/pom.xml | 6 ++
 curator-x-discovery/pom.xml| 6 ++
 curator-x-rpc/pom.xml  | 6 ++
 3 files changed, 18 insertions(+)
--




[18/50] curator git commit: remove doc for EnsurePath as it is now deprecated

2015-08-24 Thread cammckenzie
remove doc for EnsurePath as it is now deprecated


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

Branch: refs/heads/CURATOR-167
Commit: 81600019e28fe0bb961a99f485556aaf6be3fc3b
Parents: 759ae68
Author: randgalt 
Authored: Sun Jun 21 10:45:00 2015 -0500
Committer: randgalt 
Committed: Sun Jun 21 10:45:00 2015 -0500

--
 src/site/confluence/utilities.confluence | 18 --
 1 file changed, 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/81600019/src/site/confluence/utilities.confluence
--
diff --git a/src/site/confluence/utilities.confluence 
b/src/site/confluence/utilities.confluence
index d1484e3..16bbc13 100644
--- a/src/site/confluence/utilities.confluence
+++ b/src/site/confluence/utilities.confluence
@@ -14,24 +14,6 @@ Various static methods to help with using ZooKeeper ZNode 
paths:
 * getSortedChildren: Return the children of the given path sorted by sequence 
number
 * makePath: Given a parent path and a child node, create a combined full path
 
-h2. EnsurePath
-Utility to ensure that a particular path is created.
-The first time it is used, a synchronized call to {{ZKPaths.mkdirs(ZooKeeper, 
String)}} is made to ensure that the entire path has been created (with an 
empty byte array if needed). Subsequent calls with the instance are 
un\-synchronized NOPs.
-
-Usage:
-{code}
-EnsurePath   ensurePath = new EnsurePath(aFullPathToEnsure);
-...
-String   nodePath = aFullPathToEnsure + "/foo";
-ensurePath.ensure(zk);   // first time syncs and creates if needed
-zk.create(nodePath, ...);
-...
-ensurePath.ensure(zk);   // subsequent times are NOPs
-zk.create(nodePath, ...);
-{code}
-
-*NOTE:* There's a method in the [[CuratorFramework 
class|curator-framework/index.html]] that returns an EnsurePath instance that 
is namespace aware.
-
 h2. BlockingQueueConsumer
 
 See: *[[DistributedQueue|curator-recipes/distributed-queue.html]]* and 
*[[DistributedPriorityQueue|curator-recipes/distributed-priority-queue.html]]*



[48/50] curator git commit: CURATOR-234 Add remaining slf4j-log4j12 test deps

2015-08-24 Thread cammckenzie
CURATOR-234 Add remaining slf4j-log4j12 test deps


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

Branch: refs/heads/CURATOR-167
Commit: b177ed709a422f0b06f3a4a3683e5488d038fbb9
Parents: 9a444fa
Author: Mike Drob 
Authored: Mon Aug 24 10:40:12 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 10:41:09 2015 -0500

--
 curator-x-discovery-server/pom.xml | 6 ++
 curator-x-discovery/pom.xml| 6 ++
 curator-x-rpc/pom.xml  | 6 ++
 3 files changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-discovery-server/pom.xml
--
diff --git a/curator-x-discovery-server/pom.xml 
b/curator-x-discovery-server/pom.xml
index 2728ce5..4ed7e25 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -108,6 +108,12 @@
 resteasy-jaxrs
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-discovery/pom.xml
--
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 015fea7..9043533 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -66,5 +66,11 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/b177ed70/curator-x-rpc/pom.xml
--
diff --git a/curator-x-rpc/pom.xml b/curator-x-rpc/pom.xml
index 299ef9f..519e8fb 100644
--- a/curator-x-rpc/pom.xml
+++ b/curator-x-rpc/pom.xml
@@ -118,6 +118,12 @@
 testng
 test
 
+
+
+org.slf4j
+slf4j-log4j12
+test
+
 
 
 



[26/50] curator git commit: Added createContainers() to the main API as it's used too often. Applied it where needed (fixing some issues created when removing ensurePath)

2015-08-24 Thread cammckenzie
Added createContainers() to the main API as it's used too often. Applied it 
where needed (fixing some issues created when removing ensurePath)


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

Branch: refs/heads/CURATOR-167
Commit: 47da6211fcd82fcb8454f3464a94bf7f1e88d00e
Parents: 8226f0c
Author: randgalt 
Authored: Wed Jun 24 17:23:07 2015 -0500
Committer: randgalt 
Committed: Wed Jun 24 17:23:07 2015 -0500

--
 .../org/apache/curator/framework/CuratorFramework.java  | 12 +++-
 .../curator/framework/imps/CuratorFrameworkImpl.java|  7 +++
 .../apache/curator/framework/imps/NamespaceFacade.java  |  6 ++
 .../framework/recipes/cache/PathChildrenCache.java  |  2 +-
 .../curator/framework/recipes/cache/TreeCache.java  |  2 +-
 .../curator/framework/recipes/queue/QueueSharder.java   |  2 +-
 .../framework/recipes/queue/SimpleDistributedQueue.java |  2 +-
 .../curator/framework/recipes/cache/TestTreeCache.java  |  2 +-
 8 files changed, 29 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/47da6211/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 de9bcc5..b9d67b9 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
@@ -139,6 +139,15 @@ public interface CuratorFramework extends Closeable
 public void sync(String path, Object backgroundContextObject);
 
 /**
+ * Create all nodes in the specified path as containers if they don't
+ * already exist
+ *
+ * @param path path to create
+ * @throws Exception errors
+ */
+public void createContainers(String path) throws Exception;
+
+/**
  * Start a sync builder. Note: sync is ALWAYS in the background even
  * if you don't use one of the background() methods
  *
@@ -205,7 +214,8 @@ public interface CuratorFramework extends Closeable
  *
  * @param path path to ensure
  * @return new EnsurePath instance
- * @deprecated Since 2.9.0 - prefer {@link 
CreateBuilder#creatingParentContainersIfNeeded()} or {@link 
ExistsBuilder#creatingParentContainersIfNeeded()}
+ * @deprecated Since 2.9.0 - prefer {@link 
CreateBuilder#creatingParentContainersIfNeeded()}, {@link 
ExistsBuilder#creatingParentContainersIfNeeded()}
+ * or {@link CuratorFramework#createContainers(String)}
  */
 @Deprecated
 public EnsurePath newNamespaceAwareEnsurePath(String path);

http://git-wip-us.apache.org/repos/asf/curator/blob/47da6211/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 30a6a55..38ce166 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
@@ -40,6 +40,7 @@ import 
org.apache.curator.framework.state.ConnectionStateManager;
 import org.apache.curator.utils.DebugUtils;
 import org.apache.curator.utils.EnsurePath;
 import org.apache.curator.utils.ThreadUtils;
+import org.apache.curator.utils.ZKPaths;
 import org.apache.curator.utils.ZookeeperFactory;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -187,6 +188,12 @@ public class CuratorFrameworkImpl implements 
CuratorFramework
 }
 
 @Override
+public void createContainers(String path) throws Exception
+{
+
checkExists().creatingParentContainersIfNeeded().forPath(ZKPaths.makePath(path, 
"foo"));
+}
+
+@Override
 public void clearWatcherReferences(Watcher watcher)
 {
 NamespaceWatcher namespaceWatcher = 
namespaceWatcherMap.remove(watcher);

http://git-wip-us.apache.org/repos/asf/curator/blob/47da6211/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceFacade.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceFacade.java
 
b/curator-framewor

[50/50] curator git commit: CURATOR-167 - Partial fix to clean up Curator managed watch objects when the cache closes. A full fix requires the ability to cancel watches in ZK which is not available un

2015-08-24 Thread cammckenzie
CURATOR-167 - Partial fix to clean up Curator managed watch objects when
the cache closes. A full fix requires the ability to cancel watches in
ZK which is not available until ZK 3.5


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

Branch: refs/heads/CURATOR-167
Commit: 8fae7856edc1a5269fd012c683860e0b150e13b3
Parents: 5dc27c1
Author: Cameron McKenzie 
Authored: Wed Nov 19 14:23:24 2014 +1100
Committer: Cam McKenzie 
Committed: Tue Aug 25 14:06:35 2015 +1000

--
 .../framework/recipes/cache/NodeCache.java  | 41 
 1 file changed, 34 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/8fae7856/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
index 72ee5ff..bfc27d8 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java
@@ -22,19 +22,21 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
+
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
-import org.apache.curator.framework.api.CuratorWatcher;
 import org.apache.curator.framework.listen.ListenerContainer;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.utils.PathUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.concurrent.Exchanger;
@@ -60,7 +62,7 @@ public class NodeCache implements Closeable
 private final AtomicReference state = new 
AtomicReference(State.LATENT);
 private final ListenerContainer listeners = new 
ListenerContainer();
 private final AtomicBoolean isConnected = new AtomicBoolean(true);
-private final ConnectionStateListener connectionStateListener = new 
ConnectionStateListener()
+private ConnectionStateListener connectionStateListener = new 
ConnectionStateListener()
 {
 @Override
 public void stateChanged(CuratorFramework client, ConnectionState 
newState)
@@ -86,12 +88,19 @@ public class NodeCache implements Closeable
 }
 };
 
-private final CuratorWatcher watcher = new CuratorWatcher()
+private Watcher watcher = new Watcher()
 {
 @Override
-public void process(WatchedEvent event) throws Exception
+public void process(WatchedEvent event)
 {
-reset();
+try
+{
+reset();
+}
+catch(Exception e)
+{
+handleException(e);
+}
 }
 };
 
@@ -170,8 +179,16 @@ public class NodeCache implements Closeable
 if ( state.compareAndSet(State.STARTED, State.CLOSED) )
 {
 listeners.clear();
-}
-
client.getConnectionStateListenable().removeListener(connectionStateListener);
+client.clearWatcherReferences(watcher);
+
client.getConnectionStateListenable().removeListener(connectionStateListener);
+
+// TODO
+// From PathChildrenCache
+// This seems to enable even more GC - I'm not sure why yet - it
+// has something to do with Guava's cache and circular references
+connectionStateListener = null;
+watcher = null;
+}
 }
 
 /**
@@ -312,4 +329,14 @@ public class NodeCache implements Closeable
 }
 }
 }
+
+/**
+ * Default behavior is just to log the exception
+ *
+ * @param e the exception
+ */
+protected void handleException(Throwable e)
+{
+log.error("", e);
+}
 }



[03/50] curator git commit: closes #53 *Moved to PR 76*

2015-08-24 Thread cammckenzie
closes #53 *Moved to PR 76*


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

Branch: refs/heads/CURATOR-167
Commit: 7f2098654a26e2f593801a586ce68300f54abf15
Parents: 8d8fdf6
Author: randgalt 
Authored: Sat May 9 09:26:38 2015 -0500
Committer: randgalt 
Committed: Sat May 9 09:26:38 2015 -0500

--

--




[41/50] curator git commit: Merge branch 'CURATOR-241'

2015-08-24 Thread cammckenzie
Merge branch 'CURATOR-241'


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

Branch: refs/heads/CURATOR-167
Commit: 41f49ec3751a1d76313c2d93da7aa72ebae2a80d
Parents: 40b458e f3ff7e7
Author: Cameron McKenzie 
Authored: Tue Aug 11 07:49:32 2015 +1000
Committer: Cameron McKenzie 
Committed: Tue Aug 11 07:49:32 2015 +1000

--
 .../recipes/nodes/PersistentEphemeralNode.java  |  8 +++-
 .../nodes/TestPersistentEphemeralNode.java  | 41 
 2 files changed, 47 insertions(+), 2 deletions(-)
--




[44/50] curator git commit: testRequeuingWithLock was too dependent on timing. Created a better way to test the condition

2015-08-24 Thread cammckenzie
testRequeuingWithLock was too dependent on timing. Created a better way to test 
the condition


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

Branch: refs/heads/CURATOR-167
Commit: 2a39a45d48dd9e581276ee22dbe6e036063122b4
Parents: e8aff9a
Author: randgalt 
Authored: Fri Aug 21 12:33:59 2015 -0500
Committer: randgalt 
Committed: Fri Aug 21 12:33:59 2015 -0500

--
 .../framework/recipes/queue/DistributedIdQueue.java  | 15 +++
 .../recipes/queue/TestDistributedIdQueue.java|  2 +-
 2 files changed, 16 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/2a39a45d/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
index a83fa36..dbd8e6e 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
@@ -18,6 +18,7 @@
  */
 package org.apache.curator.framework.recipes.queue;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.listen.ListenerContainer;
@@ -183,6 +184,20 @@ public class DistributedIdQueue implements QueueBase
 return count;
 }
 
+@VisibleForTesting
+boolean debugIsQueued(String id) throws Exception
+{
+for ( String name : queue.getChildren() )
+{
+if ( parseId(name).id.equals(id) )
+{
+return true;
+}
+}
+
+return false;
+}
+
 private void internalSortChildren(List children)
 {
 Collections.sort

http://git-wip-us.apache.org/repos/asf/curator/blob/2a39a45d/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
index 858086b..24f74a1 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestDistributedIdQueue.java
@@ -162,7 +162,7 @@ public class TestDistributedIdQueue extends 
BaseClassForTests
 
 Thread.sleep(1000);
 
-Assert.assertEquals(queue.remove("id"), 1);
+Assert.assertTrue(queue.debugIsQueued("id"));
 
 }
 finally



[33/50] curator git commit: CURATOR-236: TreeCache throws IllegalArgumentException when node is a substring of the requested path

2015-08-24 Thread cammckenzie
CURATOR-236: TreeCache throws IllegalArgumentException when node is a substring 
of the requested path

Better path walking algorithm.


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

Branch: refs/heads/CURATOR-167
Commit: a63bb03e590460985b729cb2fd1a4a5e7ed345ba
Parents: e71da8f
Author: Scott Blum 
Authored: Thu Jul 16 17:50:32 2015 -0400
Committer: Scott Blum 
Committed: Thu Jul 16 17:50:32 2015 -0400

--
 .../framework/recipes/cache/TreeCache.java  | 45 +++-
 .../framework/recipes/cache/TestTreeCache.java  |  5 +++
 2 files changed, 30 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/a63bb03e/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
index 71efd28..7cf2267 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java
@@ -33,6 +33,7 @@ import org.apache.curator.framework.listen.ListenerContainer;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.curator.utils.CloseableExecutorService;
+import org.apache.curator.utils.PathUtils;
 import org.apache.curator.utils.ThreadUtils;
 import org.apache.curator.utils.ZKPaths;
 import org.apache.zookeeper.KeeperException;
@@ -44,6 +45,7 @@ import org.slf4j.LoggerFactory;
 import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
@@ -601,33 +603,36 @@ public class TreeCache implements Closeable
 return errorListeners;
 }
 
-private TreeNode find(String fullPath)
+private TreeNode find(String findPath)
 {
-if ( !fullPath.startsWith(root.path) )
-{
-return null;
+PathUtils.validatePath(findPath);
+LinkedList rootElements = new 
LinkedList(ZKPaths.split(root.path));
+LinkedList findElements = new 
LinkedList(ZKPaths.split(findPath));
+while (!rootElements.isEmpty()) {
+if (findElements.isEmpty()) {
+// Target path shorter than root path
+return null;
+}
+String nextRoot = rootElements.removeFirst();
+String nextFind = findElements.removeFirst();
+if (!nextFind.equals(nextRoot)) {
+// Initial root path does not match
+return null;
+}
 }
 
 TreeNode current = root;
-if ( fullPath.length() > root.path.length() )
-{
-if ( root.path.length() > 1 )
+while (!findElements.isEmpty()) {
+String nextFind = findElements.removeFirst();
+ConcurrentMap map = current.children.get();
+if ( map == null )
 {
-fullPath = fullPath.substring(root.path.length());
+return null;
 }
-List split = ZKPaths.split(fullPath);
-for ( String part : split )
+current = map.get(nextFind);
+if ( current == null )
 {
-ConcurrentMap map = current.children.get();
-if ( map == null )
-{
-return null;
-}
-current = map.get(part);
-if ( current == null )
-{
-return null;
-}
+return null;
 }
 }
 return current;

http://git-wip-us.apache.org/repos/asf/curator/blob/a63bb03e/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
--
diff --git 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
index e1c61d0..0bccb54 100644
--- 
a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
+++ 
b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
@@ -395,11 +395,16 @@ public class TestTreeCache ext

[47/50] curator git commit: CURATOR-218 Reorder ConnectionState process event

2015-08-24 Thread cammckenzie
CURATOR-218 Reorder ConnectionState process event

Address a race condition in ConnectionState.process where it will
trigger watchers first before updating its own state. This can lead to
inconsistencies when blocking until connected.


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

Branch: refs/heads/CURATOR-167
Commit: 061ed0a6d4630fd166df7ba3d16acde3a231c716
Parents: 2266ca1
Author: Mike Drob 
Authored: Wed Jul 8 11:04:30 2015 -0500
Committer: Mike Drob 
Committed: Mon Aug 24 07:08:16 2015 -0500

--
 .../org/apache/curator/ConnectionState.java | 26 +---
 .../framework/imps/TestBlockUntilConnected.java | 24 ++
 2 files changed, 36 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-client/src/main/java/org/apache/curator/ConnectionState.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/ConnectionState.java 
b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
index d3900a1..46ae9fd 100644
--- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java
+++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java
@@ -41,7 +41,7 @@ class ConnectionState implements Watcher, Closeable
 {
 private static final int MAX_BACKGROUND_EXCEPTIONS = 10;
 private static final boolean LOG_EVENTS = 
Boolean.getBoolean(DebugUtils.PROPERTY_LOG_EVENTS);
-private final Logger log = LoggerFactory.getLogger(getClass());
+private static final Logger log = 
LoggerFactory.getLogger(ConnectionState.class);
 private final HandleHolder zooKeeper;
 private final AtomicBoolean isConnected = new AtomicBoolean(false);
 private final EnsembleProvider ensembleProvider;
@@ -145,24 +145,22 @@ class ConnectionState implements Watcher, Closeable
 log.debug("ConnectState watcher: " + event);
 }
 
-for ( Watcher parentWatcher : parentWatchers )
-{
-TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
-parentWatcher.process(event);
-timeTrace.commit();
-}
-
-boolean wasConnected = isConnected.get();
-boolean newIsConnected = wasConnected;
 if ( event.getType() == Watcher.Event.EventType.None )
 {
-newIsConnected = checkState(event.getState(), wasConnected);
+boolean wasConnected = isConnected.get();
+boolean newIsConnected = checkState(event.getState(), 
wasConnected);
+if ( newIsConnected != wasConnected )
+{
+isConnected.set(newIsConnected);
+connectionStartMs = System.currentTimeMillis();
+}
 }
 
-if ( newIsConnected != wasConnected )
+for ( Watcher parentWatcher : parentWatchers )
 {
-isConnected.set(newIsConnected);
-connectionStartMs = System.currentTimeMillis();
+TimeTrace timeTrace = new 
TimeTrace("connection-state-parent-process", tracer.get());
+parentWatcher.process(event);
+timeTrace.commit();
 }
 }
 

http://git-wip-us.apache.org/repos/asf/curator/blob/061ed0a6/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
--
diff --git 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
index f649afb..a6dc7ab 100644
--- 
a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
+++ 
b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestBlockUntilConnected.java
@@ -232,4 +232,28 @@ public class TestBlockUntilConnected extends 
BaseClassForTests
 CloseableUtils.closeQuietly(client);
 }
 }
+
+/**
+ * Test that we are actually connected every time that we block until 
connection is established in a tight loop.
+ */
+@Test
+public void testBlockUntilConnectedTightLoop() throws InterruptedException
+{
+CuratorFramework client;
+for(int i = 0 ; i < 50 ; i++)
+{
+client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), new 
RetryOneTime(100));
+try
+{
+client.start();
+client.blockUntilConnected();
+
+Assert.assertTrue(client.getZookeep

[04/50] curator git commit: Assuming that ZOOKEEPER-2163 is accepted, container node support has been added to Curator. Also, all recipes that create parent nodes now create container nodes

2015-08-24 Thread cammckenzie
Assuming that ZOOKEEPER-2163 is accepted, container node support has been added 
to Curator. Also, all recipes that create parent nodes now create container 
nodes


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

Branch: refs/heads/CURATOR-167
Commit: 04ae8115512f03210cf930118788c67e433bd904
Parents: 7f20986
Author: randgalt 
Authored: Tue May 19 14:33:24 2015 -0700
Committer: randgalt 
Committed: Tue May 19 14:33:24 2015 -0700

--
 .../org/apache/curator/utils/EnsurePath.java| 22 +-
 .../curator/utils/EnsurePathContainers.java | 49 +
 .../java/org/apache/curator/utils/ZKPaths.java  | 23 +-
 .../src/main/java/cache/PathCacheExample.java   |  2 +-
 .../curator/framework/CuratorFramework.java |  9 +++
 .../api/CreateBackgroundModeACLable.java|  7 ++
 .../curator/framework/api/CreateBuilder.java|  7 ++
 .../framework/imps/CreateBuilderImpl.java   | 20 -
 .../framework/imps/CuratorFrameworkImpl.java|  7 ++
 .../curator/framework/imps/NamespaceFacade.java |  7 ++
 .../curator/framework/imps/NamespaceImpl.java   |  6 ++
 .../curator/framework/imps/TestFramework.java   | 43 +++
 .../recipes/atomic/DistributedAtomicValue.java  |  9 +--
 .../recipes/barriers/DistributedBarrier.java|  2 +-
 .../barriers/DistributedDoubleBarrier.java  |  2 +-
 .../framework/recipes/cache/NodeCache.java  |  8 +-
 .../recipes/cache/PathChildrenCache.java|  9 +--
 .../framework/recipes/leader/LeaderLatch.java   |  2 +-
 .../framework/recipes/locks/ChildReaper.java|  4 +
 .../recipes/locks/InterProcessSemaphoreV2.java  |  2 +-
 .../curator/framework/recipes/locks/Reaper.java |  4 +
 .../locks/StandardLockInternalsDriver.java  |  4 +-
 .../recipes/nodes/PersistentEphemeralNode.java  |  2 +-
 .../recipes/queue/DistributedQueue.java |  4 +-
 .../recipes/queue/SimpleDistributedQueue.java   |  8 +-
 .../framework/recipes/shared/SharedValue.java   |  2 +-
 .../src/site/confluence/index.confluence|  2 +
 .../locks/TestInterProcessMultiMutex.java   |  4 +-
 .../recipes/locks/TestInterProcessMutex.java|  2 +-
 .../locks/TestInterProcessMutexBase.java| 77 +++-
 .../locks/TestInterProcessSemaphoreMutex.java   |  2 +-
 .../discovery/details/ServiceDiscoveryImpl.java |  4 +-
 .../idl/services/CuratorProjectionService.java  |  4 +
 .../curator/x/rpc/idl/structs/CreateSpec.java   |  6 +-
 .../x/rpc/idl/structs/RpcCreateMode.java|  3 +-
 curator-x-rpc/src/main/thrift/curator.thrift|  3 +-
 .../src/site/confluence/reference.confluence|  1 +
 pom.xml |  2 +-
 src/site/confluence/utilities.confluence| 14 
 39 files changed, 324 insertions(+), 64 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/04ae8115/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java 
b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
index f072775..3181aca 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
@@ -64,7 +64,7 @@ public class EnsurePath
 }
 };
 
-private interface Helper
+interface Helper
 {
 public void ensure(CuratorZookeeperClient client, String path, final 
boolean makeLastNode) throws Exception;
 }
@@ -110,7 +110,18 @@ public class EnsurePath
 return new EnsurePath(path, helper, false, aclProvider);
 }
 
-private EnsurePath(String path, AtomicReference helper, boolean 
makeLastNode, InternalACLProvider aclProvider)
+/**
+ * Returns a view of this EnsurePath instance that does not make the last 
node and also makes containers.
+ * i.e. if the path is "/a/b/c" only "/a/b" will be ensured
+ *
+ * @return view
+ */
+public EnsurePathContainers excludingLastContainers()
+{
+return new EnsurePathContainers(path, helper, false, aclProvider);
+}
+
+protected EnsurePath(String path, AtomicReference helper, boolean 
makeLastNode, InternalACLProvider aclProvider)
 {
 this.path = path;
 this.makeLastNode = makeLastNode;
@@ -128,6 +139,11 @@ public class EnsurePath
 return this.path;
 }
 
+protected boolean asContainers()
+{
+return false;
+}
+
 private class InitialHelper implements Helper
 {
 private boolean isSet = false;  // guarded by synchron

[35/50] curator git commit: Add Mike Drob as developer

2015-08-24 Thread cammckenzie
Add Mike Drob as developer


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

Branch: refs/heads/CURATOR-167
Commit: c3426ba96810dafdbf776ef7072116bef3ca4a25
Parents: 870b4d5
Author: Mike Drob 
Authored: Thu Jul 23 15:56:26 2015 -0500
Committer: Mike Drob 
Committed: Thu Jul 23 16:03:37 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/curator/blob/c3426ba9/pom.xml
--
diff --git a/pom.xml b/pom.xml
index efcf6f7..cb747b7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -227,6 +227,18 @@
 
 
 
+mdrob
+Mike Drob
+md...@apache.org
+-6
+
+Committer
+PMC Member
+
+http://people.apache.org/~mdrob
+
+
+
 Patrick Hunt
 phu...@gmail.com
 



[14/50] curator git commit: Merge branch 'fix-deps' into CURATOR-222

2015-08-24 Thread cammckenzie
Merge branch 'fix-deps' into CURATOR-222


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

Branch: refs/heads/CURATOR-167
Commit: 25dcef9ef669dae9e4c7daf072183767e1decea9
Parents: b3cf69d db06634
Author: randgalt 
Authored: Sat Jun 6 23:37:27 2015 -0500
Committer: randgalt 
Committed: Sat Jun 6 23:37:27 2015 -0500

--
 curator-client/pom.xml | 16 
 curator-framework/pom.xml  |  6 ++
 curator-recipes/pom.xml|  6 ++
 curator-test/pom.xml   | 10 ++
 curator-x-discovery-server/pom.xml |  6 ++
 curator-x-discovery/pom.xml|  6 ++
 curator-x-rpc/pom.xml  |  6 ++
 pom.xml| 25 -
 8 files changed, 56 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/25dcef9e/pom.xml
--



[12/50] curator git commit: closes #81 - created by mistake*

2015-08-24 Thread cammckenzie
closes #81 - created by mistake*


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

Branch: refs/heads/CURATOR-167
Commit: 20e92a54c196310f84b8d385bce71b7812b18f33
Parents: 96fdf85
Author: randgalt 
Authored: Tue May 19 16:21:11 2015 -0700
Committer: randgalt 
Committed: Tue May 19 16:21:11 2015 -0700

--

--




[01/50] curator git commit: [maven-release-plugin] prepare for next development iteration

2015-08-24 Thread cammckenzie
Repository: curator
Updated Branches:
  refs/heads/CURATOR-167 44c3891e3 -> 8fae7856e (forced update)


[maven-release-plugin] prepare for next development iteration


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

Branch: refs/heads/CURATOR-167
Commit: 121efdb5008f6f5d8604ab0e44588f9576766a2e
Parents: c1466e6
Author: randgalt 
Authored: Fri May 8 13:57:12 2015 -0500
Committer: randgalt 
Committed: Fri May 8 13:57:12 2015 -0500

--
 curator-client/pom.xml | 4 ++--
 curator-examples/pom.xml   | 2 +-
 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 ++--
 curator-x-rpc/pom.xml  | 4 ++--
 pom.xml| 2 +-
 9 files changed, 16 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-client/pom.xml
--
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 4eafd2e..50b950f 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -24,11 +24,11 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-client
-2.8.0
+2.8.1-SNAPSHOT
 bundle
 
 Curator Client

http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-examples/pom.xml
--
diff --git a/curator-examples/pom.xml b/curator-examples/pom.xml
index cbdd849..85eaaf7 100644
--- a/curator-examples/pom.xml
+++ b/curator-examples/pom.xml
@@ -24,7 +24,7 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-examples

http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-framework/pom.xml
--
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index a3d2e87..3b3e346 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -24,11 +24,11 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-framework
-2.8.0
+2.8.1-SNAPSHOT
 bundle
 
 Curator Framework

http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-recipes/pom.xml
--
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index d8c4211..8913b6f 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -24,11 +24,11 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-recipes
-2.8.0
+2.8.1-SNAPSHOT
 bundle
 
 Curator Recipes

http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-test/pom.xml
--
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index fca1694..d3301ca 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -24,11 +24,11 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-test
-2.8.0
+2.8.1-SNAPSHOT
 
 Curator Testing
 Unit testing utilities.

http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-x-discovery-server/pom.xml
--
diff --git a/curator-x-discovery-server/pom.xml 
b/curator-x-discovery-server/pom.xml
index 453def0..451f46c 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -24,11 +24,11 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-x-discovery-server
-2.8.0
+2.8.1-SNAPSHOT
 bundle
 
 Curator Service Discovery Server

http://git-wip-us.apache.org/repos/asf/curator/blob/121efdb5/curator-x-discovery/pom.xml
--
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 5c0248a..8c08a10 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -24,11 +24,11 @@
 
 org.apache.curator
 apache-curator
-2.8.0
+2.8.1-SNAPSHOT
 
 
 curator-x-discovery
-2.8.0
+2.8.1-SNAPSHOT
 bundle
 
 Curator Service Discovery

http://git-wip-us.apache.org/repo

[07/50] curator git commit: refactored

2015-08-24 Thread cammckenzie
refactored


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

Branch: refs/heads/CURATOR-167
Commit: 85697633a94508637ee22f1df64a06ddb72d0660
Parents: 31130fd
Author: randgalt 
Authored: Tue May 19 14:59:41 2015 -0700
Committer: randgalt 
Committed: Tue May 19 14:59:41 2015 -0700

--
 .../main/java/org/apache/curator/utils/ZKPaths.java   | 14 +++---
 1 file changed, 11 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/85697633/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
--
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java 
b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
index f91f03d..242e4b7 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
@@ -38,10 +38,18 @@ public class ZKPaths
  */
 public static final String PATH_SEPARATOR = "/";
 
-public static class CreatModeHolder
+/**
+ * @return {@link CreateMode#CONTAINER} if the ZK JAR supports it. 
Otherwise {@link CreateMode#PERSISTENT}
+ */
+public static CreateMode getContainerCreateMode()
+{
+return CreateModeHolder.containerCreateMode;
+}
+
+private static class CreateModeHolder
 {
 private static final Logger log = 
LoggerFactory.getLogger(ZKPaths.class);
-public static final CreateMode containerCreateMode;
+private static final CreateMode containerCreateMode;
 
 static
 {
@@ -442,6 +450,6 @@ public class ZKPaths
 
 private static CreateMode getCreateMode(boolean asContainers)
 {
-return asContainers ? CreatModeHolder.containerCreateMode : 
CreateMode.PERSISTENT;
+return asContainers ? getContainerCreateMode() : CreateMode.PERSISTENT;
 }
 }



[11/50] curator git commit: closes #80 - created by mistake*

2015-08-24 Thread cammckenzie
closes #80 - created by mistake*


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

Branch: refs/heads/CURATOR-167
Commit: 96fdf851b6a585aa1ec04ca28892535e9099fa2f
Parents: 7f20986
Author: randgalt 
Authored: Tue May 19 16:20:26 2015 -0700
Committer: randgalt 
Committed: Tue May 19 16:20:26 2015 -0700

--

--




[32/50] curator git commit: CURATOR-235 LeaderSelector.internalRequeue should be private

2015-08-24 Thread cammckenzie
CURATOR-235 LeaderSelector.internalRequeue should be private


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

Branch: refs/heads/CURATOR-167
Commit: 0bec8a066f8a4b5d26ef3621800543a54b46ac3e
Parents: e71da8f
Author: Tsuyoshi Ozawa 
Authored: Wed Jul 15 13:18:40 2015 +0900
Committer: Tsuyoshi Ozawa 
Committed: Wed Jul 15 13:18:40 2015 +0900

--
 .../apache/curator/framework/recipes/leader/LeaderSelector.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/0bec8a06/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 9c09b4c..716ca96 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
@@ -230,7 +230,7 @@ public class LeaderSelector implements Closeable
 return internalRequeue();
 }
 
-public synchronized boolean internalRequeue()
+private synchronized boolean internalRequeue()
 {
 if ( !isQueued && (state.get() == State.STARTED) )
 {



[46/50] curator git commit: CURATOR-224 broke the fix for CURATOR-56. Introduced a new method, makeRequeueItemPath(), so that items can be requeued in a way that's compatible with all the queue types

2015-08-24 Thread cammckenzie
CURATOR-224 broke the fix for CURATOR-56. Introduced a new method, 
makeRequeueItemPath(), so that items can be requeued in a way that's compatible 
with all the queue types


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

Branch: refs/heads/CURATOR-167
Commit: 2266ca1fb1414bc8306fb1d6c4ac632a841f36ec
Parents: 25f5149
Author: randgalt 
Authored: Fri Aug 21 13:30:48 2015 -0500
Committer: randgalt 
Committed: Fri Aug 21 13:30:48 2015 -0500

--
 .../framework/recipes/queue/DistributedIdQueue.java| 13 -
 .../framework/recipes/queue/DistributedQueue.java  |  7 ++-
 2 files changed, 18 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/2266ca1f/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
index dbd8e6e..15045aa 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedIdQueue.java
@@ -78,6 +78,12 @@ public class DistributedIdQueue implements QueueBase
 {
 internalSortChildren(children);
 }
+
+@Override
+protected String makeRequeueItemPath(String itemPath)
+{
+return makeIdPath(parseId(itemPath).id);
+}
 };
 
 if ( queue.makeItemPath().contains(Character.toString(SEPARATOR)) )
@@ -153,7 +159,7 @@ public class DistributedIdQueue implements QueueBase
 
 queue.checkState();
 
-return queue.internalPut(item, null, queue.makeItemPath() + SEPARATOR 
+ fixId(itemId) + SEPARATOR, maxWait, unit);
+return queue.internalPut(item, null, makeIdPath(itemId), maxWait, 
unit);
 }
 
 /**
@@ -198,6 +204,11 @@ public class DistributedIdQueue implements QueueBase
 return false;
 }
 
+private String makeIdPath(String itemId)
+{
+return queue.makeItemPath() + SEPARATOR + fixId(itemId) + SEPARATOR;
+}
+
 private void internalSortChildren(List children)
 {
 Collections.sort

http://git-wip-us.apache.org/repos/asf/curator/blob/2266ca1f/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
index 3ed3218..3b63956 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java
@@ -756,7 +756,7 @@ public class DistributedQueue implements QueueBase
 client.inTransaction()
 .delete().forPath(itemPath)
 .and()
-
.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath(itemPath, bytes)
+
.create().withMode(CreateMode.PERSISTENT_SEQUENTIAL).forPath(makeRequeueItemPath(itemPath),
 bytes)
 .and()
 .commit();
 }
@@ -789,4 +789,9 @@ public class DistributedQueue implements QueueBase
 
 return false;
 }
+
+protected String makeRequeueItemPath(String itemPath)
+{
+return makeItemPath();
+}
 }



[09/50] curator git commit: Refactoring and more tests

2015-08-24 Thread cammckenzie
Refactoring and more tests


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

Branch: refs/heads/CURATOR-167
Commit: 12baea342d2b35c48bfb17327b0736a179d04d32
Parents: 35ec01c
Author: randgalt 
Authored: Tue May 19 15:26:06 2015 -0700
Committer: randgalt 
Committed: Tue May 19 15:26:06 2015 -0700

--
 .../java/org/apache/curator/utils/ZKPaths.java  | 13 ++-
 .../curator/framework/imps/TestFramework.java   | 89 +++-
 2 files changed, 93 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/12baea34/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
--
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java 
b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
index 242e4b7..13b0cba 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
@@ -53,17 +53,14 @@ public class ZKPaths
 
 static
 {
-CreateMode localCreateMode = CreateMode.PERSISTENT;
-for ( CreateMode createMode : CreateMode.class.getEnumConstants() )
+CreateMode localCreateMode;
+try
 {
-if ( createMode.name().equals("CONTAINER") )
-{
-localCreateMode = createMode;
-break;
-}
+localCreateMode = CreateMode.valueOf("CONTAINER");
 }
-if ( localCreateMode == CreateMode.PERSISTENT )
+catch ( IllegalArgumentException ignore )
 {
+localCreateMode = CreateMode.PERSISTENT;
 log.warn("The version of ZooKeeper being used doesn't support 
Container nodes. CreateMode.PERSISTENT will be used instead");
 }
 containerCreateMode = localCreateMode;

http://git-wip-us.apache.org/repos/asf/curator/blob/12baea34/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 ac21172..15db5de 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
@@ -419,13 +419,100 @@ public class TestFramework extends BaseClassForTests
 }
 
 @Test
+public void testOverrideCreateParentContainers() throws Exception
+{
+CuratorFramework client = CuratorFrameworkFactory.builder()
+.connectString(server.getConnectString())
+.retryPolicy(new RetryOneTime(1))
+.dontUseContainerParents()
+.build();
+try
+{
+client.start();
+
client.create().creatingParentContainersIfNeeded().forPath("/one/two/three", 
"foo".getBytes());
+byte[] data = client.getData().forPath("/one/two/three");
+Assert.assertEquals(data, "foo".getBytes());
+
+client.delete().forPath("/one/two/three");
+new Timing().sleepABit();
+
+Assert.assertNotNull(client.checkExists().forPath("/one/two"));
+new Timing().sleepABit();
+Assert.assertNotNull(client.checkExists().forPath("/one"));
+}
+finally
+{
+client.close();
+}
+}
+
+@Test
+public void testOverrideEnsureParentContainers() throws Exception
+{
+CuratorFramework client = CuratorFrameworkFactory.builder()
+.connectString(server.getConnectString())
+.retryPolicy(new RetryOneTime(1))
+.dontUseContainerParents()
+.build();
+try
+{
+client.start();
+EnsurePath ensurePath = 
client.newNamespaceAwareEnsurePathContainers("/one/two");
+ensurePath.ensure(client.getZookeeperClient());
+
client.create().creatingParentContainersIfNeeded().forPath("/one/two/three", 
"foo".getBytes());
+byte[] data = client.getData().forPath("/one/two/three");
+Assert.assertEquals(data, "foo".getBytes());
+
+client.delete().forPath("/one/two/three");
+new Timing().sleepABit();
+
+Assert.assertNotNull(client.checkExists().forPath("/one/two"));
+new Timing().sleepABi

[17/50] curator git commit: minor reformat

2015-08-24 Thread cammckenzie
minor reformat


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

Branch: refs/heads/CURATOR-167
Commit: 79c214dd85eb334f1f4a50b9362d5db343d291f4
Parents: 6ca
Author: randgalt 
Authored: Sat Jun 20 08:07:48 2015 -0500
Committer: randgalt 
Committed: Sat Jun 20 08:07:48 2015 -0500

--
 .../curator/x/discovery/TestServiceCache.java   | 72 ++--
 1 file changed, 37 insertions(+), 35 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/79c214dd/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/TestServiceCache.java
--
diff --git 
a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/TestServiceCache.java
 
b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/TestServiceCache.java
index 5850961..fda5c26 100644
--- 
a/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/TestServiceCache.java
+++ 
b/curator-x-discovery/src/test/java/org/apache/curator/x/discovery/TestServiceCache.java
@@ -16,18 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.curator.x.discovery;
 
 import com.google.common.collect.Lists;
-import org.apache.curator.test.BaseClassForTests;
-import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
-import org.apache.curator.test.Timing;
-import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.state.ConnectionState;
 import org.apache.curator.retry.RetryOneTime;
-import org.apache.curator.test.TestingServer;
+import org.apache.curator.test.BaseClassForTests;
+import org.apache.curator.test.ExecuteCalledWatchingExecutorService;
+import org.apache.curator.test.Timing;
+import org.apache.curator.utils.CloseableUtils;
 import org.apache.curator.x.discovery.details.ServiceCacheListener;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -43,7 +43,7 @@ import java.util.concurrent.TimeUnit;
 public class TestServiceCache extends BaseClassForTests
 {
 @Test
-public void testInitialLoad() throws Exception
+public void testInitialLoad() throws Exception
 {
 List closeables = Lists.newArrayList();
 try
@@ -52,15 +52,15 @@ public class TestServiceCache extends BaseClassForTests
 closeables.add(client);
 client.start();
 
-ServiceDiscoverydiscovery = 
ServiceDiscoveryBuilder.builder(String.class).basePath("/discovery").client(client).build();
+ServiceDiscovery discovery = 
ServiceDiscoveryBuilder.builder(String.class).basePath("/discovery").client(client).build();
 closeables.add(discovery);
 discovery.start();
 
-ServiceCachecache = 
discovery.serviceCacheBuilder().name("test").build();
+ServiceCache cache = 
discovery.serviceCacheBuilder().name("test").build();
 closeables.add(cache);
 
-final CountDownLatchlatch = new CountDownLatch(3);
-ServiceCacheListenerlistener = new ServiceCacheListener()
+final CountDownLatch latch = new CountDownLatch(3);
+ServiceCacheListener listener = new ServiceCacheListener()
 {
 @Override
 public void cacheChanged()
@@ -76,16 +76,16 @@ public class TestServiceCache extends BaseClassForTests
 cache.addListener(listener);
 cache.start();
 
-ServiceInstance instance1 = 
ServiceInstance.builder().payload("test").name("test").port(10064).build();
-ServiceInstance instance2 = 
ServiceInstance.builder().payload("test").name("test").port(10065).build();
-ServiceInstance instance3 = 
ServiceInstance.builder().payload("test").name("test").port(10066).build();
+ServiceInstance instance1 = 
ServiceInstance.builder().payload("test").name("test").port(10064).build();
+ServiceInstance instance2 = 
ServiceInstance.builder().payload("test").name("test").port(10065).build();
+ServiceInstance instance3 = 
ServiceInstance.builder().payload("test").name("test").port(10066).build();
 discovery.registerService(instance1);
 discovery.registerService(instance2);
 discovery.registerService(instance3);
 
 Assert.assertTrue(latch.await(10, TimeUnit.SECONDS));
 
-ServiceCachecache2 = 
discovery.serviceCacheBuilder().name("test").buil

[20/50] curator git commit: updated deprecated tags

2015-08-24 Thread cammckenzie
updated deprecated tags


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

Branch: refs/heads/CURATOR-167
Commit: d678de0bf8932710d47680ac9bdafc682f2e3019
Parents: f27b490
Author: randgalt 
Authored: Sun Jun 21 20:48:42 2015 -0500
Committer: randgalt 
Committed: Sun Jun 21 20:48:42 2015 -0500

--
 .../src/main/java/org/apache/curator/utils/EnsurePath.java   | 3 ++-
 .../java/org/apache/curator/framework/CuratorFramework.java  | 8 ++--
 .../java/org/apache/curator/framework/api/CreateBuilder.java | 1 +
 .../curator/framework/recipes/cache/PathChildrenCache.java   | 3 +++
 .../framework/recipes/cache/PathChildrenCacheMode.java   | 1 +
 .../apache/curator/framework/recipes/locks/ChildReaper.java  | 3 ++-
 .../framework/recipes/locks/InterProcessSemaphore.java   | 1 +
 .../org/apache/curator/framework/recipes/locks/Reaper.java   | 3 ++-
 8 files changed, 18 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/d678de0b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
--
diff --git 
a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java 
b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
index a4a8528..3845a74 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/EnsurePath.java
@@ -48,8 +48,9 @@ import java.util.concurrent.atomic.AtomicReference;
  * zk.create(nodePath, ...);
  * 
  *
- * @deprecated Prefer 
CuratorFramework.create().creatingParentContainersIfNeeded() or 
CuratorFramework.exists().creatingParentContainersIfNeeded()
+ * @deprecated Since 2.9.0 - Prefer 
CuratorFramework.create().creatingParentContainersIfNeeded() or 
CuratorFramework.exists().creatingParentContainersIfNeeded()
  */
+@Deprecated
 public class EnsurePath
 {
 private final String path;

http://git-wip-us.apache.org/repos/asf/curator/blob/d678de0b/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 fdf583c..de9bcc5 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
@@ -59,6 +59,7 @@ public interface CuratorFramework extends Closeable
  * @return true/false
  * @deprecated use {@link #getState()} instead
  */
+@Deprecated
 public boolean isStarted();
 
 /**
@@ -134,6 +135,7 @@ public interface CuratorFramework extends Closeable
  * @param backgroundContextObject optional context
  * @deprecated use {@link #sync()} instead
  */
+@Deprecated
 public void sync(String path, Object backgroundContextObject);
 
 /**
@@ -170,8 +172,9 @@ public interface CuratorFramework extends Closeable
  * pre-pend the namespace to all paths
  *
  * @return facade
- * @deprecated use {@link #usingNamespace} passing null
+ * @deprecated Since 2.9.0 - use {@link #usingNamespace} passing 
null
  */
+@Deprecated
 public CuratorFramework nonNamespaceView();
 
 /**
@@ -202,8 +205,9 @@ public interface CuratorFramework extends Closeable
  *
  * @param path path to ensure
  * @return new EnsurePath instance
- * @deprecated prefer {@link 
CreateBuilder#creatingParentContainersIfNeeded()} or {@link 
ExistsBuilder#creatingParentContainersIfNeeded()}
+ * @deprecated Since 2.9.0 - prefer {@link 
CreateBuilder#creatingParentContainersIfNeeded()} or {@link 
ExistsBuilder#creatingParentContainersIfNeeded()}
  */
+@Deprecated
 public EnsurePath newNamespaceAwareEnsurePath(String path);
 
 /**

http://git-wip-us.apache.org/repos/asf/curator/blob/d678de0b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
--
diff --git 
a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
 
b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
index 5e1bc56..0db2094 100644
--- 
a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
+++ 
b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java
@@ -50,6 +50,7 @@ public

[25/50] curator git commit: left some debug code in

2015-08-24 Thread cammckenzie
left some debug code in


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

Branch: refs/heads/CURATOR-167
Commit: 8226f0cc30ed8cc9cdb455a3fd707d3c801e25b0
Parents: b4ff095
Author: randgalt 
Authored: Wed Jun 24 16:50:26 2015 -0500
Committer: randgalt 
Committed: Wed Jun 24 16:50:26 2015 -0500

--
 .../test/java/org/apache/curator/framework/imps/TestFramework.java  | 1 -
 1 file changed, 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/8226f0cc/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 bb8747e..811631c 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
@@ -527,7 +527,6 @@ public class TestFramework extends BaseClassForTests
 client.start();
 
 Assert.assertNull(client.checkExists().forPath("/one/two"));
-
client.create().creatingParentContainersIfNeeded().forPath("/one/two/three");
 
client.checkExists().creatingParentContainersIfNeeded().forPath("/one/two/three");
 Assert.assertNotNull(client.checkExists().forPath("/one/two"));
 Assert.assertNull(client.checkExists().forPath("/one/two/three"));



[40/50] curator git commit: CURATOR-129 Add docs to PCC constructors

2015-08-24 Thread cammckenzie
CURATOR-129 Add docs to PCC constructors

Add javadocs cautioning users that ExecutorServices provided should be 
single-threaded

This closes #86


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

Branch: refs/heads/CURATOR-167
Commit: 40b458ee11391e74a2d7948c380b869e9b000f20
Parents: 3a4d541
Author: Mike Drob 
Authored: Wed Jun 24 14:13:41 2015 -0500
Committer: Mike Drob 
Committed: Fri Aug 7 14:03:03 2015 -0500

--
 .../curator/framework/recipes/cache/PathChildrenCache.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/40b458ee/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
--
diff --git 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
index 5a7b424..b5d912c 100644
--- 
a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
+++ 
b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java
@@ -199,7 +199,7 @@ public class PathChildrenCache implements Closeable
  * @param path path to watch
  * @param cacheDataif true, node contents are cached in addition 
to the stat
  * @param dataIsCompressed if true, data in the path is compressed
- * @param executorService  ExecutorService to use for the 
PathChildrenCache's background thread
+ * @param executorService  ExecutorService to use for the 
PathChildrenCache's background thread. This service should be single threaded, 
otherwise the cache may see inconsistent results.
  */
 public PathChildrenCache(CuratorFramework client, String path, boolean 
cacheData, boolean dataIsCompressed, final ExecutorService executorService)
 {
@@ -211,7 +211,7 @@ public class PathChildrenCache implements Closeable
  * @param path path to watch
  * @param cacheDataif true, node contents are cached in addition 
to the stat
  * @param dataIsCompressed if true, data in the path is compressed
- * @param executorService  Closeable ExecutorService to use for the 
PathChildrenCache's background thread
+ * @param executorService  Closeable ExecutorService to use for the 
PathChildrenCache's background thread. This service should be single threaded, 
otherwise the cache may see inconsistent results.
  */
 public PathChildrenCache(CuratorFramework client, String path, boolean 
cacheData, boolean dataIsCompressed, final CloseableExecutorService 
executorService)
 {



[19/50] curator git commit: Doc misnamed CuratorEvent

2015-08-24 Thread cammckenzie
Doc misnamed CuratorEvent


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

Branch: refs/heads/CURATOR-167
Commit: f27b49095bbe359afd904e1f3a227a00d2239578
Parents: 8160001
Author: randgalt 
Authored: Sun Jun 21 11:04:35 2015 -0500
Committer: randgalt 
Committed: Sun Jun 21 11:04:35 2015 -0500

--
 curator-framework/src/site/confluence/index.confluence | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/f27b4909/curator-framework/src/site/confluence/index.confluence
--
diff --git a/curator-framework/src/site/confluence/index.confluence 
b/curator-framework/src/site/confluence/index.confluence
index 84f794f..efde81b 100644
--- a/curator-framework/src/site/confluence/index.confluence
+++ b/curator-framework/src/site/confluence/index.confluence
@@ -51,9 +51,9 @@ CuratorFramework instance using the addListener() method. The 
listener implement
 |eventReceived()|A background operation has completed or a watch has 
triggered. Examine the given event for details|
 clientClosedDueToError()|An unrecoverable error has occurred. The 
CuratorFramework instance has been shut down|
 
-h3. ClientEvent
-The ClientEvent object is a super\-set POJO that can hold every type of 
background notification and triggered watch. The useful fields of
-ClientEvent depend on the type of event which is exposed via the getType() 
method.
+h3. CuratorEvent
+The CuratorEvent object is a super\-set POJO that can hold every type of 
background notification and triggered watch. The useful fields of
+CuratorEvent depend on the type of event which is exposed via the getType() 
method.
 
 ||Event Type||Event Methods||
 |CREATE|getResultCode() and getPath()|



[31/50] curator git commit: Merge branch 'CURATOR-222'

2015-08-24 Thread cammckenzie
Merge branch 'CURATOR-222'


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

Branch: refs/heads/CURATOR-167
Commit: e71da8f0cf759f67609888fc5005e54ffcde308b
Parents: d90325c 584e529
Author: randgalt 
Authored: Wed Jul 1 18:35:46 2015 -0500
Committer: randgalt 
Committed: Wed Jul 1 18:35:46 2015 -0500

--
 .../curator/framework/imps/NamespaceImpl.java   | 20 +++-
 1 file changed, 19 insertions(+), 1 deletion(-)
--




[22/50] curator git commit: Merge branch 'master' of https://github.com/zzcoder/curator into CURATOR-224

2015-08-24 Thread cammckenzie
Merge branch 'master' of https://github.com/zzcoder/curator into CURATOR-224


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

Branch: refs/heads/CURATOR-167
Commit: 19bb4d1c4288d09111d23ed4c47b761bb19bd79b
Parents: 79c214d 83e1a85
Author: Cameron McKenzie 
Authored: Tue Jun 23 09:58:00 2015 +1000
Committer: Cameron McKenzie 
Committed: Tue Jun 23 09:58:00 2015 +1000

--
 .../recipes/queue/DistributedQueue.java |  2 +-
 .../recipes/queue/TestDistributedIdQueue.java   | 47 
 2 files changed, 48 insertions(+), 1 deletion(-)
--




[39/50] curator git commit: Access data the same way

2015-08-24 Thread cammckenzie
Access data the same way


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

Branch: refs/heads/CURATOR-167
Commit: f3ff7e75f305a6d52a8fc9f7e9defae4299a66a2
Parents: 5cfa483
Author: Alex Brasetvik 
Authored: Mon Aug 3 02:00:44 2015 +0200
Committer: Alex Brasetvik 
Committed: Mon Aug 3 02:00:44 2015 +0200

--
 .../curator/framework/recipes/nodes/PersistentEphemeralNode.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/f3ff7e75/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 7a2ab73..0d963e0 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
@@ -338,7 +338,7 @@ public class PersistentEphemeralNode implements Closeable
 this.data.set(Arrays.copyOf(data, data.length));
 if ( isActive() )
 {
-client.setData().inBackground().forPath(getActualPath(), 
this.data.get());
+client.setData().inBackground().forPath(getActualPath(), 
getData());
 }
 }
 



[06/50] curator git commit: make public for future use

2015-08-24 Thread cammckenzie
make public for future use


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

Branch: refs/heads/CURATOR-167
Commit: 31130fd58c534239cf286eb6648dbdd64e22d457
Parents: d492f8c
Author: randgalt 
Authored: Tue May 19 14:57:23 2015 -0700
Committer: randgalt 
Committed: Tue May 19 14:57:23 2015 -0700

--
 .../src/main/java/org/apache/curator/utils/ZKPaths.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/31130fd5/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
--
diff --git a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java 
b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
index 526f705..f91f03d 100644
--- a/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
+++ b/curator-client/src/main/java/org/apache/curator/utils/ZKPaths.java
@@ -38,10 +38,10 @@ public class ZKPaths
  */
 public static final String PATH_SEPARATOR = "/";
 
-private static class CreatModeHolder
+public static class CreatModeHolder
 {
 private static final Logger log = 
LoggerFactory.getLogger(ZKPaths.class);
-private static final CreateMode containerCreateMode;
+public static final CreateMode containerCreateMode;
 
 static
 {



[38/50] curator git commit: Make getData private and access data directly in setData

2015-08-24 Thread cammckenzie
Make getData private and access data directly in setData


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

Branch: refs/heads/CURATOR-167
Commit: 5cfa4839e0798249d32b34b792b5f59f111dd022
Parents: 7fd4034
Author: Alex Brasetvik 
Authored: Mon Aug 3 01:52:51 2015 +0200
Committer: Alex Brasetvik 
Committed: Mon Aug 3 01:52:51 2015 +0200

--
 .../curator/framework/recipes/nodes/PersistentEphemeralNode.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/5cfa4839/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 1011ad5..7a2ab73 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
@@ -338,11 +338,11 @@ public class PersistentEphemeralNode implements Closeable
 this.data.set(Arrays.copyOf(data, data.length));
 if ( isActive() )
 {
-client.setData().inBackground().forPath(getActualPath(), 
getData());
+client.setData().inBackground().forPath(getActualPath(), 
this.data.get());
 }
 }
 
-byte[] getData() {
+private byte[] getData() {
 return this.data.get();
 }
 



[13/50] curator git commit: moved the hard coded deps out of the parent pom. It created problems for curator-rpc

2015-08-24 Thread cammckenzie
moved the hard coded deps out of the parent pom. It created problems for 
curator-rpc


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

Branch: refs/heads/CURATOR-167
Commit: db066346df39dd85a8fec925ba816617695f68df
Parents: 20e92a5
Author: randgalt 
Authored: Sat Jun 6 23:35:36 2015 -0500
Committer: randgalt 
Committed: Sat Jun 6 23:35:36 2015 -0500

--
 curator-client/pom.xml | 16 
 curator-framework/pom.xml  |  6 ++
 curator-recipes/pom.xml|  6 ++
 curator-test/pom.xml   | 10 ++
 curator-x-discovery-server/pom.xml |  6 ++
 curator-x-discovery/pom.xml|  6 ++
 curator-x-rpc/pom.xml  |  6 ++
 pom.xml| 25 -
 8 files changed, 56 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-client/pom.xml
--
diff --git a/curator-client/pom.xml b/curator-client/pom.xml
index 50b950f..22ccf28 100644
--- a/curator-client/pom.xml
+++ b/curator-client/pom.xml
@@ -46,6 +46,16 @@
 
 
 
+org.apache.zookeeper
+zookeeper
+
+
+
+com.google.guava
+guava
+
+
+
 org.slf4j
 slf4j-api
 
@@ -61,5 +71,11 @@
 curator-test
 test
 
+
+
+org.testng
+testng
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-framework/pom.xml
--
diff --git a/curator-framework/pom.xml b/curator-framework/pom.xml
index 3b3e346..7d9bd03 100644
--- a/curator-framework/pom.xml
+++ b/curator-framework/pom.xml
@@ -55,6 +55,12 @@
 curator-test
 test
 
+
+
+org.testng
+testng
+test
+
 
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-recipes/pom.xml
--
diff --git a/curator-recipes/pom.xml b/curator-recipes/pom.xml
index 8913b6f..8c257b7 100644
--- a/curator-recipes/pom.xml
+++ b/curator-recipes/pom.xml
@@ -61,5 +61,11 @@
 mockito-core
 test
 
+
+
+org.testng
+testng
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-test/pom.xml
--
diff --git a/curator-test/pom.xml b/curator-test/pom.xml
index d3301ca..59abda2 100644
--- a/curator-test/pom.xml
+++ b/curator-test/pom.xml
@@ -36,6 +36,11 @@
 
 
 
+org.apache.zookeeper
+zookeeper
+
+
+
 org.javassist
 javassist
 
@@ -46,6 +51,11 @@
 
 
 
+com.google.guava
+guava
+
+
+
 org.testng
 testng
 provided

http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-x-discovery-server/pom.xml
--
diff --git a/curator-x-discovery-server/pom.xml 
b/curator-x-discovery-server/pom.xml
index 451f46c..2728ce5 100644
--- a/curator-x-discovery-server/pom.xml
+++ b/curator-x-discovery-server/pom.xml
@@ -62,6 +62,12 @@
 
 
 
+org.testng
+testng
+test
+
+
+
 com.sun.jersey
 jersey-server
 test

http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-x-discovery/pom.xml
--
diff --git a/curator-x-discovery/pom.xml b/curator-x-discovery/pom.xml
index 8c08a10..015fea7 100644
--- a/curator-x-discovery/pom.xml
+++ b/curator-x-discovery/pom.xml
@@ -60,5 +60,11 @@
 curator-test
 test
 
+
+
+org.testng
+testng
+test
+
 
 

http://git-wip-us.apache.org/repos/asf/curator/blob/db066346/curator-x-rpc/pom.xml
--
diff --git a/curator-x-rpc/pom.xml b/curator-x-rpc/pom.xml
index 222310b..299ef9f 100644
--- a/curator-x-rpc/pom.xml
+++ b/curator-x-rpc/pom.xml
@@ -112,6 +112,12 @@
 
 test
 
+
+
+org.testng
+testng
+