[1/3] curator git commit: Added Locker which uses Java 7's try-with-resource feature to make locking more reliable
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 9fc755a9e -> f8814f619 Added Locker which uses Java 7's try-with-resource feature to make locking more reliable Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/58a8818b Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/58a8818b Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/58a8818b Branch: refs/heads/CURATOR-3.0 Commit: 58a8818b749475cf2089d116c06359ad0390a2cc Parents: d57aaeb Author: randgalt <randg...@apache.org> Authored: Sun Sep 6 12:26:56 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 6 12:26:56 2015 -0700 -- .../curator/framework/recipes/locks/Locker.java | 67 .../locks/TestInterProcessMutexBase.java| 22 +++ 2 files changed, 89 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/58a8818b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java -- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java new file mode 100644 index 000..97788af --- /dev/null +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java @@ -0,0 +1,67 @@ +package org.apache.curator.framework.recipes.locks; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * + * Utility for safely acquiring a lock and releasing it using Java 7's + * try-with-resource feature. + * + * + * + * Canonical usage: + * + * InterProcessMutex mutex = new InterProcessMutex(...) // or any InterProcessLock + * try ( Locker locker = new Locker(mutex, maxTimeout, unit) ) + * { + * // do work + * } + * + * + */ +public class Locker implements AutoCloseable +{ +private final InterProcessLock lock; +private final AtomicBoolean acquired; + +/** + * @param lock a lock implementation (e.g. {@link InterProcessMutex}, {@link InterProcessSemaphoreV2}, etc.) + * @param timeout max timeout to acquire lock + * @param unit time unit of timeout + * @throws Exception Curator errors or {@link TimeoutException} if the lock cannot be acquired within the timeout + */ +public Locker(InterProcessLock lock, long timeout, TimeUnit unit) throws Exception +{ +this.lock = lock; +acquired = new AtomicBoolean(acquireLock(lock, timeout, unit)); +if ( !acquired.get() ) +{ +throw new TimeoutException("Could not acquire lock within timeout of " + unit.toMillis(timeout) + "ms"); +} +} + +@Override +/** + * Relase the lock if it has been acquired. Can be safely called multiple times. + * Only the first call will unlock. + */ +public void close() throws Exception +{ +if ( acquired.compareAndSet(true, false) ) +{ +releaseLock(); +} +} + +protected void releaseLock() throws Exception +{ +lock.release(); +} + +protected boolean acquireLock(InterProcessLock lock, long timeout, TimeUnit unit) throws Exception +{ +return lock.acquire(timeout, unit); +} +} http://git-wip-us.apache.org/repos/asf/curator/blob/58a8818b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java index a784e46..5a8168e 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java @@ -56,6 +56,28 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests protected abstract InterProcessLock makeLock(CuratorFramework client); @Test +public void testLocker() throws Exception +{ +final Timing timing = new Timing(); +final CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new ExponentialBackoffRetry(100, 3)); +try +{ +client.start(); + +InterProcessLock lock = makeLock(client); +try ( Locker locker = new Locker(lock, timing.mil
[2/3] curator git commit: Added an indefinite acquire version of the constructor
Added an indefinite acquire version of the constructor Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/543860f4 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/543860f4 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/543860f4 Branch: refs/heads/CURATOR-3.0 Commit: 543860f4811fa5327cabe524dad786363eb3f504 Parents: 58a8818 Author: randgalt <randg...@apache.org> Authored: Sun Sep 6 15:43:58 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 6 15:43:58 2015 -0700 -- .../curator/framework/recipes/locks/Locker.java | 20 ++-- 1 file changed, 18 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/543860f4/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java -- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java index 97788af..7eb362d 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/Locker.java @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean; public class Locker implements AutoCloseable { private final InterProcessLock lock; -private final AtomicBoolean acquired; +private final AtomicBoolean acquired = new AtomicBoolean(false); /** * @param lock a lock implementation (e.g. {@link InterProcessMutex}, {@link InterProcessSemaphoreV2}, etc.) @@ -35,13 +35,24 @@ public class Locker implements AutoCloseable public Locker(InterProcessLock lock, long timeout, TimeUnit unit) throws Exception { this.lock = lock; -acquired = new AtomicBoolean(acquireLock(lock, timeout, unit)); +acquired.set(acquireLock(lock, timeout, unit)); if ( !acquired.get() ) { throw new TimeoutException("Could not acquire lock within timeout of " + unit.toMillis(timeout) + "ms"); } } +/** + * @param lock a lock implementation (e.g. {@link InterProcessMutex}, {@link InterProcessSemaphoreV2}, etc.) + * @throws Exception errors + */ +public Locker(InterProcessLock lock) throws Exception +{ +this.lock = lock; +acquireLock(lock); +acquired.set(true); +} + @Override /** * Relase the lock if it has been acquired. Can be safely called multiple times. @@ -60,6 +71,11 @@ public class Locker implements AutoCloseable lock.release(); } +protected void acquireLock(InterProcessLock lock) throws Exception +{ +lock.acquire(); +} + protected boolean acquireLock(InterProcessLock lock, long timeout, TimeUnit unit) throws Exception { return lock.acquire(timeout, unit);
[3/9] curator git commit: Added since tags
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-3.0 Commit: 94dff8a5a2ec336a23c05ebe5cdf7e4b117d3925 Parents: 2e1e92e Author: randgalt <randg...@apache.org> Authored: Mon Aug 24 12:31:52 2015 -0500 Committer: randgalt <randg...@apache.org> 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 {
[6/9] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-248
Merge branch 'CURATOR-3.0' into CURATOR-248 Conflicts: curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/c117b085 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/c117b085 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/c117b085 Branch: refs/heads/CURATOR-3.0 Commit: c117b0853b5779829ff732514aedd7d60b696ccc Parents: d412f23 d57aaeb Author: randgalt <randg...@apache.org> Authored: Tue Sep 1 06:27:31 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 1 06:27:31 2015 -0700 -- .../org/apache/curator/ConnectionState.java | 108 --- .../apache/curator/CuratorZookeeperClient.java | 112 +-- .../java/org/apache/curator/HandleHolder.java | 19 ++ .../main/java/org/apache/curator/RetryLoop.java | 18 +- .../ClassicConnectionHandlingPolicy.java| 68 + .../connection/ConnectionHandlingPolicy.java| 72 + .../StandardConnectionHandlingPolicy.java | 56 .../org/apache/curator/retry/RetryForever.java | 60 .../org/apache/curator/utils/DebugUtils.java| 11 +- .../java/org/apache/curator/TestEnsurePath.java | 3 + .../java/org/apache/curator/TestRetryLoop.java | 20 ++ .../framework/CuratorFrameworkFactory.java | 58 ...reateModeStatBackgroundPathAndBytesable.java | 25 ++ .../api/BackgroundPathableQuietlyable.java | 18 ++ .../api/CreateBackgroundModeStatACLable.java| 70 + .../curator/framework/api/CreateBuilder.java| 9 +- ...ateProtectACLCreateModePathAndBytesable.java | 72 + ...rotectACLCreateModeStatPathAndBytesable.java | 25 ++ .../framework/api/UnhandledErrorListener.java | 4 +- .../imps/ClassicInternalConnectionHandler.java | 58 .../framework/imps/CreateBuilderImpl.java | 299 ++- .../framework/imps/CuratorFrameworkImpl.java| 86 +++--- .../imps/InternalConnectionHandler.java | 10 + .../imps/StandardInternalConnectionHandler.java | 22 ++ .../framework/state/ConnectionState.java| 27 +- .../framework/state/ConnectionStateManager.java | 91 -- .../framework/imps/TestBlockUntilConnected.java | 1 + .../framework/imps/TestCreateReturningStat.java | 199 .../imps/TestEnabledSessionExpiredState.java| 179 +++ .../framework/imps/TestFrameworkEdges.java | 6 +- .../framework/recipes/cache/NodeCache.java | 41 ++- ...estResetConnectionWithBackgroundFailure.java | 19 +- .../recipes/leader/TestLeaderLatch.java | 15 +- .../recipes/leader/TestLeaderSelector.java | 5 +- .../recipes/leader/TestLeaderSelectorEdges.java | 6 +- .../locks/TestInterProcessMutexBase.java| 19 +- .../apache/curator/test/BaseClassForTests.java | 37 ++- .../java/org/apache/curator/test/Timing.java| 35 ++- curator-x-discovery-server/pom.xml | 6 + curator-x-discovery/pom.xml | 6 + .../discovery/details/TestServiceDiscovery.java | 2 + curator-x-rpc/pom.xml | 6 + src/site/confluence/errors.confluence | 6 +- src/site/confluence/index.confluence| 7 + 44 files changed, 1798 insertions(+), 218 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/c117b085/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java -- diff --cc curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java index 9a67684,daffa13..2b01b30 --- 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,8 -34,7 +34,9 @@@ import org.apache.curator.framework.imp 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.framework.state.ConnectionState; import org.apache.curator.utils.DefaultZookeeperFactory; import org.apache.curator.utils.ZookeeperFactory; import org.apache.zookeeper.CreateMode; @@@ -118,7 -121,7 +123,8 @@@ public class CuratorFrameworkFactor private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER; private boolean canBeReadOnly = false; pri
[5/9] curator git commit: removed import
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-3.0 Commit: d412f2320d571d23c8960214482e84a7911bec16 Parents: 5429a21 Author: randgalt <randg...@apache.org> Authored: Mon Aug 24 22:03:13 2015 -0500 Committer: randgalt <randg...@apache.org> 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;
[7/9] curator git commit: refined the tests
refined the tests Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/49b267d1 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/49b267d1 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/49b267d1 Branch: refs/heads/CURATOR-3.0 Commit: 49b267d185bddb7d2be073731ebc6a8352ed6efb Parents: c117b08 Author: randgalt <randg...@apache.org> Authored: Tue Sep 1 06:32:52 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 1 06:32:52 2015 -0700 -- .../framework/recipes/leader/TestLeaderSelector.java | 13 + 1 file changed, 9 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/49b267d1/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java index e5b9717..cd76bc1 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java @@ -37,6 +37,7 @@ import org.apache.curator.utils.CloseableUtils; import org.testng.Assert; import org.testng.annotations.Test; import org.testng.internal.annotations.Sets; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; @@ -103,8 +104,10 @@ public class TestLeaderSelector extends BaseClassForTests Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name()); Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "leader"); server.close(); - Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED.name()); - Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "release"); +List next = Lists.newArrayList(); +next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS)); +next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS)); + Assert.assertTrue(next.equals(Arrays.asList(ConnectionState.SUSPENDED.name(), "release")) || next.equals(Arrays.asList("release", ConnectionState.SUSPENDED.name())), next.toString()); Assert.assertEquals(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST.name()); selector.close(); @@ -130,8 +133,10 @@ public class TestLeaderSelector extends BaseClassForTests Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "leader"); server.stop(); Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED.name()); - Assert.assertEquals(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST.name()); - Assert.assertEquals(changes.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "release"); +next = Lists.newArrayList(); +next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS)); +next.add(changes.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS)); + Assert.assertTrue(next.equals(Arrays.asList(ConnectionState.LOST.name(), "release")) || next.equals(Arrays.asList("release", ConnectionState.LOST.name())), next.toString()); } finally {
[9/9] curator git commit: renamed to clarify meaning
renamed to clarify meaning Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/7aa371e9 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/7aa371e9 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/7aa371e9 Branch: refs/heads/CURATOR-3.0 Commit: 7aa371e9514e1a8227d1ed93833bd9e6238c60ed Parents: 940e845 Author: randgalt <randg...@apache.org> Authored: Sun Sep 6 23:51:26 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 6 23:51:26 2015 -0700 -- .../curator/framework/CuratorFramework.java | 4 ++-- .../framework/CuratorFrameworkFactory.java | 18 +- .../framework/imps/CuratorFrameworkImpl.java| 12 ++-- .../state/ConnectionStateErrorPolicy.java | 20 .../curator/framework/state/ErrorPolicy.java| 20 .../SessionConnectionStateErrorPolicy.java | 13 + .../framework/state/SessionErrorPolicy.java | 13 - .../StandardConnectionStateErrorPolicy.java | 14 ++ .../framework/state/StandardErrorPolicy.java| 14 -- .../framework/recipes/leader/LeaderLatch.java | 4 ++-- .../leader/LeaderSelectorListenerAdapter.java | 2 +- .../recipes/leader/TestLeaderLatch.java | 10 +- .../recipes/leader/TestLeaderSelector.java | 9 - 13 files changed, 76 insertions(+), 77 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/7aa371e9/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 d755d28..3d197a0 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,7 +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.framework.state.ConnectionStateErrorPolicy; import org.apache.curator.utils.EnsurePath; import org.apache.zookeeper.Watcher; @@ -304,5 +304,5 @@ public interface CuratorFramework extends Closeable * * @return error policy */ -public ErrorPolicy getErrorPolicy(); +public ConnectionStateErrorPolicy getConnectionStateErrorPolicy(); } http://git-wip-us.apache.org/repos/asf/curator/blob/7aa371e9/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 2b01b30..4db6d36 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 @@ -34,8 +34,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.framework.state.ConnectionStateErrorPolicy; +import org.apache.curator.framework.state.StandardConnectionStateErrorPolicy; import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.utils.DefaultZookeeperFactory; import org.apache.curator.utils.ZookeeperFactory; @@ -123,7 +123,7 @@ public class CuratorFrameworkFactory private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER; private boolean canBeReadOnly = false; private boolean useContainerParentsIfAvailable = true; -private ErrorPolicy errorPolicy = new StandardErrorPolicy(); +private ConnectionStateErrorPolicy connectionStateErrorPolicy = new StandardConnectionStateErrorPolicy(); private ConnectionHandlingPolicy connectionHandlingPolicy = Boolean.getBoolean("curator-use-classic-connection-handling") ? new ClassicConnectionHandlingPolicy() : new StandardConnectionHandlingPolicy(); /** @@ -353,15 +
[4/9] curator git commit: Updated LeaderLatch for error policy
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-3.0 Commit: 5429a217bb23901aaf2b187bb8c1d760d0a76bcc Parents: 94dff8a Author: randgalt <randg...@apache.org> Authored: Mon Aug 24 17:39:41 2015 -0500 Committer: randgalt <randg...@apache.org> 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; import org.t
[2/9] curator git commit: doc
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-3.0 Commit: 2e1e92e162ef812a9d076695500f0bda8b15b6c5 Parents: 45df7ba Author: randgalt <randg...@apache.org> Authored: Mon Aug 24 12:30:33 2015 -0500 Committer: randgalt <randg...@apache.org> 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: link to ZK 3.5 related jiras
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 39fd9255a -> 5d30d07fb link to ZK 3.5 related jiras Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/5d30d07f Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/5d30d07f Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/5d30d07f Branch: refs/heads/CURATOR-3.0 Commit: 5d30d07fb8c8c8a8ea2118bc23bf605bc43c789e Parents: 39fd925 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 18:52:05 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 18:52:05 2015 -0500 -- src/site/confluence/index.confluence | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/5d30d07f/src/site/confluence/index.confluence -- diff --git a/src/site/confluence/index.confluence b/src/site/confluence/index.confluence index 4b5683e..543e041 100644 --- a/src/site/confluence/index.confluence +++ b/src/site/confluence/index.confluence @@ -40,3 +40,6 @@ The are currently two released versions of Curator, 2.x.x and 3.x.x: * Curator 2.x.x \- compatible with both ZooKeeper 3.4.x and ZooKeeper 3.5.x * Curator 3.x.x \- compatible only with ZooKeeper 3.5.x and includes support for new features such as dynamic reconfiguration, etc. + +ZooKeeper 3.5.x adds a number of new features including dynamic reconfiguration, watcher removal and others. The Curator 3.x.x Jira +issues tracking these features are here: [[https://issues.apache.org/jira/browse/CURATOR-159]].
curator git commit: doc the new Locker
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 cee1c0eab -> 39fd9255a doc the new Locker Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/39fd9255 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/39fd9255 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/39fd9255 Branch: refs/heads/CURATOR-3.0 Commit: 39fd9255af429b48809577a631bd0da7f2ca341e Parents: cee1c0e Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 18:14:18 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 18:14:18 2015 -0500 -- src/site/confluence/utilities.confluence | 12 1 file changed, 12 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/39fd9255/src/site/confluence/utilities.confluence -- diff --git a/src/site/confluence/utilities.confluence b/src/site/confluence/utilities.confluence index f0d927d..efacb3c 100644 --- a/src/site/confluence/utilities.confluence +++ b/src/site/confluence/utilities.confluence @@ -14,6 +14,18 @@ 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. Locker + +Curator's Locker uses Java 7's try\-with\-resources feature to making using Curator locks safer: + +{code} +InterProcessMutex mutex = new InterProcessMutex(...) // or any InterProcessLock +try ( Locker locker = new Locker(mutex, maxTimeout, unit) ) +{ + // do work +} +{code} + h2. BlockingQueueConsumer See: *[[DistributedQueue|curator-recipes/distributed-queue.html]]* and *[[DistributedPriorityQueue|curator-recipes/distributed-priority-queue.html]]*
curator git commit: Fixed some old naming
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 30d449ea5 -> cee1c0eab Fixed some old naming Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/cee1c0ea Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/cee1c0ea Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/cee1c0ea Branch: refs/heads/CURATOR-3.0 Commit: cee1c0eabf1e899b239bf69c443fe1e8316a93cd Parents: 30d449e Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 17:52:32 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 17:52:32 2015 -0500 -- .../org/apache/curator/framework/CuratorFrameworkFactory.java | 2 +- .../curator/framework/recipes/leader/TestLeaderLatch.java | 6 +++--- .../curator/framework/recipes/leader/TestLeaderSelector.java | 4 ++-- src/site/confluence/errors.confluence | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/cee1c0ea/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 4db6d36..97185b5 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 @@ -359,7 +359,7 @@ public class CuratorFrameworkFactory * @param connectionStateErrorPolicy new error policy * @return this */ -public Builder errorPolicy(ConnectionStateErrorPolicy connectionStateErrorPolicy) +public Builder connectionStateErrorPolicy(ConnectionStateErrorPolicy connectionStateErrorPolicy) { this.connectionStateErrorPolicy = connectionStateErrorPolicy; return this; http://git-wip-us.apache.org/repos/asf/curator/blob/cee1c0ea/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 ac88fe7..ecaa433 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 @@ -73,7 +73,7 @@ public class TestLeaderLatch extends BaseClassForTests .connectionTimeoutMs(1) .sessionTimeoutMs(6) .retryPolicy(new RetryOneTime(1)) -.errorPolicy(isSessionIteration ? new SessionConnectionStateErrorPolicy() : new StandardConnectionStateErrorPolicy()) +.connectionStateErrorPolicy(isSessionIteration ? new SessionConnectionStateErrorPolicy() : new StandardConnectionStateErrorPolicy()) .build(); final BlockingQueue states = Queues.newLinkedBlockingQueue(); ConnectionStateListener stateListener = new ConnectionStateListener() @@ -143,7 +143,7 @@ public class TestLeaderLatch extends BaseClassForTests .connectionTimeoutMs(1000) .sessionTimeoutMs(timing.session()) .retryPolicy(new RetryOneTime(1)) -.errorPolicy(new StandardConnectionStateErrorPolicy()) +.connectionStateErrorPolicy(new StandardConnectionStateErrorPolicy()) .build(); try { @@ -195,7 +195,7 @@ public class TestLeaderLatch extends BaseClassForTests .connectionTimeoutMs(1000) .sessionTimeoutMs(timing.session()) .retryPolicy(new RetryOneTime(1)) -.errorPolicy(new SessionConnectionStateErrorPolicy()) +.connectionStateErrorPolicy(new SessionConnectionStateErrorPolicy()) .build(); client.getConnectionStateListenable().addListener(stateListener); client.start(); http://git-wip-us.apache.org/repos/asf/curator/blob/cee1c0ea/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelector.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLead
[2/2] curator git commit: fixed surefire spec
fixed surefire spec Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/75aa991e Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/75aa991e Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/75aa991e Branch: refs/heads/CURATOR-3.0 Commit: 75aa991e366cebdb6873509c7ced252731a2187c Parents: 9528136 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 19:23:01 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 19:23:01 2015 -0500 -- pom.xml | 11 +++ 1 file changed, 7 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/75aa991e/pom.xml -- diff --git a/pom.xml b/pom.xml index 8c984ab..d67e52b 100644 --- a/pom.xml +++ b/pom.xml @@ -58,8 +58,6 @@ 1.7 -1 - 3.5.0-alpha 2.7 @@ -82,6 +80,7 @@ 2.3 1.7.6 2.6.1 +2.18.1 @@ -554,6 +553,12 @@ clirr-maven-plugin ${clirr-maven-plugin-version} + + +org.apache.maven.plugins +maven-surefire-plugin +${maven-surefire-plugin-version} + @@ -631,8 +636,6 @@ org.apache.maven.plugins maven-surefire-plugin -${surefire-forkcount} -false true
[1/5] curator git commit: CURATOR-228 - Modified the background callback to explicitly handle the NOAUTH case. This will now log a warning and set a flag indicating that an auth failure has occured.
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 75aa991e3 -> 793ed89b1 CURATOR-228 - Modified the background callback to explicitly handle the NOAUTH case. This will now log a warning and set a flag indicating that an auth failure has occured. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/ecf67df8 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/ecf67df8 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/ecf67df8 Branch: refs/heads/CURATOR-3.0 Commit: ecf67df891d31b43acb45a65743c97976913b3ca Parents: 870b4d5 Author: Cam McKenzieAuthored: Thu Jul 23 13:38:37 2015 +1000 Committer: Cam McKenzie Committed: Thu Jul 23 13:38:37 2015 +1000 -- .../recipes/nodes/PersistentEphemeralNode.java | 14 +++ .../nodes/TestPersistentEphemeralNode.java | 39 2 files changed, 53 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/ecf67df8/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..35e18a5 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 @@ -41,6 +41,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import org.apache.curator.utils.PathUtils; @@ -65,6 +66,7 @@ public class PersistentEphemeralNode implements Closeable private final Mode mode; private final AtomicReference data = new AtomicReference (); private final AtomicReference state = new AtomicReference(State.LATENT); +private final AtomicBoolean authFailure = new AtomicBoolean(false); private final BackgroundCallback backgroundCallback; private final Watcher watcher = new Watcher() { @@ -233,8 +235,15 @@ public class PersistentEphemeralNode implements Closeable { path = event.getName(); } +else if ( event.getResultCode() == KeeperException.Code.NOAUTH.intValue() ) +{ + log.warn("Client does not have authorisation to write ephemeral node at path {}", path); + authFailure.set(true); + return; +} if ( path != null ) { + authFailure.set(false); nodePath.set(path); watchNode(); @@ -406,4 +415,9 @@ public class PersistentEphemeralNode implements Closeable { return (state.get() == State.STARTED); } + +public boolean isAuthFailure() +{ + return authFailure.get(); +} } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/curator/blob/ecf67df8/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..b199872 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 @@ -35,7 +35,9 @@ import org.apache.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -572,6 +574,43 @@ public class TestPersistentEphemeralNode extends BaseClassForTests node.close(); } } + +@Test +public void testNoWritePermission() throws Exception +{ +CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder(); +CuratorFramework
[3/5] curator git commit: Merge branch 'CURATOR-228'
Merge branch 'CURATOR-228' Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/9d2d6851 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/9d2d6851 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/9d2d6851 Branch: refs/heads/CURATOR-3.0 Commit: 9d2d6851e4c737ffe2d2dcd8ec49e946084d339d Parents: 3801167 cff86ea Author: Cam McKenzieAuthored: Tue Sep 1 07:37:39 2015 +1000 Committer: Cam McKenzie Committed: Tue Sep 1 07:37:39 2015 +1000 -- .../recipes/nodes/PersistentEphemeralNode.java | 16 .../nodes/TestPersistentEphemeralNode.java | 39 2 files changed, 55 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/9d2d6851/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java -- http://git-wip-us.apache.org/repos/asf/curator/blob/9d2d6851/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java --
[5/5] curator git commit: Merge branch 'master' into CURATOR-3.0
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/793ed89b Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/793ed89b Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/793ed89b Branch: refs/heads/CURATOR-3.0 Commit: 793ed89b1d150166f24bfb7bf297a5c0fc707f1c Parents: 75aa991 970bd39 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 19:53:32 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 19:53:32 2015 -0500 -- .../recipes/nodes/PersistentEphemeralNode.java | 15 .../nodes/TestPersistentEphemeralNode.java | 39 .../queue/TestBoundedDistributedQueue.java | 3 +- 3 files changed, 55 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/793ed89b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentEphemeralNode.java -- http://git-wip-us.apache.org/repos/asf/curator/blob/793ed89b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestPersistentEphemeralNode.java --
[2/5] curator git commit: CURATOR-228 - Remove public access from the isAuthFailure method.
CURATOR-228 - Remove public access from the isAuthFailure method. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/cff86ead Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/cff86ead Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/cff86ead Branch: refs/heads/CURATOR-3.0 Commit: cff86ead3385cf9dfd74b966906d9b3213bebf87 Parents: ecf67df Author: Cam McKenzieAuthored: Thu Jul 23 15:59:13 2015 +1000 Committer: Cam McKenzie Committed: Thu Jul 23 15:59:13 2015 +1000 -- .../curator/framework/recipes/nodes/PersistentEphemeralNode.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/cff86ead/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 35e18a5..11568c1 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 @@ -19,6 +19,7 @@ package org.apache.curator.framework.recipes.nodes; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.curator.framework.CuratorFramework; @@ -416,7 +417,8 @@ public class PersistentEphemeralNode implements Closeable return (state.get() == State.STARTED); } -public boolean isAuthFailure() +@VisibleForTesting +boolean isAuthFailure() { return authFailure.get(); }
[4/5] curator git commit: CURATOR-254 - Modified the test assertions so that they ensure that the theoretical maximum queue size is not exceeded. The previous limit was exceedable depending on how qui
CURATOR-254 - Modified the test assertions so that they ensure that the theoretical maximum queue size is not exceeded. The previous limit was exceedable depending on how quick consumers were pulling data off the queue. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/970bd396 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/970bd396 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/970bd396 Branch: refs/heads/CURATOR-3.0 Commit: 970bd3963cdca545580460e34ccaead8e9c212f7 Parents: 9d2d685 Author: Cam McKenzieAuthored: Tue Sep 1 09:13:06 2015 +1000 Committer: Cam McKenzie Committed: Tue Sep 1 09:13:06 2015 +1000 -- .../framework/recipes/queue/TestBoundedDistributedQueue.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/970bd396/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestBoundedDistributedQueue.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestBoundedDistributedQueue.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestBoundedDistributedQueue.java index 973ef51..80b4861 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestBoundedDistributedQueue.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/TestBoundedDistributedQueue.java @@ -67,7 +67,6 @@ public class TestBoundedDistributedQueue extends BaseClassForTests final int CLIENT_QTY = 4; final int MAX_ITEMS = 10; final int ADD_ITEMS = MAX_ITEMS * 100; -final int SLOP_FACTOR = 2; final QueueConsumer consumer = new QueueConsumer() { @@ -181,7 +180,7 @@ public class TestBoundedDistributedQueue extends BaseClassForTests for ( int count : counts ) { -Assert.assertTrue(count <= (MAX_ITEMS * SLOP_FACTOR), counts.toString()); +Assert.assertTrue(count <= (MAX_ITEMS * CLIENT_QTY), counts.toString()); } } finally
[43/45] curator git commit: Use negotiated session timeout in processEvents()
Use negotiated session timeout in processEvents() Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/c641c029 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/c641c029 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/c641c029 Branch: refs/heads/CURATOR-248 Commit: c641c0296275bb12a156c2a6a1655e0cc968fad4 Parents: 667b1d2 Author: randgalt <randg...@apache.org> Authored: Mon Aug 31 19:50:21 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 31 19:50:21 2015 -0700 -- .../apache/curator/framework/state/ConnectionStateManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/c641c029/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java index daa33f6..3d44d45 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java @@ -250,7 +250,9 @@ public class ConnectionStateManager implements Closeable { while ( !Thread.currentThread().isInterrupted() && (state.get() == State.STARTED) ) { -int pollMaxMs = (sessionTimeoutMs * 2) / 3; // 2/3 of session timeout +int lastNegotiatedSessionTimeoutMs = client.getZookeeperClient().getLastNegotiatedSessionTimeoutMs(); +int useSessionTimeoutMs = (lastNegotiatedSessionTimeoutMs > 0) ? lastNegotiatedSessionTimeoutMs : sessionTimeoutMs; +int pollMaxMs = (useSessionTimeoutMs * 2) / 3; // 2/3 of session timeout final ConnectionState newState = eventQueue.poll(pollMaxMs, TimeUnit.MILLISECONDS); if ( newState != null ) {
[31/45] curator git commit: Merge branch 'CURATOR-234'
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-248 Commit: 5dc27c1fa9b8e3bc6ed3e1b198bb1eca6b378c9a Parents: 061ed0a b177ed7 Author: Mike DrobAuthored: 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(+) --
[06/45] curator git commit: license
license Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/c5a49216 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/c5a49216 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/c5a49216 Branch: refs/heads/CURATOR-248 Commit: c5a49216cc78b05201a8ded357e50e0b6313 Parents: 62f3c33 Author: randgalt <randg...@apache.org> Authored: Fri Aug 21 17:37:15 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Aug 21 17:37:15 2015 -0500 -- .../api/BackgroundPathableQuietlyable.java| 18 ++ .../imps/TestEnabledSessionExpiredState.java | 18 ++ 2 files changed, 36 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/c5a49216/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java index 8ed73fa..cd88739 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.framework.api; public interface BackgroundPathableQuietlyable extends BackgroundPathable, Quietly<BackgroundPathable> http://git-wip-us.apache.org/repos/asf/curator/blob/c5a49216/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/TestEnabledSessionExpiredState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java index 030a292..4c5ff81 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.framework.imps; import com.google.common.collect.Queues;
[34/45] curator git commit: CURATOR-161 Added missing license header
CURATOR-161 Added missing license header Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/0f5668b3 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/0f5668b3 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/0f5668b3 Branch: refs/heads/CURATOR-248 Commit: 0f5668b3127f280e6dd1d6d3ec665420e2d46b2d Parents: f9af0ce Author: Mike DrobAuthored: Tue Aug 25 15:14:01 2015 -0500 Committer: Mike Drob Committed: Tue Aug 25 15:14:01 2015 -0500 -- .../api/BackgroundPathableQuietlyable.java| 18 ++ 1 file changed, 18 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/0f5668b3/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java index 8ed73fa..cd88739 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.framework.api; public interface BackgroundPathableQuietlyable extends BackgroundPathable, Quietly
[16/45] curator git commit: Curator has a big problem with thread interrupted states getting cleared. There are several issues on this (CURATOR-208, CURATOR-205, CURATOR-228, CURATOR-109
Curator has a big problem with thread interrupted states getting cleared. There are several issues on this (CURATOR-208, CURATOR-205, CURATOR-228, CURATOR-109 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/23554479 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/23554479 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/23554479 Branch: refs/heads/CURATOR-248 Commit: 23554479597d654fa8318cdc579fc3cc29bc2c54 Parents: 30bd7b6 Author: randgalt <randg...@apache.org> Authored: Sat Aug 22 20:10:34 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Aug 22 20:10:34 2015 -0500 -- .../org/apache/curator/framework/state/ConnectionStateManager.java | 2 +- .../java/org/apache/curator/framework/imps/TestFrameworkEdges.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/23554479/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java index 406099d..44d511b 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java @@ -248,7 +248,7 @@ public class ConnectionStateManager implements Closeable { try { -while ( !Thread.currentThread().isInterrupted() ) +while ( !Thread.currentThread().isInterrupted() && (state.get() == State.STARTED) ) { int pollMaxMs = (sessionTimeoutMs * 2) / 3; // 2/3 of session timeout final ConnectionState newState = eventQueue.poll(pollMaxMs, TimeUnit.MILLISECONDS); http://git-wip-us.apache.org/repos/asf/curator/blob/23554479/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java index 7407eab..15781dc 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java @@ -195,7 +195,6 @@ public class TestFrameworkEdges extends BaseClassForTests @Test public void testGetAclNoStat() throws Exception { - CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)); client.start(); try
[02/45] curator git commit: CURATOR-239 - Interrupting current thread after catching InterruptedException
CURATOR-239 - Interrupting current thread after catching InterruptedException Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/3e3a8103 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/3e3a8103 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/3e3a8103 Branch: refs/heads/CURATOR-248 Commit: 3e3a810312f996874c327f42c24aa5b10a91161a Parents: 6e8c108 Author: Leandro NunesAuthored: Fri Jul 24 00:18:18 2015 +0100 Committer: Leandro Nunes Committed: Fri Jul 24 00:18:18 2015 +0100 -- .../src/main/java/org/apache/curator/retry/RetryForever.java| 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/3e3a8103/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java -- diff --git a/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java b/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java index 27444b9..89f0276 100644 --- a/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java +++ b/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java @@ -52,6 +52,7 @@ public class RetryForever implements RetryPolicy catch (InterruptedException e) { log.warn("Error occurred while sleeping", e); +Thread.currentThread().interrupt(); } return true; }
[10/45] curator git commit: Fixed testRetry() for new LOST behavior
Fixed testRetry() for new LOST behavior Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/ec2f9bd5 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/ec2f9bd5 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/ec2f9bd5 Branch: refs/heads/CURATOR-248 Commit: ec2f9bd555d01b324bd5ef690b1036d98e1f3702 Parents: 847cc0d Author: randgalt <randg...@apache.org> Authored: Sat Aug 22 11:06:33 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Aug 22 11:06:33 2015 -0500 -- .../org/apache/curator/CuratorZookeeperClient.java | 17 +++-- .../main/java/org/apache/curator/RetryLoop.java| 12 +++- .../java/org/apache/curator/utils/DebugUtils.java | 12 +++- .../curator/framework/imps/TestFrameworkEdges.java | 4 +++- .../org/apache/curator/test/BaseClassForTests.java | 5 + 5 files changed, 41 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/ec2f9bd5/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java -- diff --git a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java index ce6e9d3..a065d78 100644 --- a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java +++ b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java @@ -19,6 +19,7 @@ package org.apache.curator; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.curator.drivers.TracerDriver; import org.apache.curator.ensemble.EnsembleProvider; @@ -51,6 +52,7 @@ public class CuratorZookeeperClient implements Closeable private final AtomicBoolean started = new AtomicBoolean(false); private final AtomicReference tracer = new AtomicReference(new DefaultTracerDriver()); private final boolean manageTimeouts; +private final AtomicReference debugException = new AtomicReference<>(); /** * @@ -207,8 +209,7 @@ public class CuratorZookeeperClient implements Closeable if ( !started.compareAndSet(false, true) ) { -IllegalStateException ise = new IllegalStateException("Already started"); -throw ise; +throw new IllegalStateException("Already started"); } state.start(); @@ -337,6 +338,18 @@ public class CuratorZookeeperClient implements Closeable return manageTimeouts; } +@VisibleForTesting +public void setDebugException(Exception e) +{ +debugException.set(e); +} + +@VisibleForTesting +Exception getDebugException() +{ +return debugException.get(); +} + void addParentWatcher(Watcher watcher) { state.addParentWatcher(watcher); http://git-wip-us.apache.org/repos/asf/curator/blob/ec2f9bd5/curator-client/src/main/java/org/apache/curator/RetryLoop.java -- diff --git a/curator-client/src/main/java/org/apache/curator/RetryLoop.java b/curator-client/src/main/java/org/apache/curator/RetryLoop.java index 8d77cf7..f6abf21 100644 --- a/curator-client/src/main/java/org/apache/curator/RetryLoop.java +++ b/curator-client/src/main/java/org/apache/curator/RetryLoop.java @@ -74,6 +74,7 @@ public class RetryLoop unit.sleep(time); } }; +private static final boolean checkInjectedDebugExceptions = Boolean.getBoolean(DebugUtils.PROPERTY_CHECK_INJECTED_DEBUG_EXCEPTIONS); /** * Returns the default retry sleeper @@ -103,13 +104,22 @@ public class RetryLoop { try { +if ( checkInjectedDebugExceptions ) +{ +Exception debugException = client.getDebugException(); +if ( debugException != null ) +{ +throw debugException; +} +} + client.internalBlockUntilConnectedOrTimedOut(); if ( !client.isConnected() && !client.retryConnectionTimeouts() ) { connectionFailed = true; break; } - + result = proc.call(); retryLoop.markComplete(); } http://git-wip-us.apache.org/repos/asf/curator/blob/ec2f9bd5/curator-client/src/main/java/org/apache/curator/utils/DebugUtils.java -- diff --git a/curator-client/src/main/java/org/apache/curator/ut
[04/45] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-247
Merge branch 'CURATOR-3.0' into CURATOR-247 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/2343daf2 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/2343daf2 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/2343daf2 Branch: refs/heads/CURATOR-248 Commit: 2343daf29388566b0efa0b0a2ad21574fb534a27 Parents: 344634a 2fc9e37 Author: randgalt <randg...@apache.org> Authored: Fri Aug 21 15:11:59 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Aug 21 15:11:59 2015 -0500 -- .../recipes/queue/DistributedIdQueue.java | 28 +++- .../recipes/queue/DistributedQueue.java | 7 - .../recipes/queue/TestDistributedIdQueue.java | 2 +- .../recipes/queue/TestDistributedQueue.java | 2 +- 4 files changed, 35 insertions(+), 4 deletions(-) --
[40/45] curator git commit: Merge branch 'CURATOR-214' into CURATOR-3.0
Merge branch 'CURATOR-214' 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/667c97e7 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/667c97e7 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/667c97e7 Branch: refs/heads/CURATOR-248 Commit: 667c97e7f3e2966153fb5c105020deaac7d0f510 Parents: d01eabe 6c5a154 Author: Cam McKenzieAuthored: Tue Sep 1 09:35:26 2015 +1000 Committer: Cam McKenzie Committed: Tue Sep 1 09:35:26 2015 +1000 -- ...reateModeStatBackgroundPathAndBytesable.java | 25 ++ .../api/CreateBackgroundModeStatACLable.java| 70 + .../curator/framework/api/CreateBuilder.java| 9 +- ...ateProtectACLCreateModePathAndBytesable.java | 72 + ...rotectACLCreateModeStatPathAndBytesable.java | 25 ++ .../framework/imps/CreateBuilderImpl.java | 299 ++- .../framework/imps/TestCreateReturningStat.java | 199 .../framework/imps/TestFrameworkEdges.java | 4 +- 8 files changed, 683 insertions(+), 20 deletions(-) --
[33/45] 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
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-248 Commit: 8fae7856edc1a5269fd012c683860e0b150e13b3 Parents: 5dc27c1 Author: Cameron McKenzieAuthored: 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); +} }
[13/45] curator git commit: Merge remote-tracking branch 'origin/CURATOR-3.0' into CURATOR-214
Merge remote-tracking branch 'origin/CURATOR-3.0' into CURATOR-214 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/ba4b5d8c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/ba4b5d8c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/ba4b5d8c Branch: refs/heads/CURATOR-248 Commit: ba4b5d8cb1f9733d3901b0b619528454d3dbf8c8 Parents: af369de 2fc9e37 Author: Cam McKenzieAuthored: Sun Aug 23 09:35:05 2015 +1000 Committer: Cam McKenzie Committed: Sun Aug 23 09:35:05 2015 +1000 -- .../recipes/queue/DistributedIdQueue.java | 28 +++- .../recipes/queue/DistributedQueue.java | 7 - .../recipes/queue/TestDistributedIdQueue.java | 2 +- .../recipes/queue/TestDistributedQueue.java | 2 +- 4 files changed, 35 insertions(+), 4 deletions(-) --
[32/45] curator git commit: minor typo
minor typo Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/bc57786c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/bc57786c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/bc57786c Branch: refs/heads/CURATOR-248 Commit: bc57786c9c55f06f8918e0e2255277bfb2fbbecb Parents: b8240a1 Author: randgalt <randg...@apache.org> Authored: Mon Aug 24 11:11:16 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 24 11:11:16 2015 -0500 -- src/site/confluence/errors.confluence | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/bc57786c/src/site/confluence/errors.confluence -- diff --git a/src/site/confluence/errors.confluence b/src/site/confluence/errors.confluence index 0111cf4..a744128 100644 --- a/src/site/confluence/errors.confluence +++ b/src/site/confluence/errors.confluence @@ -26,7 +26,7 @@ appropriate action. These are the possible state changes: |READ_ONLY|The connection has gone into read\-only mode. This can only happen if you pass true for CuratorFrameworkFactory.Builder.canBeReadOnly(). See the ZooKeeper doc regarding read only connections: [[http://wiki.apache.org/hadoop/ZooKeeper/GSoCReadOnlyMode]]. The connection will remain in read only mode until another state change is sent.| |SUSPENDED|There has been a loss of connection. Leaders, locks, etc. should suspend until the connection is re\-established.| |RECONNECTED|A suspended or lost connection has been re\-established.| -|LOST|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 Watcher.Event.KeeperState.Expired or KeeperException.Code.SESSIONEXPIRED; b) Curator closes the internally managed ZooKeeper instance; c) The session timeout elapses during a network partition. It is possible to get a RECONNECTED state after this but you should still consider any locks, etc. as dirty/unstable. *NOTE* The meaning of LOST has changed since Curator 3.0.0. Prior to 3.0.0 LOST only meant that the retry policy had expired.| +|LOST|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 Watcher.Event.KeeperState.Expired or KeeperException.Code.SESSIONEXPIRED; b) Curator closes the internally managed ZooKeeper instance; c) The session timeout elapses during a network partition. It is possible to get a RECONNECTED state after this but you should still consider any locks, etc. as dirty/unstable. *NOTE*: The meaning of LOST has changed since Curator 3.0.0. Prior to 3.0.0 LOST only meant that the retry policy had expired.| {{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.
[26/45] curator git commit: code for useSessionTimeoutMs was wrong
code for useSessionTimeoutMs was wrong Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/dd788163 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/dd788163 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/dd788163 Branch: refs/heads/CURATOR-248 Commit: dd788163a278e2c10f71d2f0fd8b68137806f9f0 Parents: 966b8df Author: randgalt <randg...@apache.org> Authored: Sun Aug 23 20:33:55 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Aug 23 20:33:55 2015 -0500 -- .../apache/curator/framework/state/ConnectionStateManager.java| 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/dd788163/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java index 3da7534..de71264 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/state/ConnectionStateManager.java @@ -292,7 +292,8 @@ public class ConnectionStateManager implements Closeable if ( (currentConnectionState == ConnectionState.SUSPENDED) && (startOfSuspendedEpoch != 0) ) { long elapsedMs = System.currentTimeMillis() - startOfSuspendedEpoch; -int useSessionTimeoutMs = Math.max(client.getZookeeperClient().getLastNegotiatedSessionTimeoutMs(), sessionTimeoutMs); +int lastNegotiatedSessionTimeoutMs = client.getZookeeperClient().getLastNegotiatedSessionTimeoutMs(); +int useSessionTimeoutMs = (lastNegotiatedSessionTimeoutMs > 0) ? lastNegotiatedSessionTimeoutMs : sessionTimeoutMs; if ( elapsedMs >= useSessionTimeoutMs ) { log.warn(String.format("Session timeout has elapsed while SUSPENDED. Posting LOST event and resetting the connection. Elapsed ms: %d. Session Timeout ms: %d", elapsedMs, useSessionTimeoutMs));
[03/45] curator git commit: First pass at new (optional) definition of state LOST
First pass at new (optional) definition of state LOST Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/344634ac Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/344634ac Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/344634ac Branch: refs/heads/CURATOR-248 Commit: 344634ac6e34e61bc0cc7b41923a1df4089c7948 Parents: 7d97259 Author: randgalt <randg...@apache.org> Authored: Fri Aug 21 12:10:24 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Aug 21 12:10:24 2015 -0500 -- .../framework/CuratorFrameworkFactory.java | 19 + .../framework/api/UnhandledErrorListener.java | 4 +- .../framework/imps/CuratorFrameworkImpl.java| 10 ++- .../framework/state/ConnectionState.java| 35 +++-- .../framework/state/ConnectionStateManager.java | 75 +++- 5 files changed, 113 insertions(+), 30 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/344634ac/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..6209b06 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,7 @@ 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.ConnectionState; import org.apache.curator.utils.DefaultZookeeperFactory; import org.apache.curator.utils.ZookeeperFactory; import org.apache.zookeeper.CreateMode; @@ -116,6 +117,7 @@ public class CuratorFrameworkFactory private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER; private boolean canBeReadOnly = false; private boolean useContainerParentsIfAvailable = true; +private boolean enableSessionExpiredState = false; /** * Apply the current values and build a new CuratorFramework @@ -343,6 +345,18 @@ public class CuratorFrameworkFactory return this; } +/** + * Changes the meaning of {@link ConnectionState#LOST} from it's pre Curator 3.0.0 meaning + * to a true lost session state. See the {@link ConnectionState#LOST} doc for details. + * + * @return this + */ +public Builder enableSessionExpiredState() +{ +this.enableSessionExpiredState = true; +return this; +} + public ACLProvider getAclProvider() { return aclProvider; @@ -398,6 +412,11 @@ public class CuratorFrameworkFactory return useContainerParentsIfAvailable; } +public boolean getEnableSessionExpiredState() +{ +return enableSessionExpiredState; +} + @Deprecated public String getAuthScheme() { http://git-wip-us.apache.org/repos/asf/curator/blob/344634ac/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java b/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java index b463af2..3721d4b 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java @@ -24,9 +24,7 @@ import org.apache.curator.framework.state.ConnectionStateListener; public interface UnhandledErrorListener { /** - * Called when an exception is caught in a background thread, handler, etc. Before this - * listener is called, the error will have been logged and a {@link ConnectionState#LOST} - * event will have been queued for any {@link ConnectionStateListener}s. + * Called when an exception is caught in a background thread, handler, etc. * * @param message Source message * @param e exception http://git-wip-us.apache.org/repos/asf/curator/blob/344634ac/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java -- diff --gi
[45/45] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-248
Merge branch 'CURATOR-3.0' into CURATOR-248 Conflicts: curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/c117b085 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/c117b085 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/c117b085 Branch: refs/heads/CURATOR-248 Commit: c117b0853b5779829ff732514aedd7d60b696ccc Parents: d412f23 d57aaeb Author: randgalt <randg...@apache.org> Authored: Tue Sep 1 06:27:31 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 1 06:27:31 2015 -0700 -- .../org/apache/curator/ConnectionState.java | 108 --- .../apache/curator/CuratorZookeeperClient.java | 112 +-- .../java/org/apache/curator/HandleHolder.java | 19 ++ .../main/java/org/apache/curator/RetryLoop.java | 18 +- .../ClassicConnectionHandlingPolicy.java| 68 + .../connection/ConnectionHandlingPolicy.java| 72 + .../StandardConnectionHandlingPolicy.java | 56 .../org/apache/curator/retry/RetryForever.java | 60 .../org/apache/curator/utils/DebugUtils.java| 11 +- .../java/org/apache/curator/TestEnsurePath.java | 3 + .../java/org/apache/curator/TestRetryLoop.java | 20 ++ .../framework/CuratorFrameworkFactory.java | 58 ...reateModeStatBackgroundPathAndBytesable.java | 25 ++ .../api/BackgroundPathableQuietlyable.java | 18 ++ .../api/CreateBackgroundModeStatACLable.java| 70 + .../curator/framework/api/CreateBuilder.java| 9 +- ...ateProtectACLCreateModePathAndBytesable.java | 72 + ...rotectACLCreateModeStatPathAndBytesable.java | 25 ++ .../framework/api/UnhandledErrorListener.java | 4 +- .../imps/ClassicInternalConnectionHandler.java | 58 .../framework/imps/CreateBuilderImpl.java | 299 ++- .../framework/imps/CuratorFrameworkImpl.java| 86 +++--- .../imps/InternalConnectionHandler.java | 10 + .../imps/StandardInternalConnectionHandler.java | 22 ++ .../framework/state/ConnectionState.java| 27 +- .../framework/state/ConnectionStateManager.java | 91 -- .../framework/imps/TestBlockUntilConnected.java | 1 + .../framework/imps/TestCreateReturningStat.java | 199 .../imps/TestEnabledSessionExpiredState.java| 179 +++ .../framework/imps/TestFrameworkEdges.java | 6 +- .../framework/recipes/cache/NodeCache.java | 41 ++- ...estResetConnectionWithBackgroundFailure.java | 19 +- .../recipes/leader/TestLeaderLatch.java | 15 +- .../recipes/leader/TestLeaderSelector.java | 5 +- .../recipes/leader/TestLeaderSelectorEdges.java | 6 +- .../locks/TestInterProcessMutexBase.java| 19 +- .../apache/curator/test/BaseClassForTests.java | 37 ++- .../java/org/apache/curator/test/Timing.java| 35 ++- curator-x-discovery-server/pom.xml | 6 + curator-x-discovery/pom.xml | 6 + .../discovery/details/TestServiceDiscovery.java | 2 + curator-x-rpc/pom.xml | 6 + src/site/confluence/errors.confluence | 6 +- src/site/confluence/index.confluence| 7 + 44 files changed, 1798 insertions(+), 218 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/c117b085/curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java -- diff --cc curator-framework/src/main/java/org/apache/curator/framework/CuratorFrameworkFactory.java index 9a67684,daffa13..2b01b30 --- 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,8 -34,7 +34,9 @@@ import org.apache.curator.framework.imp 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.framework.state.ConnectionState; import org.apache.curator.utils.DefaultZookeeperFactory; import org.apache.curator.utils.ZookeeperFactory; import org.apache.zookeeper.CreateMode; @@@ -118,7 -121,7 +123,8 @@@ public class CuratorFrameworkFactor private ACLProvider aclProvider = DEFAULT_ACL_PROVIDER; private boolean canBeReadOnly = false; pri
[07/45] curator git commit: more tests
more tests Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/d3170099 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/d3170099 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/d3170099 Branch: refs/heads/CURATOR-248 Commit: d3170099757c7e17ff8fbee0c37d620aacb60d65 Parents: c5a4921 Author: randgalt <randg...@apache.org> Authored: Fri Aug 21 17:49:55 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Aug 21 17:49:55 2015 -0500 -- .../imps/TestEnabledSessionExpiredState.java| 24 1 file changed, 24 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/d3170099/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/TestEnabledSessionExpiredState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java index 4c5ff81..150eb50 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java @@ -109,6 +109,30 @@ public class TestEnabledSessionExpiredState extends BaseClassForTests Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED); } +@Test +public void testSessionExpirationFromTimeout() throws Exception +{ +Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED); +server.stop(); +Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED); +Assert.assertEquals(states.poll(timing.multiple(2).session(), TimeUnit.MILLISECONDS), ConnectionState.LOST); +} + +@Test +public void testSessionExpirationFromTimeoutWithRestart() throws Exception +{ +Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED); +server.stop(); +Thread.sleep(timing.multiple(1.2).session()); +Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED); +Assert.assertEquals(states.poll(timing.multiple(2).session(), TimeUnit.MILLISECONDS), ConnectionState.LOST); +server.restart(); +client.checkExists().forPath("/"); +Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED); + +Assert.assertNull(states.poll(timing.multiple(.5).milliseconds(), TimeUnit.MILLISECONDS)); // there should be no other events +} + @Override protected boolean enabledSessionExpiredStateAware() {
[22/45] curator git commit: more work on repairing tests for new connection handling
more work on repairing tests for new connection handling Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/1a2a94b6 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/1a2a94b6 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/1a2a94b6 Branch: refs/heads/CURATOR-248 Commit: 1a2a94b625e7e1b5e535414e397e9b3a4173ca1b Parents: e001e00 Author: randgalt <randg...@apache.org> Authored: Sun Aug 23 10:54:29 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Aug 23 10:54:29 2015 -0500 -- .../curator/framework/recipes/leader/TestLeaderSelectorEdges.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/1a2a94b6/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java index 03708f1..0085968 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderSelectorEdges.java @@ -158,7 +158,7 @@ public class TestLeaderSelectorEdges extends BaseClassForTests final CuratorFramework client = CuratorFrameworkFactory.builder() .connectString(server.getConnectString()) -.retryPolicy(new RetryNTimes(2, 1)) +.retryPolicy(new RetryNTimes(2, 100)) .connectionTimeoutMs(1000) .sessionTimeoutMs(6) .build();
[41/45] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-247
Merge branch 'CURATOR-3.0' into CURATOR-247 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/72f72237 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/72f72237 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/72f72237 Branch: refs/heads/CURATOR-248 Commit: 72f722372aedc221f6dd633113b5f9c30e8bd8e2 Parents: bc57786 d01eabe Author: randgalt <randg...@apache.org> Authored: Mon Aug 31 19:44:24 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 31 19:44:24 2015 -0700 -- .../org/apache/curator/retry/RetryForever.java | 60 .../java/org/apache/curator/TestRetryLoop.java | 20 +++ .../framework/recipes/cache/NodeCache.java | 41 ++--- curator-x-discovery-server/pom.xml | 6 ++ curator-x-discovery/pom.xml | 6 ++ curator-x-rpc/pom.xml | 6 ++ src/site/confluence/index.confluence| 7 +++ 7 files changed, 139 insertions(+), 7 deletions(-) --
[05/45] curator git commit: Continued work on new LOST behavior. Added some tests. To get correct behavior it's necessary to not retry connection failures. Retrying connection failures was never a goo
Continued work on new LOST behavior. Added some tests. To get correct behavior it's necessary to not retry connection failures. Retrying connection failures was never a good idea and here's a good opportunity to fix it as this requires client action to enable Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/62f3c33c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/62f3c33c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/62f3c33c Branch: refs/heads/CURATOR-248 Commit: 62f3c33cdb556eccf6fe1cc87ee74b3458431777 Parents: 2343daf Author: randgalt <randg...@apache.org> Authored: Fri Aug 21 17:35:44 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Aug 21 17:35:44 2015 -0500 -- .../org/apache/curator/ConnectionState.java | 24 ++--- .../apache/curator/CuratorZookeeperClient.java | 58 +++- .../main/java/org/apache/curator/RetryLoop.java | 12 +++ .../framework/CuratorFrameworkFactory.java | 2 +- .../framework/imps/CuratorFrameworkImpl.java| 43 - .../framework/state/ConnectionState.java| 5 + .../framework/state/ConnectionStateManager.java | 13 ++- .../framework/imps/TestBlockUntilConnected.java | 1 + .../imps/TestEnabledSessionExpiredState.java| 99 .../apache/curator/test/BaseClassForTests.java | 37 +++- .../java/org/apache/curator/test/Timing.java| 2 +- 11 files changed, 253 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/62f3c33c/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..1dfdbef 100644 --- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java +++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java @@ -171,6 +171,18 @@ class ConnectionState implements Watcher, Closeable return ensembleProvider; } +synchronized void reset() throws Exception +{ +log.debug("reset"); + +instanceIndex.incrementAndGet(); + +isConnected.set(false); +connectionStartMs = System.currentTimeMillis(); +zooKeeper.closeAndReset(); +zooKeeper.getZooKeeper(); // initiate connection +} + private synchronized void checkTimeouts() throws Exception { int minTimeout = Math.min(sessionTimeoutMs, connectionTimeoutMs); @@ -206,18 +218,6 @@ class ConnectionState implements Watcher, Closeable } } -private synchronized void reset() throws Exception -{ -log.debug("reset"); - -instanceIndex.incrementAndGet(); - -isConnected.set(false); -connectionStartMs = System.currentTimeMillis(); -zooKeeper.closeAndReset(); -zooKeeper.getZooKeeper(); // initiate connection -} - private boolean checkState(Event.KeeperState state, boolean wasConnected) { boolean isConnected = wasConnected; http://git-wip-us.apache.org/repos/asf/curator/blob/62f3c33c/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java -- diff --git a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java index 09b28b2..fbb2f4c 100644 --- a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java +++ b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.curator; import com.google.common.base.Preconditions; @@ -43,12 +44,12 @@ import java.util.concurrent.atomic.AtomicReference; @SuppressWarnings("UnusedDeclaration") public class CuratorZookeeperClient implements Closeable { -private final Loggerlog = LoggerFactory.getLogger(getClass()); -private final ConnectionState state; -private final AtomicReference retryPolicy = new AtomicReference(); -private final int connectionTimeoutMs; -private final AtomicBoolean started = new AtomicBoolean(false); -private final AtomicReference tracer = new AtomicReference(new DefaultTracerDriver()); +private final Logger log = LoggerFactory.getLogger(getClass()); +private final ConnectionState state; +private final AtomicReference retryPolicy = new AtomicReference();
[15/45] curator git commit: further refactoring. Abstracted old framework-level connection handling into ClassicInternalConnectionHandler. Probably more to do here
further refactoring. Abstracted old framework-level connection handling into ClassicInternalConnectionHandler. Probably more to do here Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/30bd7b65 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/30bd7b65 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/30bd7b65 Branch: refs/heads/CURATOR-248 Commit: 30bd7b655d201762d8ff74062964621879ac7134 Parents: e239137 Author: randgalt <randg...@apache.org> Authored: Sat Aug 22 19:29:36 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Aug 22 19:29:36 2015 -0500 -- .../imps/ClassicInternalConnectionHandler.java | 58 ++ .../framework/imps/CuratorFrameworkImpl.java| 64 ++-- .../imps/InternalConnectionHandler.java | 10 +++ .../imps/StandardInternalConnectionHandler.java | 22 +++ .../framework/state/ConnectionStateManager.java | 8 ++- 5 files changed, 112 insertions(+), 50 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/30bd7b65/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java new file mode 100644 index 000..1de6e80 --- /dev/null +++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/ClassicInternalConnectionHandler.java @@ -0,0 +1,58 @@ +package org.apache.curator.framework.imps; + +import org.apache.curator.framework.state.ConnectionState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class ClassicInternalConnectionHandler implements InternalConnectionHandler +{ +private final Logger log = LoggerFactory.getLogger(getClass()); + +@Override +public void checkNewConnection(CuratorFrameworkImpl client) +{ +// NOP +} + +@Override +public boolean checkSessionExpirationEnabled() +{ +return false; +} + +@Override +public void suspendConnection(CuratorFrameworkImpl client) +{ +if ( client.setToSuspended() ) +{ +doSyncForSuspendedConnection(client, client.getZookeeperClient().getInstanceIndex()); +} +} + +private void doSyncForSuspendedConnection(final CuratorFrameworkImpl client, final long instanceIndex) +{ +// we appear to have disconnected, force a new ZK event and see if we can connect to another server +final BackgroundOperation operation = new BackgroundSyncImpl(client, null); +OperationAndData.ErrorCallback errorCallback = new OperationAndData.ErrorCallback() +{ +@Override +public void retriesExhausted(OperationAndData operationAndData) +{ +// if instanceIndex != newInstanceIndex, the ZooKeeper instance was reset/reallocated +// so the pending background sync is no longer valid. +// if instanceIndex is -1, this is the second try to sync - punt and mark the connection lost +if ( (instanceIndex < 0) || (instanceIndex == client.getZookeeperClient().getInstanceIndex()) ) +{ +client.addStateChange(ConnectionState.LOST); +} +else +{ +log.debug("suspendConnection() failure ignored as the ZooKeeper instance was reset. Retrying."); +// send -1 to signal that if it happens again, punt and mark the connection lost +doSyncForSuspendedConnection(client, -1); +} +} +}; +client.performBackgroundOperation(new OperationAndData(operation, "/", null, errorCallback, null)); +} +} http://git-wip-us.apache.org/repos/asf/curator/blob/30bd7b65/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 44a8ec6..b04987d 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 @@ -85,6 +85,7 @@ public class CuratorFrameworkImpl implements CuratorFramework private final NamespaceWatcherMap namespaceWa
[01/45] curator git commit: CURATOR-239 - Adding RetryForever retry policy
Repository: curator Updated Branches: refs/heads/CURATOR-248 d412f2320 -> c117b0853 CURATOR-239 - Adding RetryForever retry policy Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/6e8c1084 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/6e8c1084 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/6e8c1084 Branch: refs/heads/CURATOR-248 Commit: 6e8c10847d4b453a1a1aeafcb6c58f51889f2d1f Parents: 870b4d5 Author: Leandro NunesAuthored: Fri Jul 24 00:06:26 2015 +0100 Committer: Leandro Nunes Committed: Fri Jul 24 00:06:26 2015 +0100 -- .../org/apache/curator/retry/RetryForever.java | 58 .../java/org/apache/curator/TestRetryLoop.java | 20 +++ 2 files changed, 78 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/6e8c1084/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java -- diff --git a/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java b/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java new file mode 100644 index 000..27444b9 --- /dev/null +++ b/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java @@ -0,0 +1,58 @@ +/** + * 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.retry; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.RetrySleeper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.TimeUnit; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * {@link RetryPolicy} implementation that always allowsRetry. + */ +public class RetryForever implements RetryPolicy +{ +private static final Logger log = LoggerFactory.getLogger(RetryForever.class); + +private final int retryIntervalMs; + +public RetryForever(int retryIntervalMs) +{ +checkArgument(retryIntervalMs > 0); +this.retryIntervalMs = retryIntervalMs; +} + +@Override +public boolean allowRetry(int retryCount, long elapsedTimeMs, RetrySleeper sleeper) +{ +try +{ +sleeper.sleepFor(retryIntervalMs, TimeUnit.MILLISECONDS); +} +catch (InterruptedException e) +{ +log.warn("Error occurred while sleeping", e); +} +return true; +} +} http://git-wip-us.apache.org/repos/asf/curator/blob/6e8c1084/curator-client/src/test/java/org/apache/curator/TestRetryLoop.java -- diff --git a/curator-client/src/test/java/org/apache/curator/TestRetryLoop.java b/curator-client/src/test/java/org/apache/curator/TestRetryLoop.java index 0fa9020..17bb91e 100644 --- a/curator-client/src/test/java/org/apache/curator/TestRetryLoop.java +++ b/curator-client/src/test/java/org/apache/curator/TestRetryLoop.java @@ -19,14 +19,19 @@ package org.apache.curator; import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.retry.RetryForever; import org.apache.curator.retry.RetryOneTime; import org.apache.curator.test.BaseClassForTests; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; + import java.util.concurrent.TimeUnit; +import static org.mockito.Mockito.times; + public class TestRetryLoop extends BaseClassForTests { @Test @@ -142,4 +147,19 @@ public class TestRetryLoop extends BaseClassForTests client.close(); } } + +@Test +public void testRetryForever() throws Exception +{ +int retryIntervalMs = 1; +RetrySleeper sleeper = Mockito.mock(RetrySleeper.class); +RetryForever retryForever = new RetryForever(retryIntervalMs); + +for (int i = 0; i < 10; i++) +{ +boolean allowed = retryForever.allowRetry(i,
[17/45] curator git commit: When the connection timeout elapses and there is more than one server in the connection string, reset the connection and try again
When the connection timeout elapses and there is more than one server in the connection string, reset the connection and try again Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/05d241da Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/05d241da Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/05d241da Branch: refs/heads/CURATOR-248 Commit: 05d241da642c6ba0d16b3ce97557128fad4dfe41 Parents: 2355447 Author: randgalt <randg...@apache.org> Authored: Sat Aug 22 20:32:41 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Aug 22 20:32:41 2015 -0500 -- .../src/main/java/org/apache/curator/RetryLoop.java| 5 + .../curator/connection/ConnectionHandlingPolicy.java | 5 + .../connection/StandardConnectionHandlingPolicy.java | 13 - 3 files changed, 22 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/05d241da/curator-client/src/main/java/org/apache/curator/RetryLoop.java -- diff --git a/curator-client/src/main/java/org/apache/curator/RetryLoop.java b/curator-client/src/main/java/org/apache/curator/RetryLoop.java index 35d55a1..a17cbf3 100644 --- a/curator-client/src/main/java/org/apache/curator/RetryLoop.java +++ b/curator-client/src/main/java/org/apache/curator/RetryLoop.java @@ -121,6 +121,11 @@ public class RetryLoop break; } +case WAIT_FOR_CONNECTION: +{ +break; // just loop +} + case EXIT_RETRIES: { retryLoop.markComplete(); http://git-wip-us.apache.org/repos/asf/curator/blob/05d241da/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java index f3ecce6..7f19159 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java @@ -63,6 +63,11 @@ public interface ConnectionHandlingPolicy CALL_PROC, /** + * Wait again for connection success or timeout + */ +WAIT_FOR_CONNECTION, + +/** * Do not call the procedure and exit the retry loop */ EXIT_RETRIES, http://git-wip-us.apache.org/repos/asf/curator/blob/05d241da/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java index 06285ca..cbbceac 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java @@ -1,10 +1,15 @@ package org.apache.curator.connection; +import com.google.common.base.Splitter; import org.apache.curator.CuratorZookeeperClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.concurrent.Callable; public class StandardConnectionHandlingPolicy implements ConnectionHandlingPolicy { +private final Logger log = LoggerFactory.getLogger(getClass()); + @Override public boolean isEmulatingClassicHandling() { @@ -24,9 +29,15 @@ public class StandardConnectionHandlingPolicy implements ConnectionHandlingPolic @Override public PreRetryResult preRetry(CuratorZookeeperClient client) throws Exception { -// TODO - see if there are other servers to connect to if ( !client.isConnected() ) { +int serverCount = Splitter.on(",").omitEmptyStrings().splitToList(client.getCurrentConnectionString()).size(); +if ( serverCount > 1 ) +{ +log.info("Connection timed out and connection string is > 1. Resetting connection and trying again."); +client.reset(); // unfortunately, there's no way to guarantee that ZK tries a different server. Internally it calls Collections.shuffle(). Hopefully, this will result in a different server each time. +return PreRetryResult.WAIT_FOR_CONNECTI
[25/45] curator git commit: The true session timeout is a negotiated value between client and server. The new session timeout handling should use the negotiated value if available
The true session timeout is a negotiated value between client and server. The new session timeout handling should use the negotiated value if available Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/966b8dfc Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/966b8dfc Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/966b8dfc Branch: refs/heads/CURATOR-248 Commit: 966b8dfc8897f766f70a2334c90fa0913e06f996 Parents: 9c7cf5d Author: randgalt <randg...@apache.org> Authored: Sun Aug 23 20:31:21 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Aug 23 20:31:21 2015 -0500 -- .../java/org/apache/curator/ConnectionState.java | 12 .../apache/curator/CuratorZookeeperClient.java | 10 ++ .../java/org/apache/curator/HandleHolder.java| 19 +++ .../framework/state/ConnectionStateManager.java | 5 +++-- 4 files changed, 44 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/966b8dfc/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 d6ddd33..555a52d 100644 --- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java +++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java @@ -36,6 +36,7 @@ import java.util.Queue; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -46,6 +47,7 @@ class ConnectionState implements Watcher, Closeable private final Logger log = LoggerFactory.getLogger(getClass()); 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; @@ -141,6 +143,11 @@ class ConnectionState implements Watcher, Closeable return instanceIndex.get(); } +int getLastNegotiatedSessionTimeoutMs() +{ +return lastNegotiatedSessionTimeoutMs.get(); +} + @Override public void process(WatchedEvent event) { @@ -167,6 +174,11 @@ class ConnectionState implements Watcher, Closeable { isConnected.set(newIsConnected); connectionStartMs = System.currentTimeMillis(); +if ( newIsConnected ) +{ + lastNegotiatedSessionTimeoutMs.set(zooKeeper.getNegotiatedSessionTimeoutMs()); +log.debug("Negotiated session timeout: " + lastNegotiatedSessionTimeoutMs.get()); +} } } http://git-wip-us.apache.org/repos/asf/curator/blob/966b8dfc/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java -- diff --git a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java index d6c2072..471adf0 100644 --- a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java +++ b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java @@ -334,6 +334,16 @@ public class CuratorZookeeperClient implements Closeable return connectionHandlingPolicy; } +/** + * Return the most recent value of {@link ZooKeeper#getSessionTimeout()} or 0 + * + * @return session timeout or 0 + */ +public int getLastNegotiatedSessionTimeoutMs() +{ +return state.getLastNegotiatedSessionTimeoutMs(); +} + void addParentWatcher(Watcher watcher) { state.addParentWatcher(watcher); http://git-wip-us.apache.org/repos/asf/curator/blob/966b8dfc/curator-client/src/main/java/org/apache/curator/HandleHolder.java -- diff --git a/curator-client/src/main/java/org/apache/curator/HandleHolder.java b/curator-client/src/main/java/org/apache/curator/HandleHolder.java index 1f7cd91..8652f0c 100644 --- a/curator-client/src/main/java/org/apache/curator/HandleHolder.java +++ b/curator-client/src/main/java/org/apache/curator/HandleHolder.java @@ -39,6 +39,8 @@ class HandleHolder ZooKeeper getZooKeeper() throws Exception;
[36/45] curator git commit: Include a note about 2 versions of Curator
Include a note about 2 versions of Curator Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/fed0bee8 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/fed0bee8 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/fed0bee8 Branch: refs/heads/CURATOR-248 Commit: fed0bee8450993bfae05d8ce9f350c1f9675de62 Parents: c9cb199 Author: randgalt <randg...@apache.org> Authored: Sun Aug 30 14:07:27 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Sun Aug 30 14:07:27 2015 -0700 -- src/site/confluence/index.confluence | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/fed0bee8/src/site/confluence/index.confluence -- diff --git a/src/site/confluence/index.confluence b/src/site/confluence/index.confluence index 0f0e2dc..4b5683e 100644 --- a/src/site/confluence/index.confluence +++ b/src/site/confluence/index.confluence @@ -39,4 +39,4 @@ h2. Versions The are currently two released versions of Curator, 2.x.x and 3.x.x: * Curator 2.x.x \- compatible with both ZooKeeper 3.4.x and ZooKeeper 3.5.x -* Curator 3.x.x \- compatible only with ZooKeeper 3.5.x and includes support for new features such as containers, dynamic reconfiguration, etc. +* Curator 3.x.x \- compatible only with ZooKeeper 3.5.x and includes support for new features such as dynamic reconfiguration, etc.
[37/45] curator git commit: Merge branch 'CURATOR-239' of github.com:leandronunes85/curator into CURATOR-239
Merge branch 'CURATOR-239' of github.com:leandronunes85/curator into CURATOR-239 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/35b20630 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/35b20630 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/35b20630 Branch: refs/heads/CURATOR-248 Commit: 35b206307447d9dcc8465c941f644ee9697e4723 Parents: 8fae785 3e3a810 Author: randgalt <randg...@apache.org> Authored: Mon Aug 31 07:17:23 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 31 07:17:23 2015 -0700 -- .../org/apache/curator/retry/RetryForever.java | 59 .../java/org/apache/curator/TestRetryLoop.java | 20 +++ 2 files changed, 79 insertions(+) --
[28/45] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-247
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-248 Commit: 1bbf301ab75debf8866c78f13a32e65049a20772 Parents: 81bab45 f9af0ce Author: randgalt <randg...@apache.org> Authored: Mon Aug 24 08:02:08 2015 -0500 Committer: randgalt <randg...@apache.org> 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 --
[14/45] curator git commit: major refactoring. Abstracting old/new behavior into a pluggable ConnectionHandlingPolicy. Also, IMPORTANT, made the new behavior the default. This needs to be discussed bu
major refactoring. Abstracting old/new behavior into a pluggable ConnectionHandlingPolicy. Also, IMPORTANT, made the new behavior the default. This needs to be discussed but it's a major improvement and we should default to it. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/e2391370 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/e2391370 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/e2391370 Branch: refs/heads/CURATOR-248 Commit: e239137019608f02cabb23c27ab13adcef88c027 Parents: 6381ccb Author: randgalt <randg...@apache.org> Authored: Sat Aug 22 19:06:55 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Aug 22 19:06:55 2015 -0500 -- .../org/apache/curator/ConnectionState.java | 85 .../apache/curator/CuratorZookeeperClient.java | 32 .../main/java/org/apache/curator/RetryLoop.java | 28 +-- .../ClassicConnectionHandlingPolicy.java| 48 +++ .../connection/ConnectionHandlingPolicy.java| 84 +++ .../StandardConnectionHandlingPolicy.java | 35 .../java/org/apache/curator/TestEnsurePath.java | 5 +- .../framework/CuratorFrameworkFactory.java | 54 +++-- .../framework/imps/CuratorFrameworkImpl.java| 43 +- .../framework/state/ConnectionState.java| 20 + .../framework/state/ConnectionStateManager.java | 9 +-- .../imps/TestEnabledSessionExpiredState.java| 5 +- .../apache/curator/test/BaseClassForTests.java | 6 +- 13 files changed, 336 insertions(+), 118 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/e2391370/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 c3d6921..d6ddd33 100644 --- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java +++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java @@ -18,9 +18,10 @@ */ package org.apache.curator; -import org.apache.curator.utils.CloseableUtils; +import org.apache.curator.connection.ConnectionHandlingPolicy; import org.apache.curator.drivers.TracerDriver; import org.apache.curator.ensemble.EnsembleProvider; +import org.apache.curator.utils.CloseableUtils; import org.apache.curator.utils.DebugUtils; import org.apache.curator.utils.ZookeeperFactory; import org.apache.zookeeper.KeeperException; @@ -32,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.io.Closeable; import java.io.IOException; import java.util.Queue; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -48,18 +50,19 @@ class ConnectionState implements Watcher, Closeable private final int sessionTimeoutMs; private final int connectionTimeoutMs; private final AtomicReference tracer; +private final ConnectionHandlingPolicy connectionHandlingPolicy; private final Queue backgroundExceptions = new ConcurrentLinkedQueue(); private final Queue parentWatchers = new ConcurrentLinkedQueue(); private final AtomicLong instanceIndex = new AtomicLong(); private volatile long connectionStartMs = 0; -private final AtomicBoolean enableTimeoutChecks = new AtomicBoolean(true); -ConnectionState(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher parentWatcher, AtomicReference tracer, boolean canBeReadOnly) +ConnectionState(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher parentWatcher, AtomicReference tracer, boolean canBeReadOnly, ConnectionHandlingPolicy connectionHandlingPolicy) { this.ensembleProvider = ensembleProvider; this.sessionTimeoutMs = sessionTimeoutMs; this.connectionTimeoutMs = connectionTimeoutMs; this.tracer = tracer; +this.connectionHandlingPolicy = connectionHandlingPolicy; if ( parentWatcher != null ) { parentWatchers.offer(parentWatcher); @@ -68,11 +71,6 @@ class ConnectionState implements Watcher, Closeable zooKeeper = new HandleHolder(zookeeperFactory, this, ensembleProvider, sessionTimeoutMs, canBeReadOnly); } -void disableTimeoutChecks() -{ -enableTimeoutChecks.set(false); -} - ZooKeeper getZooKeeper() throws Exception { if ( SessionFailRetryLoop.sessionForThreadHasFailed() ) @@ -87,13 +85
[39/45] curator git commit: Merge branch 'master' into CURATOR-3.0
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/d01eabe5 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/d01eabe5 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/d01eabe5 Branch: refs/heads/CURATOR-248 Commit: d01eabe5b3ff7624fa2bd423179316931d94c1e5 Parents: fed0bee 3801167 Author: randgalt <randg...@apache.org> Authored: Mon Aug 31 07:19:53 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 31 07:19:53 2015 -0700 -- .../org/apache/curator/retry/RetryForever.java | 60 .../java/org/apache/curator/TestRetryLoop.java | 20 +++ .../framework/recipes/cache/NodeCache.java | 41 ++--- curator-x-discovery-server/pom.xml | 6 ++ curator-x-discovery/pom.xml | 6 ++ curator-x-rpc/pom.xml | 6 ++ 6 files changed, 132 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/d01eabe5/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java -- diff --cc curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCache.java index 49b9a3f,bfc27d8..8b70db1 --- 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,11 -22,10 +22,11 @@@ import com.google.common.annotations.Vi 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.WatcherRemoveCuratorFramework; 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; @@@ -170,10 -178,17 +179,18 @@@ public class NodeCache implements Close { if ( state.compareAndSet(State.STARTED, State.CLOSED) ) { +client.removeWatchers(); 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; + } } /** http://git-wip-us.apache.org/repos/asf/curator/blob/d01eabe5/curator-x-discovery-server/pom.xml -- http://git-wip-us.apache.org/repos/asf/curator/blob/d01eabe5/curator-x-discovery/pom.xml -- http://git-wip-us.apache.org/repos/asf/curator/blob/d01eabe5/curator-x-rpc/pom.xml --
[19/45] curator git commit: CURATOR-214 - Missed file.
CURATOR-214 - Missed file. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/6c5a154c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/6c5a154c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/6c5a154c Branch: refs/heads/CURATOR-248 Commit: 6c5a154c8cb33bc0fa62f5ad9daac8d45db50262 Parents: ba4b5d8 Author: Cam McKenzieAuthored: Sun Aug 23 14:40:12 2015 +1000 Committer: Cam McKenzie Committed: Sun Aug 23 14:40:12 2015 +1000 -- ...rotectACLCreateModeStatPathAndBytesable.java | 25 1 file changed, 25 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/6c5a154c/curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java new file mode 100644 index 000..07638d5 --- /dev/null +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.curator.framework.api; + +public interface ProtectACLCreateModeStatPathAndBytesable extends +ProtectACLCreateModePathAndBytesable, +Statable +{ +}
[12/45] curator git commit: CURATOR-214 - Implemented new ZK API to allow Stat objects to be returned as part of the creation of a new ZNode.
CURATOR-214 - Implemented new ZK API to allow Stat objects to be returned as part of the creation of a new ZNode. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/af369dec Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/af369dec Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/af369dec Branch: refs/heads/CURATOR-248 Commit: af369dec7a3ba20c60c68b502198bd851e34906f Parents: 7d97259 Author: Cam McKenzieAuthored: Sun Aug 23 09:28:49 2015 +1000 Committer: Cam McKenzie Committed: Sun Aug 23 09:28:49 2015 +1000 -- ...reateModeStatBackgroundPathAndBytesable.java | 25 ++ .../api/CreateBackgroundModeStatACLable.java| 70 + .../curator/framework/api/CreateBuilder.java| 9 +- ...ateProtectACLCreateModePathAndBytesable.java | 72 + .../framework/imps/CreateBuilderImpl.java | 299 ++- .../framework/imps/TestCreateReturningStat.java | 199 .../framework/imps/TestFrameworkEdges.java | 4 +- 7 files changed, 658 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java new file mode 100755 index 000..0becfb9 --- /dev/null +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.curator.framework.api; + +public interface ACLCreateModeStatBackgroundPathAndBytesable extends +ACLCreateModeBackgroundPathAndBytesable, +Statable +{ +} http://git-wip-us.apache.org/repos/asf/curator/blob/af369dec/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java new file mode 100755 index 000..2ebe63e --- /dev/null +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBackgroundModeStatACLable.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.curator.framework.api; + +import org.apache.zookeeper.CreateMode; + +public interface CreateBackgroundModeStatACLable extends +BackgroundPathAndBytesable, +CreateModable , +ACLCreateModeBackgroundPathAndBytesable, +Statable +{ +/** + * Causes any parent nodes to get created if they haven't already been + * + * @return this + */ +public ACLCreateModePathAndBytesablecreatingParentsIfNeeded(); + +
[38/45] curator git commit: return false if interrupted
return false if interrupted Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/38011678 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/38011678 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/38011678 Branch: refs/heads/CURATOR-248 Commit: 38011678c4faebb1cd9388e56f0d13d29862c609 Parents: 35b2063 Author: randgalt <randg...@apache.org> Authored: Mon Aug 31 07:19:21 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 31 07:19:21 2015 -0700 -- .../src/main/java/org/apache/curator/retry/RetryForever.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/38011678/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java -- diff --git a/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java b/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java index 89f0276..59f3d0d 100644 --- a/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java +++ b/curator-client/src/main/java/org/apache/curator/retry/RetryForever.java @@ -51,8 +51,9 @@ public class RetryForever implements RetryPolicy } catch (InterruptedException e) { -log.warn("Error occurred while sleeping", e); Thread.currentThread().interrupt(); +log.warn("Error occurred while sleeping", e); +return false; } return true; }
[42/45] curator git commit: Use the negotiated session timeout in checkTimeouts()
Use the negotiated session timeout in checkTimeouts() Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/667b1d2c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/667b1d2c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/667b1d2c Branch: refs/heads/CURATOR-248 Commit: 667b1d2c7bd4b267585ae973dfff0f8b49f6b6ba Parents: 72f7223 Author: randgalt <randg...@apache.org> Authored: Mon Aug 31 19:48:12 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Mon Aug 31 19:48:12 2015 -0700 -- .../src/main/java/org/apache/curator/ConnectionState.java | 6 -- .../apache/curator/connection/ConnectionHandlingPolicy.java| 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/667b1d2c/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 f0996d8..eea2ce0 100644 --- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java +++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java @@ -207,7 +207,9 @@ class ConnectionState implements Watcher, Closeable return zooKeeper.hasNewConnectionString(); } }; -ConnectionHandlingPolicy.CheckTimeoutsResult result = connectionHandlingPolicy.checkTimeouts(hasNewConnectionString, connectionStartMs, sessionTimeoutMs, connectionTimeoutMs); +int lastNegotiatedSessionTimeoutMs = getLastNegotiatedSessionTimeoutMs(); +int useSessionTimeoutMs = (lastNegotiatedSessionTimeoutMs > 0) ? lastNegotiatedSessionTimeoutMs : sessionTimeoutMs; +ConnectionHandlingPolicy.CheckTimeoutsResult result = connectionHandlingPolicy.checkTimeouts(hasNewConnectionString, connectionStartMs, useSessionTimeoutMs, connectionTimeoutMs); switch ( result ) { default: @@ -227,7 +229,7 @@ class ConnectionState implements Watcher, Closeable if ( !Boolean.getBoolean(DebugUtils.PROPERTY_DONT_LOG_CONNECTION_ISSUES) ) { long elapsed = System.currentTimeMillis() - connectionStartMs; -int maxTimeout = Math.max(sessionTimeoutMs, connectionTimeoutMs); +int maxTimeout = Math.max(useSessionTimeoutMs, connectionTimeoutMs); log.warn(String.format("Connection attempt unsuccessful after %d (greater than max timeout of %d). Resetting connection and trying again with a new connection.", elapsed, maxTimeout)); } reset(); http://git-wip-us.apache.org/repos/asf/curator/blob/667b1d2c/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java index 6cea67d..9b6b895 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java @@ -63,7 +63,7 @@ public interface ConnectionHandlingPolicy * @param hasNewConnectionString proc to call to check if there is a new connection string. Important: the internal state is cleared after * this is called so you MUST handle the new connection string if true is returned * @param connectionStartMs the epoch/ms time that the connection was first initiated - * @param sessionTimeoutMs the configured session timeout in milliseconds + * @param sessionTimeoutMs the configured/negotiated session timeout in milliseconds * @param connectionTimeoutMs the configured connection timeout in milliseconds * @return result * @throws Exception errors
[3/3] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-253
Merge branch 'CURATOR-3.0' into CURATOR-253 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/9fc755a9 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/9fc755a9 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/9fc755a9 Branch: refs/heads/CURATOR-3.0 Commit: 9fc755a9e3e91c590f92a149e5894614975d3837 Parents: 145da21 d57aaeb Author: randgalt <randg...@apache.org> Authored: Sun Sep 6 12:51:01 2015 -0700 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 6 12:51:01 2015 -0700 -- .../org/apache/curator/ConnectionState.java | 108 --- .../apache/curator/CuratorZookeeperClient.java | 112 +-- .../java/org/apache/curator/HandleHolder.java | 19 ++ .../main/java/org/apache/curator/RetryLoop.java | 18 +- .../ClassicConnectionHandlingPolicy.java| 68 + .../connection/ConnectionHandlingPolicy.java| 72 + .../StandardConnectionHandlingPolicy.java | 56 .../org/apache/curator/retry/RetryForever.java | 60 .../org/apache/curator/utils/DebugUtils.java| 11 +- .../java/org/apache/curator/TestEnsurePath.java | 3 + .../java/org/apache/curator/TestRetryLoop.java | 20 ++ .../framework/CuratorFrameworkFactory.java | 58 ...reateModeStatBackgroundPathAndBytesable.java | 25 ++ .../api/CreateBackgroundModeStatACLable.java| 70 + .../curator/framework/api/CreateBuilder.java| 9 +- ...ateProtectACLCreateModePathAndBytesable.java | 72 + ...rotectACLCreateModeStatPathAndBytesable.java | 25 ++ .../framework/api/UnhandledErrorListener.java | 4 +- .../imps/ClassicInternalConnectionHandler.java | 58 .../framework/imps/CreateBuilderImpl.java | 299 ++- .../framework/imps/CuratorFrameworkImpl.java| 86 +++--- .../imps/InternalConnectionHandler.java | 10 + .../imps/StandardInternalConnectionHandler.java | 22 ++ .../framework/state/ConnectionState.java| 27 +- .../framework/state/ConnectionStateManager.java | 91 -- .../framework/imps/TestBlockUntilConnected.java | 1 + .../framework/imps/TestCreateReturningStat.java | 199 .../imps/TestEnabledSessionExpiredState.java| 179 +++ .../framework/imps/TestFrameworkEdges.java | 6 +- .../framework/recipes/cache/NodeCache.java | 41 ++- ...estResetConnectionWithBackgroundFailure.java | 19 +- .../recipes/leader/TestLeaderSelectorEdges.java | 6 +- .../locks/TestInterProcessMutexBase.java| 19 +- .../apache/curator/test/BaseClassForTests.java | 37 ++- .../java/org/apache/curator/test/Timing.java| 35 ++- curator-x-discovery-server/pom.xml | 6 + curator-x-discovery/pom.xml | 6 + .../discovery/details/TestServiceDiscovery.java | 2 + curator-x-rpc/pom.xml | 6 + src/site/confluence/errors.confluence | 6 +- src/site/confluence/index.confluence| 7 + 41 files changed, 1767 insertions(+), 211 deletions(-) --
curator git commit: removed import
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 randg...@apache.org Authored: Mon Aug 24 22:03:13 2015 -0500 Committer: randgalt randg...@apache.org 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;
curator git commit: Ignore any shutdown errors
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 b161867c1 -> a69f3a435 Ignore any shutdown errors Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/a69f3a43 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/a69f3a43 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/a69f3a43 Branch: refs/heads/CURATOR-3.0 Commit: a69f3a435755b1dfdd1e706cb028d3169a7c9632 Parents: b161867 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 12:37:20 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 12:37:20 2015 -0500 -- .../org/apache/curator/test/TestingZooKeeperMain.java | 12 ++-- 1 file changed, 10 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/a69f3a43/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java index bb70da5..637166a 100644 --- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java +++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java @@ -119,7 +119,15 @@ public class TestingZooKeeperMain extends ZooKeeperServerMain implements ZooKeep @Override public void close() throws IOException { -shutdown(); +try +{ +shutdown(); +} +catch ( Throwable e ) +{ +// ignore any errors +e.printStackTrace(); +} try { @@ -138,7 +146,7 @@ public class TestingZooKeeperMain extends ZooKeeperServerMain implements ZooKeep } } } -catch ( Exception e ) +catch ( Throwable e ) { e.printStackTrace();// just ignore - this class is only for testing }
[1/2] curator git commit: Ignore any shutdown errors
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 a69f3a435 -> 2d487836f Ignore any shutdown errors Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/c4f3a505 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/c4f3a505 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/c4f3a505 Branch: refs/heads/CURATOR-3.0 Commit: c4f3a505b70f02bf209613307b3bdefaff7bf4fe Parents: b161867 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 12:41:46 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 12:41:46 2015 -0500 -- .../org/apache/curator/test/TestingZooKeeperMain.java| 11 +-- 1 file changed, 9 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/c4f3a505/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java index bb70da5..a13ce9c 100644 --- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java +++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java @@ -119,7 +119,14 @@ public class TestingZooKeeperMain extends ZooKeeperServerMain implements ZooKeep @Override public void close() throws IOException { -shutdown(); +try +{ +shutdown(); +} +catch ( Throwable e ) +{ +e.printStackTrace();// just ignore - this class is only for testing +} try { @@ -138,7 +145,7 @@ public class TestingZooKeeperMain extends ZooKeeperServerMain implements ZooKeep } } } -catch ( Exception e ) +catch ( Throwable e ) { e.printStackTrace();// just ignore - this class is only for testing }
curator git commit: ChaosMonkeyCnxnFactory didn't know about create2 opcode
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 2d487836f -> b969e33af ChaosMonkeyCnxnFactory didn't know about create2 opcode Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/b969e33a Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/b969e33a Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/b969e33a Branch: refs/heads/CURATOR-3.0 Commit: b969e33af4cb9c6d7542a7162263deffd051a7d9 Parents: 2d48783 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 12:49:26 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 12:49:26 2015 -0500 -- .../curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/b969e33a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java index ce5b23d..1c0e50e 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java @@ -89,7 +89,7 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory log.debug("Applied : " + si.toString()); super.submitRequest(si); // Raise an error if a lock is created -if ( si.type == ZooDefs.OpCode.create ) +if ( (si.type == ZooDefs.OpCode.create) || (si.type == ZooDefs.OpCode.create2) ) { CreateRequest createRequest = new CreateRequest(); try
curator git commit: licenses
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 b969e33af -> 95278759b licenses Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/95278759 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/95278759 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/95278759 Branch: refs/heads/CURATOR-3.0 Commit: 95278759ba54f4db3de4fbabe7346187e5c3428a Parents: b969e33 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 12:53:37 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 12:53:37 2015 -0500 -- .../ClassicConnectionHandlingPolicy.java | 18 ++ .../connection/ConnectionHandlingPolicy.java | 18 ++ .../StandardConnectionHandlingPolicy.java | 18 ++ .../imps/ClassicInternalConnectionHandler.java| 18 ++ .../framework/imps/InternalConnectionHandler.java | 18 ++ .../imps/StandardInternalConnectionHandler.java | 18 ++ .../state/ConnectionStateErrorPolicy.java | 18 ++ .../state/SessionConnectionStateErrorPolicy.java | 18 ++ .../state/StandardConnectionStateErrorPolicy.java | 18 ++ .../curator/framework/recipes/locks/Locker.java | 18 ++ 10 files changed, 180 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/95278759/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java index 1d53728..e4c59f4 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.connection; import org.apache.curator.CuratorZookeeperClient; http://git-wip-us.apache.org/repos/asf/curator/blob/95278759/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java index 9b6b895..ae77861 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.connection; import org.apache.curator.CuratorZookeeperClient; http://git-wip-us.apache.org/repos/asf/curator/blob/95278759/curator-client/src/main/java/org/apache/curator/connection/StandardConnectionHandlingPolicy.java -
[3/3] curator git commit: Killed sessions now go directly to LOST (as they should)
Killed sessions now go directly to LOST (as they should) Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/b161867c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/b161867c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/b161867c Branch: refs/heads/CURATOR-3.0 Commit: b161867c1db03c279be0c0ce1eb091bb3372811d Parents: d5b0d0d Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 11:47:19 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 11:47:19 2015 -0500 -- .../curator/framework/imps/TestEnabledSessionExpiredState.java| 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/b161867c/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/TestEnabledSessionExpiredState.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java index 490e75e..7114e48 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestEnabledSessionExpiredState.java @@ -124,8 +124,7 @@ public class TestEnabledSessionExpiredState extends BaseClassForTests KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString()); -Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.SUSPENDED); -Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST); + Assert.assertEquals(states.poll(timing.forSessionSleep().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.LOST); Assert.assertEquals(states.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.RECONNECTED); }
curator git commit: Use forSessionSleep after session killed
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 95278759b -> 48824ebe0 Use forSessionSleep after session killed Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/48824ebe Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/48824ebe Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/48824ebe Branch: refs/heads/CURATOR-3.0 Commit: 48824ebe0df4f384fdb1542d0d9768798c33c058 Parents: 9527875 Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 13:00:09 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 13:00:09 2015 -0500 -- .../framework/recipes/nodes/TestPersistentEphemeralNode.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/48824ebe/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 3a0d564..e182ec0 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 @@ -311,7 +311,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests KillSession.kill(curator.getZookeeperClient().getZooKeeper()); // Make sure the node got deleted - assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS)); + assertTrue(deletedTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS)); node.debugReconnectLatch.countDown(); } finally @@ -340,7 +340,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests KillSession.kill(curator.getZookeeperClient().getZooKeeper()); // Make sure the node got deleted... - assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS)); + assertTrue(deletedTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS)); node.debugReconnectLatch.countDown(); // Check for it to be recreated... @@ -379,7 +379,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests KillSession.kill(curator.getZookeeperClient().getZooKeeper()); // Make sure the node ended up getting deleted... - assertTrue(deletionTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS)); + assertTrue(deletionTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS)); node.debugReconnectLatch.countDown(); // Now put a watch in the background looking to see if it gets created...
curator git commit: sleep a bit so that session can actually expire
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 48824ebe0 -> 30d449ea5 sleep a bit so that session can actually expire Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/30d449ea Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/30d449ea Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/30d449ea Branch: refs/heads/CURATOR-3.0 Commit: 30d449ea5b369805621ee78a36d0a8724473f366 Parents: 48824eb Author: randgalt <randg...@apache.org> Authored: Tue Sep 8 13:09:01 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 8 13:09:01 2015 -0500 -- .../framework/recipes/nodes/TestPersistentEphemeralNode.java | 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/30d449ea/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 e182ec0..b62675e 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 @@ -378,6 +378,8 @@ public class TestPersistentEphemeralNode extends BaseClassForTests // Kill the session, thus cleaning up the node... KillSession.kill(curator.getZookeeperClient().getZooKeeper()); +timing.sleepABit(); + // Make sure the node ended up getting deleted... assertTrue(deletionTrigger.firedWithin(timing.forSessionSleep().seconds(), TimeUnit.SECONDS)); node.debugReconnectLatch.countDown();
curator git commit: use Timing
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 793ed89b1 -> 520fc2c99 use Timing Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/520fc2c9 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/520fc2c9 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/520fc2c9 Branch: refs/heads/CURATOR-3.0 Commit: 520fc2c9976767910317ebf5037386d43ca93005 Parents: 793ed89 Author: randgalt <randg...@apache.org> Authored: Wed Sep 9 01:35:38 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Wed Sep 9 01:35:38 2015 -0500 -- .../apache/curator/framework/imps/TestReconfiguration.java | 9 + 1 file changed, 5 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/520fc2c9/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index 133e690..2438ef8 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -29,6 +29,7 @@ import org.apache.curator.framework.ensemble.EnsembleTracker; import org.apache.curator.retry.RetryOneTime; import org.apache.curator.test.InstanceSpec; import org.apache.curator.test.TestingCluster; +import org.apache.curator.test.Timing; import org.apache.curator.utils.CloseableUtils; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.quorum.flexible.QuorumMaj; @@ -43,12 +44,12 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; public class TestReconfiguration { +private static final Timing timing = new Timing(); private TestingCluster cluster; private DynamicEnsembleProvider dynamicEnsembleProvider; private WaitOnDelegateListener waitOnDelegateListener; @@ -157,7 +158,7 @@ public class TestReconfiguration CountDownLatch latch = new CountDownLatch(1); client.getConfig().inBackground(callback, latch).forEnsemble(); -latch.await(5, TimeUnit.SECONDS); +Assert.assertTrue(timing.awaitLatch(latch)); Assert.assertNotNull(bytes.get()); QuorumVerifier qv = getQuorumVerifier(bytes.get()); Assert.assertEquals(qv.getAllMembers().size(), 5); @@ -274,7 +275,7 @@ public class TestReconfiguration CountDownLatch latch = new CountDownLatch(1); client.getConfig().inBackground(callback, latch).forEnsemble(); -latch.await(5, TimeUnit.SECONDS); +Assert.assertTrue(timing.awaitLatch(latch)); Assert.assertNotNull(bytes.get()); QuorumVerifier qv = getQuorumVerifier(bytes.get()); Assert.assertEquals(qv.getAllMembers().size(), 5); @@ -404,7 +405,7 @@ public class TestReconfiguration public void waitForEvent() throws InterruptedException, TimeoutException { -if ( latch.await(5, TimeUnit.SECONDS) ) +if ( timing.awaitLatch(latch) ) { latch = new CountDownLatch(1); }
curator git commit: moved to ZK 3.5.1.
Repository: curator Updated Branches: refs/heads/CURATOR-261 911f02b15 -> fad270861 moved to ZK 3.5.1. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/fad27086 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/fad27086 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/fad27086 Branch: refs/heads/CURATOR-261 Commit: fad270861feb4dbda36fdea76b4db9ec29e92001 Parents: 911f02b Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 09:22:11 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 09:22:11 2015 -0500 -- .../recipes/leader/ChaosMonkeyCnxnFactory.java | 2 -- .../curator/test/TestingZooKeeperMain.java | 36 +++- pom.xml | 11 +- 3 files changed, 29 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/fad27086/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java index 3aeec81..4cb342c 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java @@ -31,8 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.SelectionKey; -import java.nio.channels.SocketChannel; /** * A connection factory that will behave like the NIOServerCnxnFactory except that http://git-wip-us.apache.org/repos/asf/curator/blob/fad27086/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java index 702824a..a4cca8e 100644 --- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java +++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java @@ -30,12 +30,14 @@ import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.Assert; import javax.management.JMException; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.nio.channels.ServerSocketChannel; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; public class TestingZooKeeperMain implements ZooKeeperMainFace @@ -48,7 +50,9 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace private volatile ServerCnxnFactory cnxnFactory; private volatile ZooKeeperServer zkServer; -static final int MAX_WAIT_MS = new Timing().milliseconds(); +private static final Timing timing = new Timing(); + +static final int MAX_WAIT_MS = timing.milliseconds(); @Override public void kill() @@ -124,7 +128,7 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace @Override public void blockUntilStarted() throws Exception { -latch.await(); +Assert.assertTrue(timing.awaitLatch(latch)); if ( zkServer != null ) { @@ -139,7 +143,7 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace } else { -throw new Exception("No zkServer. zkServer is volatile: " + Modifier.isVolatile(cnxnFactory.getClass().getDeclaredField("zkServer").getModifiers())); +throw new Exception("No zkServer."); } Exception exception = startingException.get(); @@ -201,9 +205,17 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace txnLog = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir()); zkServer = new TestZooKeeperServer(txnLog, config); -cnxnFactory = ServerCnxnFactory.createFactory(); -cnxnFactory.configure(config.getClientPortAddress(), -config.getMaxClientCnxns()); +try +{ +cnxnFactory = ServerCnxnFactory.createFactory(); +cnxnFactory.configure(config.getClientPortAddress(), +config.getMaxClie
curator git commit: allow session time to elapse
Repository: curator Updated Branches: refs/heads/CURATOR-261 a9271ff60 -> 763aa39a1 allow session time to elapse Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/763aa39a Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/763aa39a Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/763aa39a Branch: refs/heads/CURATOR-261 Commit: 763aa39a1467a6963447f7a6b0ab5ff956b1d703 Parents: a9271ff Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 23:43:14 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 23:43:14 2015 -0500 -- .../curator/framework/recipes/locks/TestInterProcessMutexBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/763aa39a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java index 0cf8d45..febf499 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java @@ -192,7 +192,7 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests Assert.assertTrue(timing.acquireSemaphore(semaphore, 1)); KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString()); -Assert.assertTrue(timing.acquireSemaphore(semaphore, 1)); + Assert.assertTrue(timing.forSessionSleep().acquireSemaphore(semaphore, 1)); } finally {
[1/2] curator git commit: wip
Repository: curator Updated Branches: refs/heads/CURATOR-261 fad270861 -> f8f13c25d wip Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/0603e958 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/0603e958 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/0603e958 Branch: refs/heads/CURATOR-261 Commit: 0603e95834d8596d2397620cd29235d5a62da700 Parents: fad2708 Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 20:43:37 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 20:43:37 2015 -0500 -- .../curator/framework/imps/TestFramework.java | 4 +-- .../framework/imps/TestReconfiguration.java | 3 +- .../locks/TestInterProcessMutexBase.java| 4 +-- .../curator/test/QuorumConfigBuilder.java | 36 ++-- .../curator/test/TestingZooKeeperMain.java | 18 +- .../curator/test/TestingZooKeeperServer.java| 2 ++ pom.xml | 3 +- 7 files changed, 61 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/0603e958/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 811631c..472e0fc 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 @@ -59,7 +59,7 @@ public class TestFramework extends BaseClassForTests @Override public void setup() throws Exception { -System.setProperty("container.checkIntervalMs", "1000"); +System.setProperty("znode.container.checkIntervalMs", "1000"); super.setup(); } @@ -67,7 +67,7 @@ public class TestFramework extends BaseClassForTests @Override public void teardown() throws Exception { -System.clearProperty("container.checkIntervalMs"); +System.clearProperty("znode.container.checkIntervalMs"); super.teardown(); } http://git-wip-us.apache.org/repos/asf/curator/blob/0603e958/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index 2438ef8..a99617b 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -27,6 +27,7 @@ import org.apache.curator.framework.api.BackgroundCallback; import org.apache.curator.framework.api.CuratorEvent; import org.apache.curator.framework.ensemble.EnsembleTracker; import org.apache.curator.retry.RetryOneTime; +import org.apache.curator.test.BaseClassForTests; import org.apache.curator.test.InstanceSpec; import org.apache.curator.test.TestingCluster; import org.apache.curator.test.Timing; @@ -47,7 +48,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; -public class TestReconfiguration +public class TestReconfiguration extends BaseClassForTests { private static final Timing timing = new Timing(); private TestingCluster cluster; http://git-wip-us.apache.org/repos/asf/curator/blob/0603e958/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java index 5a8168e..0cf8d45 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java @@ -211,7 +211,7 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests server.close(); -System.setProperty("container.checkIntervalMs", "10"); +System.setProperty("znode.container.checkIntervalMs", "10&q
curator git commit: test assumed first latch would be leader. Fixed it so that this is so
Repository: curator Updated Branches: refs/heads/CURATOR-261 f8f13c25d -> a9271ff60 test assumed first latch would be leader. Fixed it so that this is so Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/a9271ff6 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/a9271ff6 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/a9271ff6 Branch: refs/heads/CURATOR-261 Commit: a9271ff6033557efeda01e4c543d0f840519e64d Parents: f8f13c2 Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 22:37:57 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 22:37:57 2015 -0500 -- .../apache/curator/framework/recipes/leader/TestLeaderLatch.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/a9271ff6/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 ecaa433..991e6fc 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 @@ -403,10 +403,9 @@ public class TestLeaderLatch extends BaseClassForTests LeaderLatch latch = new LeaderLatch(client, PATH_NAME); latch.start(); latches.add(latch); +waitForALeader(latches, timing); } -waitForALeader(latches, timing); - //we need to close a Participant that doesn't be actual leader (first Participant) nor the last latches.get(PARTICIPANT_ID).close();
curator git commit: Added an option for session expiration management to be a fraction of the negotiated session timeout. This is meant to account for timing/network differences between the client and
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 24de710d2 -> 64bb8841a Added an option for session expiration management to be a fraction of the negotiated session timeout. This is meant to account for timing/network differences between the client and server. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/64bb8841 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/64bb8841 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/64bb8841 Branch: refs/heads/CURATOR-3.0 Commit: 64bb8841a39e1d82de091d23d1865ba47236e5ad Parents: 24de710 Author: randgalt <randg...@apache.org> Authored: Mon Sep 14 21:04:36 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Mon Sep 14 21:04:36 2015 -0500 -- .../ClassicConnectionHandlingPolicy.java| 4 +-- .../connection/ConnectionHandlingPolicy.java| 27 +--- .../StandardConnectionHandlingPolicy.java | 17 ++-- .../imps/ClassicInternalConnectionHandler.java | 6 - .../framework/imps/CuratorFrameworkImpl.java| 7 ++--- .../imps/InternalConnectionHandler.java | 2 -- .../imps/StandardInternalConnectionHandler.java | 6 - .../framework/state/ConnectionStateManager.java | 13 +- 8 files changed, 52 insertions(+), 30 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/64bb8841/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java index e4c59f4..8116308 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/ClassicConnectionHandlingPolicy.java @@ -28,9 +28,9 @@ import java.util.concurrent.Callable; public class ClassicConnectionHandlingPolicy implements ConnectionHandlingPolicy { @Override -public boolean isEmulatingClassicHandling() +public int getSimulatedSessionExpirationPercent() { -return true; +return 0; } @Override http://git-wip-us.apache.org/repos/asf/curator/blob/64bb8841/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java -- diff --git a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java index ae77861..c47577d 100644 --- a/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java +++ b/curator-client/src/main/java/org/apache/curator/connection/ConnectionHandlingPolicy.java @@ -29,11 +29,32 @@ import java.util.concurrent.Callable; public interface ConnectionHandlingPolicy { /** - * Return true if this policy should behave like the pre-3.0.0 version of Curator + * + * Prior to 3.0.0, Curator did not try to manage session expiration + * other than the functionality provided by ZooKeeper itself. Starting with + * 3.0.0, Curator has the option of attempting to monitor session expiration + * above what is provided by ZooKeeper. The percentage returned by this method + * determines how and if Curator will check for session expiration. + * * - * @return true/false + * + * If this method returns 0, Curator does not + * do any additional checking for session expiration. + * + * + * + * If a positive number is returned, Curator will check for session expiration + * as follows: when ZooKeeper sends a Disconnect event, Curator will start a timer. + * If re-connection is not achieved before the elapsed time exceeds the negotiated + * session time multiplied by the session expiration percent, Curator will simulate + * a session expiration. Due to timing/network issues, it is not possible for + * a client to match the server's session timeout with complete accuracy. Thus, the need + * for a session expiration percentage. + * + * + * @return a percentage from 0 to 100 (0 implied no extra session checking) */ -boolean isEmulatingClassicHandling(); +int getSimulatedSessionExpirationPercent(); /** * Called by {@link RetryLoop#callWithRetry(CuratorZookeeperClient, Callable)} to do the work http://git-wip-us.apache.org/repos/asf/curator/blob/64bb8841/curator-client/src/main/
curator git commit: Removed usages of Javaassist and simplified how the in-memory ZK server is created and run. Tests should be more reliable and the testing server should respond faster
Repository: curator Updated Branches: refs/heads/CURATOR-261 [created] 911f02b15 Removed usages of Javaassist and simplified how the in-memory ZK server is created and run. Tests should be more reliable and the testing server should respond faster Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/911f02b1 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/911f02b1 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/911f02b1 Branch: refs/heads/CURATOR-261 Commit: 911f02b15e9585e893bdd2c9bdd860fbf85bba4c Parents: 73b05aa Author: randgalt <randg...@apache.org> Authored: Wed Sep 9 20:13:56 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Wed Sep 9 20:13:56 2015 -0500 -- .../recipes/leader/ChaosMonkeyCnxnFactory.java | 22 ++- curator-test/pom.xml| 5 - .../apache/curator/test/BaseClassForTests.java | 1 + .../apache/curator/test/ByteCodeRewrite.java| 131 .../org/apache/curator/test/TestingCluster.java | 5 - .../org/apache/curator/test/TestingServer.java | 5 - .../curator/test/TestingZooKeeperMain.java | 197 +++ pom.xml | 7 - 8 files changed, 134 insertions(+), 239 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/911f02b1/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java index 1c0e50e..3aeec81 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java @@ -19,6 +19,7 @@ package org.apache.curator.framework.recipes.leader; +import org.apache.curator.test.TestingZooKeeperMain; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.server.ByteBufferInputStream; @@ -49,23 +50,20 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory /* How long after the first error, connections are rejected */ public static final long LOCKOUT_DURATION_MS = 6000; -public ChaosMonkeyCnxnFactory() throws IOException -{ -} - @Override public void startup(ZooKeeperServer zks) throws IOException, InterruptedException { super.startup(new ChaosMonkeyZookeeperServer(zks)); } - public static class ChaosMonkeyZookeeperServer extends ZooKeeperServer { +private final ZooKeeperServer zks; private long firstError = 0; public ChaosMonkeyZookeeperServer(ZooKeeperServer zks) { +this.zks = zks; setTxnLogFactory(zks.getTxnLogFactory()); setTickTime(zks.getTickTime()); setMinSessionTimeout(zks.getMinSessionTimeout()); @@ -73,6 +71,20 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory } @Override +public void startup() +{ +super.startup(); +if ( zks instanceof TestingZooKeeperMain.TestZooKeeperServer ) +{ +((TestingZooKeeperMain.TestZooKeeperServer)zks).noteStartup(); +} +else +{ +throw new RuntimeException("Unknown ZooKeeperServer: " + zks.getClass()); +} +} + +@Override public void submitRequest(Request si) { long remaining = firstError != 0 ? LOCKOUT_DURATION_MS - (System.currentTimeMillis() - firstError) : 0; http://git-wip-us.apache.org/repos/asf/curator/blob/911f02b1/curator-test/pom.xml -- diff --git a/curator-test/pom.xml b/curator-test/pom.xml index 80eedb2..d401da5 100644 --- a/curator-test/pom.xml +++ b/curator-test/pom.xml @@ -41,11 +41,6 @@ -org.javassist -javassist - - - org.apache.commons commons-math http://git-wip-us.apache.org/repos/asf/curator/blob/911f02b1/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java b/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java index 1f6503d..a3bc2b5 100644 --- a/curator-test/sr
curator git commit: The tests assume sequential server IDs starting at 1. Ensure this.
Repository: curator Updated Branches: refs/heads/CURATOR-261 763aa39a1 -> f6734ca18 The tests assume sequential server IDs starting at 1. Ensure this. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/f6734ca1 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/f6734ca1 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/f6734ca1 Branch: refs/heads/CURATOR-261 Commit: f6734ca182bf2269fee6a639107872996ab3ab5c Parents: 763aa39 Author: randgalt <randg...@apache.org> Authored: Fri Sep 11 12:06:18 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 11 12:06:18 2015 -0500 -- .../framework/imps/TestReconfiguration.java| 17 - 1 file changed, 12 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/f6734ca1/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index a99617b..2a616de 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -19,6 +19,7 @@ package org.apache.curator.framework.imps; +import com.google.common.collect.ImmutableList; import org.apache.curator.ensemble.EnsembleListener; import org.apache.curator.ensemble.dynamic.DynamicEnsembleProvider; import org.apache.curator.framework.CuratorFramework; @@ -64,7 +65,13 @@ public class TestReconfiguration extends BaseClassForTests @BeforeMethod public void setup() throws Exception { -cluster = new TestingCluster(5); +ImmutableList.Builder builder = ImmutableList.builder(); +for ( int i = 1; i <= 5; ++i ) +{ +builder.add(new InstanceSpec(null, -1, -1, -1, true, i, -1, -1)); +} + +cluster = new TestingCluster(builder.build()); cluster.start(); connectionString1to5 = cluster.getConnectString(); @@ -333,14 +340,14 @@ public class TestReconfiguration extends BaseClassForTests Assert.assertEquals(qv.getAllMembers().size(), 5); } -static QuorumVerifier getQuorumVerifier(byte[] bytes) throws Exception +private static QuorumVerifier getQuorumVerifier(byte[] bytes) throws Exception { Properties properties = new Properties(); properties.load(new StringReader(new String(bytes))); return new QuorumMaj(properties); } -static InstanceSpec getInstance(TestingCluster cluster, int id) +private static InstanceSpec getInstance(TestingCluster cluster, int id) { for ( InstanceSpec spec : cluster.getInstances() ) { @@ -352,7 +359,7 @@ public class TestReconfiguration extends BaseClassForTests throw new IllegalStateException("InstanceSpec with id:" + id + " not found"); } -static String getServerString(QuorumVerifier qv, TestingCluster cluster, long id) throws Exception +private static String getServerString(QuorumVerifier qv, TestingCluster cluster, long id) throws Exception { String str = qv.getAllMembers().get(id).toString(); //check if connection string is already there. @@ -366,7 +373,7 @@ public class TestReconfiguration extends BaseClassForTests } } -static String getConnectionString(TestingCluster cluster, long... ids) throws Exception +private static String getConnectionString(TestingCluster cluster, long... ids) throws Exception { StringBuilder sb = new StringBuilder(); Map<Long, InstanceSpec> specs = new HashMap<>();
curator git commit: enforce single thread
Repository: curator Updated Branches: refs/heads/CURATOR-265 a8ec17ece -> 2ff89856d enforce single thread Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/2ff89856 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/2ff89856 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/2ff89856 Branch: refs/heads/CURATOR-265 Commit: 2ff89856d6ee2d373cf91bc1a3bd388890604373 Parents: a8ec17e Author: randgalt <randg...@apache.org> Authored: Fri Sep 25 23:59:47 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 25 23:59:47 2015 -0500 -- pom.xml | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/2ff89856/pom.xml -- diff --git a/pom.xml b/pom.xml index 0e7a732..0047d96 100644 --- a/pom.xml +++ b/pom.xml @@ -622,6 +622,7 @@ org.apache.maven.plugins maven-surefire-plugin +1 false true
[1/2] curator git commit: fixed compile error
Repository: curator Updated Branches: refs/heads/CURATOR-266 6e56e8ae9 -> cb34e6f6a fixed compile error Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/2827ba81 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/2827ba81 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/2827ba81 Branch: refs/heads/CURATOR-266 Commit: 2827ba81b5c66ec4f864cc2e3582776e955158ac Parents: 6e56e8a Author: randgalt <randg...@apache.org> Authored: Sat Sep 26 18:07:10 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Sep 26 18:07:10 2015 -0500 -- .../org/apache/curator/framework/imps/TestReconfiguration.java | 6 ++ 1 file changed, 6 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/2827ba81/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index 101360a..0ec796b 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -322,6 +322,12 @@ public class TestReconfiguration extends BaseClassForTests } @Override +public boolean updateServerListEnabled() +{ +return false; +} + +@Override public String getConnectionString() { return connectString.get();
curator git commit: Support getting at the cached config from the ensemble tracker
Repository: curator Updated Branches: refs/heads/CURATOR-266 cb34e6f6a -> b89091e93 Support getting at the cached config from the ensemble tracker Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/b89091e9 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/b89091e9 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/b89091e9 Branch: refs/heads/CURATOR-266 Commit: b89091e9363e760aa34028bcfb57baf6ca921957 Parents: cb34e6f Author: randgalt <randg...@apache.org> Authored: Fri Oct 2 09:37:13 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Oct 2 09:37:13 2015 -0500 -- .../curator/framework/CuratorFramework.java | 8 ++ .../framework/imps/CuratorFrameworkImpl.java| 7 + .../curator/framework/imps/EnsembleTracker.java | 27 +++- .../curator/framework/imps/NamespaceFacade.java | 7 + .../framework/imps/WatcherRemovalFacade.java| 7 + .../framework/imps/TestReconfiguration.java | 12 - 6 files changed, 55 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/b89091e9/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 3d197a0..29c5f06 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 @@ -31,6 +31,7 @@ import org.apache.curator.framework.state.ConnectionStateListener; import org.apache.curator.framework.state.ConnectionStateErrorPolicy; import org.apache.curator.utils.EnsurePath; import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import java.io.Closeable; import java.util.concurrent.TimeUnit; @@ -305,4 +306,11 @@ public interface CuratorFramework extends Closeable * @return error policy */ public ConnectionStateErrorPolicy getConnectionStateErrorPolicy(); + +/** + * Current maintains a cached view of the Zookeeper quorum config. + * + * @return the current config + */ +public QuorumVerifier getCurrentConfig(); } http://git-wip-us.apache.org/repos/asf/curator/blob/b89091e9/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 c3215ad..db18594 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 @@ -50,6 +50,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Arrays; @@ -171,6 +172,12 @@ public class CuratorFrameworkImpl implements CuratorFramework return new WatcherRemovalFacade(this); } +@Override +public QuorumVerifier getCurrentConfig() +{ +return (ensembleTracker != null) ? ensembleTracker.getCurrentConfig() : null; +} + private ZookeeperFactory makeZookeeperFactory(final ZookeeperFactory actualZookeeperFactory) { return new ZookeeperFactory() http://git-wip-us.apache.org/repos/asf/curator/blob/b89091e9/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java b/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java index acd01ee..a46fed1 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java @@ -21,6 +21,7 @@ package org.apache.curator.framework.imps; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; import org.apache.curator.ensemble.EnsembleProvider; import org.apache.curator.framework.C
curator git commit: Support delete().quietly() and create().orSetData()
Repository: curator Updated Branches: refs/heads/CURATOR-268 [created] add56dc92 Support delete().quietly() and create().orSetData() Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/add56dc9 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/add56dc9 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/add56dc9 Branch: refs/heads/CURATOR-268 Commit: add56dc92e4564ce5d146a8f3c0a8ae51c6dc46f Parents: afa8f7a Author: randgalt <randg...@apache.org> Authored: Fri Oct 2 17:22:48 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Oct 2 17:22:48 2015 -0500 -- .../curator/framework/api/CreateBuilder.java| 65 +-- .../framework/api/CreateBuilderMain.java| 86 .../curator/framework/api/DeleteBuilder.java| 4 +- .../framework/api/DeleteBuilderMain.java| 23 ++ .../framework/imps/CreateBuilderImpl.java | 61 ++ .../framework/imps/DeleteBuilderImpl.java | 21 + .../curator/framework/imps/TestFramework.java | 75 + 7 files changed, 271 insertions(+), 64 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java index fa1a68e..564d11b 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilder.java @@ -18,69 +18,8 @@ */ package org.apache.curator.framework.api; -import org.apache.zookeeper.CreateMode; - public interface CreateBuilder extends -BackgroundPathAndBytesable, -CreateModable<ACLBackgroundPathAndBytesable>, -ACLCreateModeBackgroundPathAndBytesable, -Compressible, -Statable<CreateProtectACLCreateModePathAndBytesable> +CreateBuilderMain { -/** - * Causes any parent nodes to get created if they haven't already been - * - * @return this - */ -public ProtectACLCreateModeStatPathAndBytesable creatingParentsIfNeeded(); - -/** - * Causes any parent nodes to get created using {@link CreateMode#CONTAINER} if they haven't already been. - * IMPORTANT NOTE: container creation is a new feature in recent versions of ZooKeeper. - * If the ZooKeeper version you're using does not support containers, the parent nodes - * are created as ordinary PERSISTENT nodes. - * - * @return this - */ -public ProtectACLCreateModeStatPathAndBytesable creatingParentContainersIfNeeded(); - -/** - * @deprecated this has been generalized to support all create modes. Instead, use: - * - * client.create().withProtection().withMode(CreateMode.PERSISTENT_SEQUENTIAL)... - * - * @return this - */ -@Deprecated -public ACLPathAndBytesable withProtectedEphemeralSequential(); - -/** - * - * Hat-tip to https://github.com/sbridges for pointing this out - * - * - * - * It turns out there is an edge case that exists when creating sequential-ephemeral - * nodes. The creation can succeed on the server, but the server can crash before - * the created node name is returned to the client. However, the ZK session is still - * valid so the ephemeral node is not deleted. Thus, there is no way for the client to - * determine what node was created for them. - * - * - * - * Even without sequential-ephemeral, however, the create can succeed on the sever - * but the client (for various reasons) will not know it. - * - * - * - * Putting the create builder into protection mode works around this. - * The name of the node that is created is prefixed with a GUID. If node creation fails - * the normal retry mechanism will occur. On the retry, the parent path is first searched - * for a node that has the GUID in it. If that node is found, it is assumed to be the lost - * node that was successfully created on the first try and is returned to the caller. - * - * - * @return this - */ -public ACLCreateModeStatBackgroundPathAndBytesable withProtection(); +CreateBuilderMain orSetData(); } http://git-wip-us.apache.org/repos/asf/curator/blob/add56dc9/curator-framework/src/main/java/org/apache/curator/framework/api/CreateBuilderMain.java -
curator git commit: 1. EnsembleTracker should always be on, it now is 2. Removed DynamicEnsembleProvider. This should not be optional. EnsembleTracker now always publishes config changes which will en
Repository: curator Updated Branches: refs/heads/CURATOR-266 [created] 26364c618 1. EnsembleTracker should always be on, it now is 2. Removed DynamicEnsembleProvider. This should not be optional. EnsembleTracker now always publishes config changes which will end up calling ZooKeeper.updateServerList() 3. Testing Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/26364c61 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/26364c61 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/26364c61 Branch: refs/heads/CURATOR-266 Commit: 26364c6186fc7c09a9462557b1ca791e9aa70006 Parents: a7076bc Author: randgalt <randg...@apache.org> Authored: Sat Sep 26 13:13:02 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Sep 26 13:13:02 2015 -0500 -- .../org/apache/curator/ConnectionState.java | 32 +++- .../java/org/apache/curator/HandleHolder.java | 4 +- .../ClassicConnectionHandlingPolicy.java| 4 +- .../connection/ConnectionHandlingPolicy.java| 6 +- .../StandardConnectionHandlingPolicy.java | 4 +- .../curator/ensemble/EnsembleListener.java | 24 --- .../curator/ensemble/EnsembleProvider.java | 2 + .../dynamic/DynamicEnsembleProvider.java| 61 -- .../exhibitor/ExhibitorEnsembleProvider.java| 6 + .../ensemble/fixed/FixedEnsembleProvider.java | 13 +- .../framework/ensemble/EnsembleTracker.java | 191 --- .../framework/imps/CuratorFrameworkImpl.java| 7 + .../curator/framework/imps/EnsembleTracker.java | 150 +++ .../framework/imps/TestReconfiguration.java | 57 +- .../locks/TestInterProcessSemaphoreCluster.java | 5 + 15 files changed, 263 insertions(+), 303 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/26364c61/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 eea2ce0..4c1e6ad 100644 --- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java +++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java @@ -199,12 +199,14 @@ class ConnectionState implements Watcher, Closeable private synchronized void checkTimeouts() throws Exception { -Callable hasNewConnectionString = new Callable() +final AtomicReference newConnectionString = new AtomicReference<>(); +Callable hasNewConnectionString = new Callable() { @Override -public Boolean call() +public String call() { -return zooKeeper.hasNewConnectionString(); +newConnectionString.set(zooKeeper.getNewConnectionString()); +return newConnectionString.get(); } }; int lastNegotiatedSessionTimeoutMs = getLastNegotiatedSessionTimeoutMs(); @@ -220,7 +222,7 @@ class ConnectionState implements Watcher, Closeable case NEW_CONNECTION_STRING: { -handleNewConnectionString(); +handleNewConnectionString(newConnectionString.get()); break; } @@ -298,22 +300,34 @@ class ConnectionState implements Watcher, Closeable } } -if ( checkNewConnectionString && zooKeeper.hasNewConnectionString() ) +if ( checkNewConnectionString ) { -handleNewConnectionString(); +String newConnectionString = zooKeeper.getNewConnectionString(); +if ( newConnectionString != null ) +{ +handleNewConnectionString(newConnectionString); +} } return isConnected; } -private void handleNewConnectionString() +private void handleNewConnectionString(String newConnectionString) { -log.info("Connection string changed"); +log.info("Connection string changed to: " + newConnectionString); tracer.get().addCount("connection-string-changed", 1); try { -reset(); +ZooKeeper zooKeeper = this.zooKeeper.getZooKeeper(); +if ( zooKeeper == null ) +{ +log.warn("Could not update the connection string because getZooKeeper() returned null."); +} +else +{ +zooKeeper.updateServerList(newConnectionString); +} } catch ( Exception e ) { http://git-wip-us.apache.org/repos/asf/curator/blob/26364c61
curator git commit: better test - shouldn't fail if slow
Repository: curator Updated Branches: refs/heads/CURATOR-265 c1084663a -> a7076bc8d better test - shouldn't fail if slow Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/a7076bc8 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/a7076bc8 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/a7076bc8 Branch: refs/heads/CURATOR-265 Commit: a7076bc8dd806bb824e33dcd2f8b5e53626b5691 Parents: c108466 Author: randgalt <randg...@apache.org> Authored: Sat Sep 26 10:03:20 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Sep 26 10:03:20 2015 -0500 -- .../framework/imps/TestFrameworkBackground.java| 17 - 1 file changed, 8 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/a7076bc8/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java index 26cc941..6575018 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkBackground.java @@ -20,6 +20,7 @@ package org.apache.curator.framework.imps; import com.google.common.collect.Lists; +import com.google.common.collect.Queues; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.framework.api.BackgroundCallback; @@ -30,15 +31,15 @@ import org.apache.curator.framework.state.ConnectionStateListener; import org.apache.curator.retry.RetryNTimes; import org.apache.curator.retry.RetryOneTime; import org.apache.curator.test.BaseClassForTests; -import org.apache.curator.test.TestingServer; import org.apache.curator.test.Timing; import org.apache.curator.utils.CloseableUtils; import org.apache.zookeeper.KeeperException.Code; import org.testng.Assert; import org.testng.annotations.Test; -import java.util.Arrays; import java.util.List; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -149,24 +150,22 @@ public class TestFrameworkBackground extends BaseClassForTests { client.start(); -final CountDownLatch latch = new CountDownLatch(3); -final List paths = Lists.newArrayList(); -BackgroundCallback callback = new BackgroundCallback() +final BlockingQueue paths = Queues.newLinkedBlockingQueue(); +BackgroundCallback callback = new BackgroundCallback() { @Override public void processResult(CuratorFramework client, CuratorEvent event) throws Exception { paths.add(event.getPath()); -latch.countDown(); } }; client.create().inBackground(callback).forPath("/one"); client.create().inBackground(callback).forPath("/one/two"); client.create().inBackground(callback).forPath("/one/two/three"); -latch.await(); - -Assert.assertEquals(paths, Arrays.asList("/one", "/one/two", "/one/two/three")); +Assert.assertEquals(paths.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), "/one"); +Assert.assertEquals(paths.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), "/one/two"); +Assert.assertEquals(paths.poll(timing.milliseconds(), TimeUnit.MILLISECONDS), "/one/two/three"); } finally {
curator git commit: First pass implementation of group membership
Repository: curator Updated Branches: refs/heads/CURATOR-267 [created] 3029856c8 First pass implementation of group membership Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/3029856c Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/3029856c Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/3029856c Branch: refs/heads/CURATOR-267 Commit: 3029856c894a72ed2c5762e0c6acd2c0a8cd3937 Parents: f8f05be Author: randgalt <randg...@apache.org> Authored: Sun Sep 27 16:59:48 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 27 16:59:48 2015 -0500 -- .../framework/CuratorFrameworkFactory.java | 31 ++-- .../framework/recipes/nodes/GroupMember.java| 140 +++ .../src/site/confluence/group-member.confluence | 42 ++ .../src/site/confluence/index.confluence| 3 +- .../recipes/nodes/TestGroupMember.java | 79 +++ 5 files changed, 281 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/3029856c/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..41ff9cd 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 @@ -100,6 +100,24 @@ public class CuratorFrameworkFactory build(); } +/** + * Return the local address as bytes that can be used as a node payload + * + * @return local address bytes + */ +public static byte[] getLocalAddress() +{ +try +{ +return InetAddress.getLocalHost().getHostAddress().getBytes(); +} +catch ( UnknownHostException ignore ) +{ +// ignore +} +return new byte[0]; +} + public static class Builder { private EnsembleProvider ensembleProvider; @@ -465,19 +483,6 @@ public class CuratorFrameworkFactory } } -private static byte[] getLocalAddress() -{ -try -{ -return InetAddress.getLocalHost().getHostAddress().getBytes(); -} -catch ( UnknownHostException ignore ) -{ -// ignore -} -return new byte[0]; -} - private CuratorFrameworkFactory() { } http://git-wip-us.apache.org/repos/asf/curator/blob/3029856c/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java -- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java new file mode 100644 index 000..5aa8ca2 --- /dev/null +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.curator.framework.recipes.nodes; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.utils.CloseableUtils; +import org.apache.curator.utils.ZKPaths; +import java.io.Closeable; +import java.util.Arrays; +import java.util.Map; + +/** + * Group membership management. Adds this instance into a group
curator git commit: updated doc
Repository: curator Updated Branches: refs/heads/CURATOR-267 a49d2bb4f -> 275b1d5e9 updated doc Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/275b1d5e Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/275b1d5e Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/275b1d5e Branch: refs/heads/CURATOR-267 Commit: 275b1d5e9f13c8d4dd76961a2fc6f2c86536cf9b Parents: a49d2bb Author: randgalt <randg...@apache.org> Authored: Sun Sep 27 17:22:15 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 27 17:22:15 2015 -0500 -- curator-recipes/src/site/confluence/group-member.confluence | 2 ++ curator-recipes/src/site/confluence/index.confluence| 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/275b1d5e/curator-recipes/src/site/confluence/group-member.confluence -- diff --git a/curator-recipes/src/site/confluence/group-member.confluence b/curator-recipes/src/site/confluence/group-member.confluence index dcb27b3..a370675 100644 --- a/curator-recipes/src/site/confluence/group-member.confluence +++ b/curator-recipes/src/site/confluence/group-member.confluence @@ -5,6 +5,8 @@ Group membership management. Adds this instance into a group and keeps a cache o h2. Participating Classes * GroupMember +* PersistentEphemeralNode +* PathChildrenCache h2. Usage h3. Creating a GroupMember http://git-wip-us.apache.org/repos/asf/curator/blob/275b1d5e/curator-recipes/src/site/confluence/index.confluence -- diff --git a/curator-recipes/src/site/confluence/index.confluence b/curator-recipes/src/site/confluence/index.confluence index 71de8df..01dfc7e 100644 --- a/curator-recipes/src/site/confluence/index.confluence +++ b/curator-recipes/src/site/confluence/index.confluence @@ -30,7 +30,7 @@ regarding "Curator Recipes Own Their ZNode/Paths". ||Nodes|| |[[Persistent Ephemeral Node|persistent-ephemeral-node.html]] \- An ephemeral node that attempts to stay present in ZooKeeper, even through connection and session interruptions.| -|[Group Member|group-member-node.html]] \- Group membership management. Adds this instance into a group and keeps a cache of members in the group.| +|[Group Member|group-member.html]] \- Group membership management. Adds this instance into a group and keeps a cache of members in the group.| ||Queues|| |[[Distributed Queue|distributed-queue.html]] \- An implementation of the Distributed Queue ZK recipe. Items put into the queue are guaranteed to be ordered (by means of ZK's PERSISTENT_SEQUENTIAL node). If a single consumer takes items out of the queue, they will be ordered FIFO. If ordering is important, use a LeaderSelector to nominate a single consumer.|
curator git commit: api for setting data
Repository: curator Updated Branches: refs/heads/CURATOR-267 3029856c8 -> a49d2bb4f api for setting data Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/a49d2bb4 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/a49d2bb4 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/a49d2bb4 Branch: refs/heads/CURATOR-267 Commit: a49d2bb4f83a9627ad546a149a8f77831d174ef1 Parents: 3029856 Author: randgalt <randg...@apache.org> Authored: Sun Sep 27 17:06:46 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 27 17:06:46 2015 -0500 -- .../framework/recipes/nodes/GroupMember.java| 22 .../recipes/nodes/PersistentEphemeralNode.java | 7 ++- .../recipes/nodes/TestGroupMember.java | 7 +++ 3 files changed, 31 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/a49d2bb4/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java -- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java index 5aa8ca2..b914ba4 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/GroupMember.java @@ -28,7 +28,6 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.utils.CloseableUtils; import org.apache.curator.utils.ZKPaths; import java.io.Closeable; -import java.util.Arrays; import java.util.Map; /** @@ -40,7 +39,6 @@ public class GroupMember implements Closeable private final PersistentEphemeralNode pen; private final PathChildrenCache cache; private final String thisId; -private final byte[] payload; /** * @param client client @@ -61,7 +59,6 @@ public class GroupMember implements Closeable public GroupMember(CuratorFramework client, String membershipPath, String thisId, byte[] payload) { this.thisId = Preconditions.checkNotNull(thisId, "thisId cannot be null"); -this.payload = Arrays.copyOf(payload, payload.length); cache = newPathChildrenCache(client, membershipPath); pen = newPersistentEphemeralNode(client, membershipPath, thisId, payload); @@ -85,6 +82,23 @@ public class GroupMember implements Closeable } /** + * Change the data stored in this instance's node + * + * @param data new data (cannot be null) + */ +public void setThisData(byte[] data) +{ +try +{ +pen.setData(data); +} +catch ( Exception e ) +{ +Throwables.propagate(e); +} +} + +/** * Have thisId leave the group and stop caching membership */ @Override @@ -112,7 +126,7 @@ public class GroupMember implements Closeable } if ( !thisIdAdded ) { -builder.put(thisId, payload); // this instance is always a member +builder.put(thisId, pen.getData()); // this instance is always a member } return builder.build(); } http://git-wip-us.apache.org/repos/asf/curator/blob/a49d2bb4/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 7e00e10..ff9cb12 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 @@ -352,7 +352,12 @@ public class PersistentEphemeralNode implements Closeable } } -private byte[] getData() { +/** + * Return the current value of our data + * + * @return our data + */ +public byte[] getData() { return this.data.get(); } http://git-wip-us.apache.org/repos/asf/curator/blob/a49d2bb4/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestGroupMember.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestGroupMember.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/nodes/TestGroupMember.java inde
[2/2] curator git commit: continued work on tests, etc.
continued work on tests, etc. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/cb34e6f6 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/cb34e6f6 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/cb34e6f6 Branch: refs/heads/CURATOR-266 Commit: cb34e6f6a41b08c9d4e6179d9f893b0e48e7860c Parents: 2827ba8 Author: randgalt <randg...@apache.org> Authored: Sun Sep 27 13:31:32 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sun Sep 27 13:31:32 2015 -0500 -- .../ensemble/fixed/FixedEnsembleProvider.java | 16 ++- .../framework/imps/CuratorFrameworkImpl.java| 5 +++ .../curator/framework/imps/EnsembleTracker.java | 45 .../src/site/confluence/index.confluence| 1 + .../framework/imps/TestFrameworkBackground.java | 9 ++-- .../framework/imps/TestReconfiguration.java | 9 +++- .../recipes/nodes/PersistentEphemeralNode.java | 27 ++-- .../curator/framework/imps/TestCleanState.java | 9 .../locks/TestInterProcessSemaphoreCluster.java | 6 +++ .../nodes/TestPersistentEphemeralNode.java | 16 --- src/site/confluence/utilities.confluence| 7 +-- 11 files changed, 104 insertions(+), 46 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/cb34e6f6/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java -- diff --git a/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java b/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java index 28ad1b6..5f486f4 100644 --- a/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java +++ b/curator-client/src/main/java/org/apache/curator/ensemble/fixed/FixedEnsembleProvider.java @@ -20,6 +20,7 @@ package org.apache.curator.ensemble.fixed; import com.google.common.base.Preconditions; import org.apache.curator.ensemble.EnsembleProvider; +import org.apache.zookeeper.ZooKeeper; import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; @@ -29,6 +30,7 @@ import java.util.concurrent.atomic.AtomicReference; public class FixedEnsembleProvider implements EnsembleProvider { private final AtomicReference connectionString = new AtomicReference<>(); +private final boolean updateServerListEnabled; /** * The connection string to use @@ -37,6 +39,18 @@ public class FixedEnsembleProvider implements EnsembleProvider */ public FixedEnsembleProvider(String connectionString) { +this(connectionString, true); +} + +/** + * The connection string to use + * + * @param connectionString connection string + * @param updateServerListEnabled if true, allow Curator to call {@link ZooKeeper#updateServerList(String)} + */ +public FixedEnsembleProvider(String connectionString, boolean updateServerListEnabled) +{ +this.updateServerListEnabled = updateServerListEnabled; this.connectionString.set(Preconditions.checkNotNull(connectionString, "connectionString cannot be null")); } @@ -67,6 +81,6 @@ public class FixedEnsembleProvider implements EnsembleProvider @Override public boolean updateServerListEnabled() { -return true; +return updateServerListEnabled; } } http://git-wip-us.apache.org/repos/asf/curator/blob/cb34e6f6/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 f2f578c..c3215ad 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 @@ -770,6 +770,11 @@ public class CuratorFrameworkImpl implements CuratorFramework connectionStateManager.addStateChange(newConnectionState); } +EnsembleTracker getEnsembleTracker() +{ +return ensembleTracker; +} + @SuppressWarnings({"ThrowableResultOfMethodCallIgnored"}) private boolean checkBackgroundRetry(OperationAndData operationAndData, CuratorEvent event) { http://git-wip-us.apache.org/repos/asf/curator/blob/cb34e6f6/curator-framework/src/main/java/org/apache/curator/framework/imps/EnsembleTracker.java -- diff --git a/curator-framework/src/
curator git commit: ExhibitorEnsembleProvider is not compatible with updateServerList()
Repository: curator Updated Branches: refs/heads/CURATOR-266 56d9ba665 -> 6e56e8ae9 ExhibitorEnsembleProvider is not compatible with updateServerList() Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/6e56e8ae Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/6e56e8ae Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/6e56e8ae Branch: refs/heads/CURATOR-266 Commit: 6e56e8ae9f04ffdd76505858dbbe5b1ff04dbd49 Parents: 56d9ba6 Author: randgalt <randg...@apache.org> Authored: Sat Sep 26 18:03:06 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Sep 26 18:03:06 2015 -0500 -- .../org/apache/curator/ConnectionState.java | 9 +- .../curator/ensemble/EnsembleProvider.java | 12 .../exhibitor/ExhibitorEnsembleProvider.java| 30 .../ensemble/fixed/FixedEnsembleProvider.java | 6 4 files changed, 44 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/6e56e8ae/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 4c1e6ad..0b21643 100644 --- a/curator-client/src/main/java/org/apache/curator/ConnectionState.java +++ b/curator-client/src/main/java/org/apache/curator/ConnectionState.java @@ -326,7 +326,14 @@ class ConnectionState implements Watcher, Closeable } else { -zooKeeper.updateServerList(newConnectionString); +if ( ensembleProvider.updateServerListEnabled() ) +{ +zooKeeper.updateServerList(newConnectionString); +} +else +{ +reset(); +} } } catch ( Exception e ) http://git-wip-us.apache.org/repos/asf/curator/blob/6e56e8ae/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java -- diff --git a/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java b/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java index c03726f..4db8348 100644 --- a/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java +++ b/curator-client/src/main/java/org/apache/curator/ensemble/EnsembleProvider.java @@ -52,5 +52,17 @@ public interface EnsembleProvider extends Closeable */ public void close() throws IOException; +/** + * A new connection string event was received + * + * @param connectionString the new connection string + */ public void setConnectionString(String connectionString); + +/** + * Return true if this ensemble provider supports {@link ZooKeeper#updateServerList(String)} + * + * @return true/false + */ +public boolean updateServerListEnabled(); } http://git-wip-us.apache.org/repos/asf/curator/blob/6e56e8ae/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java -- diff --git a/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java b/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java index 4cbf5ee..4573724 100644 --- a/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java +++ b/curator-client/src/main/java/org/apache/curator/ensemble/exhibitor/ExhibitorEnsembleProvider.java @@ -116,19 +116,19 @@ public class ExhibitorEnsembleProvider implements EnsembleProvider Preconditions.checkState(state.compareAndSet(State.LATENT, State.STARTED), "Cannot be started more than once"); service.scheduleWithFixedDelay -( -new Runnable() -{ -@Override -public void run() +( +new Runnable() { -poll(); -} -}, -pollingMs, -pollingMs, -TimeUnit.MILLISECONDS -); +@Override +public void run() +{ +poll(); +} +}, +pollingMs, +pollingMs, +TimeUnit.MILLISECONDS +); } @Override @@ -151,6 +151,12 @@ public class ExhibitorEnsembleProvider imp
curator git commit: License header
Repository: curator Updated Branches: refs/heads/CURATOR-265 2ff89856d -> c1084663a License header Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/c1084663 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/c1084663 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/c1084663 Branch: refs/heads/CURATOR-265 Commit: c1084663aa8991a06cc7390180d07cbaf6c330e1 Parents: 2ff8985 Author: randgalt <randg...@apache.org> Authored: Sat Sep 26 09:15:54 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Sat Sep 26 09:15:54 2015 -0500 -- .../framework/api/BackgroundEnsembleable.java | 18 ++ .../api/WatchBackgroundEnsembleable.java | 18 ++ 2 files changed, 36 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/c1084663/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java index ae2b226..c8b323f 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.framework.api; public interface BackgroundEnsembleable extends http://git-wip-us.apache.org/repos/asf/curator/blob/c1084663/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java index 073cfe3..a0e5a13 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.curator.framework.api; public interface WatchBackgroundEnsembleable extends
[2/2] curator git commit: Use javaassist to make fields we're accessing volatile
Use javaassist to make fields we're accessing volatile Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/73b05aac Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/73b05aac Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/73b05aac Branch: refs/heads/CURATOR-3.0 Commit: 73b05aac94099dd1c1e5684d50a51078ae844e0e Parents: d110cbb Author: randgalt <randg...@apache.org> Authored: Wed Sep 9 13:35:18 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Wed Sep 9 13:35:18 2015 -0500 -- .../apache/curator/test/ByteCodeRewrite.java| 34 1 file changed, 34 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/73b05aac/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java b/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java index eeca3d4..d8a1c3a 100644 --- a/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java +++ b/curator-test/src/main/java/org/apache/curator/test/ByteCodeRewrite.java @@ -22,8 +22,10 @@ package org.apache.curator.test; import javassist.CannotCompileException; import javassist.ClassPool; import javassist.CtClass; +import javassist.CtField; import javassist.CtMethod; import javassist.NotFoundException; +import javassist.bytecode.AccessFlag; public class ByteCodeRewrite { @@ -73,6 +75,26 @@ public class ByteCodeRewrite { // ignore } + +try +{ +CtClass cc = pool.get("org.apache.zookeeper.server.ZooKeeperServerMain"); +makeVolatile(cc); +} +catch ( NotFoundException e ) +{ +// ignore +} + +try +{ +CtClass cc = pool.get("org.apache.zookeeper.server.ServerCnxnFactory"); +makeVolatile(cc); +} +catch ( NotFoundException e ) +{ +// ignore +} } catch ( Exception e ) { @@ -80,6 +102,18 @@ public class ByteCodeRewrite } } +private static void makeVolatile(CtClass cc) throws CannotCompileException +{ +for ( CtField field : cc.getDeclaredFields() ) +{ +if ( (field.getModifiers() & (AccessFlag.ABSTRACT | AccessFlag.NATIVE | AccessFlag.SYNTHETIC | AccessFlag.STATIC | AccessFlag.FINAL)) == 0 ) +{ +field.setModifiers(field.getModifiers() | AccessFlag.VOLATILE); +} +} +cc.toClass(); +} + private static void fixMethods(CtClass cc, String... methodNames) throws CannotCompileException { for ( CtMethod method : cc.getDeclaredMethods() )
curator git commit: for some reason, one of the constructors got messed up
Repository: curator Updated Branches: refs/heads/CURATOR-268 add56dc92 -> 537156db4 for some reason, one of the constructors got messed up Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/537156db Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/537156db Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/537156db Branch: refs/heads/CURATOR-268 Commit: 537156db46becd67542a653d6ab055168ed507d0 Parents: add56dc Author: randgalt <randg...@apache.org> Authored: Mon Oct 5 20:27:46 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Mon Oct 5 20:27:46 2015 -0500 -- .../src/main/java/org/apache/curator/CuratorZookeeperClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/537156db/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java -- diff --git a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java index 471adf0..18d8572 100644 --- a/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java +++ b/curator-client/src/main/java/org/apache/curator/CuratorZookeeperClient.java @@ -93,7 +93,7 @@ public class CuratorZookeeperClient implements Closeable */ public CuratorZookeeperClient(ZookeeperFactory zookeeperFactory, EnsembleProvider ensembleProvider, int sessionTimeoutMs, int connectionTimeoutMs, Watcher watcher, RetryPolicy retryPolicy, boolean canBeReadOnly) { -this(new DefaultZookeeperFactory(), ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, canBeReadOnly, new ClassicConnectionHandlingPolicy()); +this(zookeeperFactory, ensembleProvider, sessionTimeoutMs, connectionTimeoutMs, watcher, retryPolicy, canBeReadOnly, new ClassicConnectionHandlingPolicy()); } /**
[2/2] curator git commit: Merge branch 'master' into CURATOR-3.0
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/ea36769a Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/ea36769a Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/ea36769a Branch: refs/heads/CURATOR-3.0 Commit: ea36769afa77b510e2206d34faed17e6f4a57bd3 Parents: 64bb884 f8f05be Author: randgalt <randg...@apache.org> Authored: Wed Sep 23 08:21:35 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Wed Sep 23 08:21:35 2015 -0500 -- .../framework/imps/CreateBuilderImpl.java | 24 ++-- .../framework/imps/TestFrameworkEdges.java | 41 2 files changed, 53 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/ea36769a/curator-framework/src/main/java/org/apache/curator/framework/imps/CreateBuilderImpl.java -- http://git-wip-us.apache.org/repos/asf/curator/blob/ea36769a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java --
[1/2] curator git commit: CURATOR-45 added findAndDeleteProtectedNodeInBackground to handle cases where a protected node can get lost. However, the code wasn't correctly handling namespaces
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 64bb8841a -> ea36769af CURATOR-45 added findAndDeleteProtectedNodeInBackground to handle cases where a protected node can get lost. However, the code wasn't correctly handling namespaces Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/f8f05be2 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/f8f05be2 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/f8f05be2 Branch: refs/heads/CURATOR-3.0 Commit: f8f05be2e097c4c9be65e5110a376d461fd9cd9a Parents: c8cc3f4 Author: randgalt <randg...@apache.org> Authored: Tue Sep 22 14:59:41 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 22 14:59:41 2015 -0500 -- .../framework/imps/CreateBuilderImpl.java | 24 +-- .../framework/imps/TestFrameworkEdges.java | 42 +++- 2 files changed, 53 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/f8f05be2/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 7a4a96f..b72b7b6 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 @@ -464,13 +464,13 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperationnull to create a new one + * @param unAdjustedPath the path - raw without namespace resolution + * @param protectedIdthe protected id + * @param callback callback to use, null to create a new one */ -private void findAndDeleteProtectedNodeInBackground(String path, String protectedId, FindProtectedNodeCB callback) +private void findAndDeleteProtectedNodeInBackground(String unAdjustedPath, String protectedId, FindProtectedNodeCB callback) { if ( client.getState() == CuratorFrameworkState.STARTED ) { if ( callback == null ) { -callback = new FindProtectedNodeCB(path, protectedId); +callback = new FindProtectedNodeCB(unAdjustedPath, protectedId); } try { - client.getChildren().inBackground(callback).forPath(ZKPaths.getPathAndNode(path).getPath()); + client.getChildren().inBackground(callback).forPath(ZKPaths.getPathAndNode(unAdjustedPath).getPath()); } catch ( Exception e ) { -findAndDeleteProtectedNodeInBackground(path, protectedId, callback); +findAndDeleteProtectedNodeInBackground(unAdjustedPath, protectedId, callback); } } } @@ -830,7 +830,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperationhttp://git-wip-us.apache.org/repos/asf/curator/blob/f8f05be2/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java index cd3ae77..95c3792 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java @@ -29,10 +29,10 @@ import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.api.CuratorListener; import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionStateListener; +import org.apache.curator.retry.RetryNTimes; import org.apache.curator.retry.RetryOneTime; import org.apache.curator.test.BaseClassForTests; import org.apache.curator.test.KillSession; -import org.apache.curator.test.TestingServer; import org.apache.curator.test.Timing; import org.apache.curator.utils.CloseableUtils; import org.apache.curator.utils.ZKPaths; @@ -43,6 +43,7 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; import org.testng.Assert; import org.testng.annotations.Test; +import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; @@ -56,6 +57,45 @@ public class TestFrameworkEdges extends BaseClassForTests private final Timing timi
curator git commit: CURATOR-45 added findAndDeleteProtectedNodeInBackground to handle cases where a protected node can get lost. However, the code wasn't correctly handling namespaces
Repository: curator Updated Branches: refs/heads/CURATOR-264 [created] f8f05be2e CURATOR-45 added findAndDeleteProtectedNodeInBackground to handle cases where a protected node can get lost. However, the code wasn't correctly handling namespaces Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/f8f05be2 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/f8f05be2 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/f8f05be2 Branch: refs/heads/CURATOR-264 Commit: f8f05be2e097c4c9be65e5110a376d461fd9cd9a Parents: c8cc3f4 Author: randgalt <randg...@apache.org> Authored: Tue Sep 22 14:59:41 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Sep 22 14:59:41 2015 -0500 -- .../framework/imps/CreateBuilderImpl.java | 24 +-- .../framework/imps/TestFrameworkEdges.java | 42 +++- 2 files changed, 53 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/f8f05be2/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 7a4a96f..b72b7b6 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 @@ -464,13 +464,13 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperationnull to create a new one + * @param unAdjustedPath the path - raw without namespace resolution + * @param protectedIdthe protected id + * @param callback callback to use, null to create a new one */ -private void findAndDeleteProtectedNodeInBackground(String path, String protectedId, FindProtectedNodeCB callback) +private void findAndDeleteProtectedNodeInBackground(String unAdjustedPath, String protectedId, FindProtectedNodeCB callback) { if ( client.getState() == CuratorFrameworkState.STARTED ) { if ( callback == null ) { -callback = new FindProtectedNodeCB(path, protectedId); +callback = new FindProtectedNodeCB(unAdjustedPath, protectedId); } try { - client.getChildren().inBackground(callback).forPath(ZKPaths.getPathAndNode(path).getPath()); + client.getChildren().inBackground(callback).forPath(ZKPaths.getPathAndNode(unAdjustedPath).getPath()); } catch ( Exception e ) { -findAndDeleteProtectedNodeInBackground(path, protectedId, callback); +findAndDeleteProtectedNodeInBackground(unAdjustedPath, protectedId, callback); } } } @@ -830,7 +830,7 @@ class CreateBuilderImpl implements CreateBuilder, BackgroundOperationhttp://git-wip-us.apache.org/repos/asf/curator/blob/f8f05be2/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java index cd3ae77..95c3792 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFrameworkEdges.java @@ -29,10 +29,10 @@ import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.api.CuratorListener; import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionStateListener; +import org.apache.curator.retry.RetryNTimes; import org.apache.curator.retry.RetryOneTime; import org.apache.curator.test.BaseClassForTests; import org.apache.curator.test.KillSession; -import org.apache.curator.test.TestingServer; import org.apache.curator.test.Timing; import org.apache.curator.utils.CloseableUtils; import org.apache.curator.utils.ZKPaths; @@ -43,6 +43,7 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.data.Stat; import org.testng.Assert; import org.testng.annotations.Test; +import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; @@ -56,6 +57,45 @@ public class TestFrameworkEdges extends BaseClassForTests private final Timing timing
[7/8] curator git commit: The tests assume sequential server IDs starting at 1. Ensure this.
The tests assume sequential server IDs starting at 1. Ensure this. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/f6734ca1 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/f6734ca1 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/f6734ca1 Branch: refs/heads/CURATOR-3.0 Commit: f6734ca182bf2269fee6a639107872996ab3ab5c Parents: 763aa39 Author: randgalt <randg...@apache.org> Authored: Fri Sep 11 12:06:18 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 11 12:06:18 2015 -0500 -- .../framework/imps/TestReconfiguration.java| 17 - 1 file changed, 12 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/f6734ca1/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index a99617b..2a616de 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -19,6 +19,7 @@ package org.apache.curator.framework.imps; +import com.google.common.collect.ImmutableList; import org.apache.curator.ensemble.EnsembleListener; import org.apache.curator.ensemble.dynamic.DynamicEnsembleProvider; import org.apache.curator.framework.CuratorFramework; @@ -64,7 +65,13 @@ public class TestReconfiguration extends BaseClassForTests @BeforeMethod public void setup() throws Exception { -cluster = new TestingCluster(5); +ImmutableList.Builder builder = ImmutableList.builder(); +for ( int i = 1; i <= 5; ++i ) +{ +builder.add(new InstanceSpec(null, -1, -1, -1, true, i, -1, -1)); +} + +cluster = new TestingCluster(builder.build()); cluster.start(); connectionString1to5 = cluster.getConnectString(); @@ -333,14 +340,14 @@ public class TestReconfiguration extends BaseClassForTests Assert.assertEquals(qv.getAllMembers().size(), 5); } -static QuorumVerifier getQuorumVerifier(byte[] bytes) throws Exception +private static QuorumVerifier getQuorumVerifier(byte[] bytes) throws Exception { Properties properties = new Properties(); properties.load(new StringReader(new String(bytes))); return new QuorumMaj(properties); } -static InstanceSpec getInstance(TestingCluster cluster, int id) +private static InstanceSpec getInstance(TestingCluster cluster, int id) { for ( InstanceSpec spec : cluster.getInstances() ) { @@ -352,7 +359,7 @@ public class TestReconfiguration extends BaseClassForTests throw new IllegalStateException("InstanceSpec with id:" + id + " not found"); } -static String getServerString(QuorumVerifier qv, TestingCluster cluster, long id) throws Exception +private static String getServerString(QuorumVerifier qv, TestingCluster cluster, long id) throws Exception { String str = qv.getAllMembers().get(id).toString(); //check if connection string is already there. @@ -366,7 +373,7 @@ public class TestReconfiguration extends BaseClassForTests } } -static String getConnectionString(TestingCluster cluster, long... ids) throws Exception +private static String getConnectionString(TestingCluster cluster, long... ids) throws Exception { StringBuilder sb = new StringBuilder(); Map<Long, InstanceSpec> specs = new HashMap<>();
[6/8] curator git commit: allow session time to elapse
allow session time to elapse Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/763aa39a Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/763aa39a Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/763aa39a Branch: refs/heads/CURATOR-3.0 Commit: 763aa39a1467a6963447f7a6b0ab5ff956b1d703 Parents: a9271ff Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 23:43:14 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 23:43:14 2015 -0500 -- .../curator/framework/recipes/locks/TestInterProcessMutexBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/763aa39a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java index 0cf8d45..febf499 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java @@ -192,7 +192,7 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests Assert.assertTrue(timing.acquireSemaphore(semaphore, 1)); KillSession.kill(client.getZookeeperClient().getZooKeeper(), server.getConnectString()); -Assert.assertTrue(timing.acquireSemaphore(semaphore, 1)); + Assert.assertTrue(timing.forSessionSleep().acquireSemaphore(semaphore, 1)); } finally {
[3/8] curator git commit: wip
wip Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/0603e958 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/0603e958 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/0603e958 Branch: refs/heads/CURATOR-3.0 Commit: 0603e95834d8596d2397620cd29235d5a62da700 Parents: fad2708 Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 20:43:37 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 20:43:37 2015 -0500 -- .../curator/framework/imps/TestFramework.java | 4 +-- .../framework/imps/TestReconfiguration.java | 3 +- .../locks/TestInterProcessMutexBase.java| 4 +-- .../curator/test/QuorumConfigBuilder.java | 36 ++-- .../curator/test/TestingZooKeeperMain.java | 18 +- .../curator/test/TestingZooKeeperServer.java| 2 ++ pom.xml | 3 +- 7 files changed, 61 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/0603e958/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 811631c..472e0fc 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 @@ -59,7 +59,7 @@ public class TestFramework extends BaseClassForTests @Override public void setup() throws Exception { -System.setProperty("container.checkIntervalMs", "1000"); +System.setProperty("znode.container.checkIntervalMs", "1000"); super.setup(); } @@ -67,7 +67,7 @@ public class TestFramework extends BaseClassForTests @Override public void teardown() throws Exception { -System.clearProperty("container.checkIntervalMs"); +System.clearProperty("znode.container.checkIntervalMs"); super.teardown(); } http://git-wip-us.apache.org/repos/asf/curator/blob/0603e958/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index 2438ef8..a99617b 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -27,6 +27,7 @@ import org.apache.curator.framework.api.BackgroundCallback; import org.apache.curator.framework.api.CuratorEvent; import org.apache.curator.framework.ensemble.EnsembleTracker; import org.apache.curator.retry.RetryOneTime; +import org.apache.curator.test.BaseClassForTests; import org.apache.curator.test.InstanceSpec; import org.apache.curator.test.TestingCluster; import org.apache.curator.test.Timing; @@ -47,7 +48,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; -public class TestReconfiguration +public class TestReconfiguration extends BaseClassForTests { private static final Timing timing = new Timing(); private TestingCluster cluster; http://git-wip-us.apache.org/repos/asf/curator/blob/0603e958/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java index 5a8168e..0cf8d45 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/TestInterProcessMutexBase.java @@ -211,7 +211,7 @@ public abstract class TestInterProcessMutexBase extends BaseClassForTests server.close(); -System.setProperty("container.checkIntervalMs", "10"); +System.setProperty("znode.container.checkIntervalMs", "10"); try { server = new TestingServer(); @@ -272,7
[2/8] curator git commit: moved to ZK 3.5.1.
moved to ZK 3.5.1. Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/fad27086 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/fad27086 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/fad27086 Branch: refs/heads/CURATOR-3.0 Commit: fad270861feb4dbda36fdea76b4db9ec29e92001 Parents: 911f02b Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 09:22:11 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 09:22:11 2015 -0500 -- .../recipes/leader/ChaosMonkeyCnxnFactory.java | 2 -- .../curator/test/TestingZooKeeperMain.java | 36 +++- pom.xml | 11 +- 3 files changed, 29 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/fad27086/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java index 3aeec81..4cb342c 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java @@ -31,8 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.SelectionKey; -import java.nio.channels.SocketChannel; /** * A connection factory that will behave like the NIOServerCnxnFactory except that http://git-wip-us.apache.org/repos/asf/curator/blob/fad27086/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java index 702824a..a4cca8e 100644 --- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java +++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java @@ -30,12 +30,14 @@ import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.Assert; import javax.management.JMException; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.nio.channels.ServerSocketChannel; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; public class TestingZooKeeperMain implements ZooKeeperMainFace @@ -48,7 +50,9 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace private volatile ServerCnxnFactory cnxnFactory; private volatile ZooKeeperServer zkServer; -static final int MAX_WAIT_MS = new Timing().milliseconds(); +private static final Timing timing = new Timing(); + +static final int MAX_WAIT_MS = timing.milliseconds(); @Override public void kill() @@ -124,7 +128,7 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace @Override public void blockUntilStarted() throws Exception { -latch.await(); +Assert.assertTrue(timing.awaitLatch(latch)); if ( zkServer != null ) { @@ -139,7 +143,7 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace } else { -throw new Exception("No zkServer. zkServer is volatile: " + Modifier.isVolatile(cnxnFactory.getClass().getDeclaredField("zkServer").getModifiers())); +throw new Exception("No zkServer."); } Exception exception = startingException.get(); @@ -201,9 +205,17 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace txnLog = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir()); zkServer = new TestZooKeeperServer(txnLog, config); -cnxnFactory = ServerCnxnFactory.createFactory(); -cnxnFactory.configure(config.getClientPortAddress(), -config.getMaxClientCnxns()); +try +{ +cnxnFactory = ServerCnxnFactory.createFactory(); +cnxnFactory.configure(config.getClientPortAddress(), +config.getMaxClientCnxns()); +} +catch ( IOException e ) +{ +
[4/8] curator git commit: oops - forgot to try again on bind exception
oops - forgot to try again on bind exception Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/f8f13c25 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/f8f13c25 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/f8f13c25 Branch: refs/heads/CURATOR-3.0 Commit: f8f13c25d65e96efa6a0718515e8e035d72de7e9 Parents: 0603e95 Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 21:20:14 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 21:20:14 2015 -0500 -- .../main/java/org/apache/curator/test/TestingZooKeeperMain.java | 3 +++ 1 file changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/f8f13c25/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java index 82bcd9b..795e5d0 100644 --- a/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java +++ b/curator-test/src/main/java/org/apache/curator/test/TestingZooKeeperMain.java @@ -224,6 +224,9 @@ public class TestingZooKeeperMain implements ZooKeeperMainFace { log.info("Could not server. Waiting and trying one more time.", e); timing.sleepABit(); +cnxnFactory = ServerCnxnFactory.createFactory(); +cnxnFactory.configure(config.getClientPortAddress(), +config.getMaxClientCnxns()); } cnxnFactory.startup(zkServer); containerManager = new ContainerManager(zkServer.getZKDatabase(), zkServer.getFirstProcessor(), Integer.getInteger("znode.container.checkIntervalMs", (int)TimeUnit.MINUTES.toMillis(1L)).intValue(), Integer.getInteger("znode.container.maxPerMinute", 1).intValue());
[1/8] curator git commit: Removed usages of Javaassist and simplified how the in-memory ZK server is created and run. Tests should be more reliable and the testing server should respond faster
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 ea36769af -> afa8f7a45 Removed usages of Javaassist and simplified how the in-memory ZK server is created and run. Tests should be more reliable and the testing server should respond faster Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/911f02b1 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/911f02b1 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/911f02b1 Branch: refs/heads/CURATOR-3.0 Commit: 911f02b15e9585e893bdd2c9bdd860fbf85bba4c Parents: 73b05aa Author: randgalt <randg...@apache.org> Authored: Wed Sep 9 20:13:56 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Wed Sep 9 20:13:56 2015 -0500 -- .../recipes/leader/ChaosMonkeyCnxnFactory.java | 22 ++- curator-test/pom.xml| 5 - .../apache/curator/test/BaseClassForTests.java | 1 + .../apache/curator/test/ByteCodeRewrite.java| 131 .../org/apache/curator/test/TestingCluster.java | 5 - .../org/apache/curator/test/TestingServer.java | 5 - .../curator/test/TestingZooKeeperMain.java | 197 +++ pom.xml | 7 - 8 files changed, 134 insertions(+), 239 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/911f02b1/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java -- diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java index 1c0e50e..3aeec81 100644 --- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java +++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/ChaosMonkeyCnxnFactory.java @@ -19,6 +19,7 @@ package org.apache.curator.framework.recipes.leader; +import org.apache.curator.test.TestingZooKeeperMain; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.server.ByteBufferInputStream; @@ -49,23 +50,20 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory /* How long after the first error, connections are rejected */ public static final long LOCKOUT_DURATION_MS = 6000; -public ChaosMonkeyCnxnFactory() throws IOException -{ -} - @Override public void startup(ZooKeeperServer zks) throws IOException, InterruptedException { super.startup(new ChaosMonkeyZookeeperServer(zks)); } - public static class ChaosMonkeyZookeeperServer extends ZooKeeperServer { +private final ZooKeeperServer zks; private long firstError = 0; public ChaosMonkeyZookeeperServer(ZooKeeperServer zks) { +this.zks = zks; setTxnLogFactory(zks.getTxnLogFactory()); setTickTime(zks.getTickTime()); setMinSessionTimeout(zks.getMinSessionTimeout()); @@ -73,6 +71,20 @@ public class ChaosMonkeyCnxnFactory extends NIOServerCnxnFactory } @Override +public void startup() +{ +super.startup(); +if ( zks instanceof TestingZooKeeperMain.TestZooKeeperServer ) +{ +((TestingZooKeeperMain.TestZooKeeperServer)zks).noteStartup(); +} +else +{ +throw new RuntimeException("Unknown ZooKeeperServer: " + zks.getClass()); +} +} + +@Override public void submitRequest(Request si) { long remaining = firstError != 0 ? LOCKOUT_DURATION_MS - (System.currentTimeMillis() - firstError) : 0; http://git-wip-us.apache.org/repos/asf/curator/blob/911f02b1/curator-test/pom.xml -- diff --git a/curator-test/pom.xml b/curator-test/pom.xml index 80eedb2..d401da5 100644 --- a/curator-test/pom.xml +++ b/curator-test/pom.xml @@ -41,11 +41,6 @@ -org.javassist -javassist - - - org.apache.commons commons-math http://git-wip-us.apache.org/repos/asf/curator/blob/911f02b1/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java -- diff --git a/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java b/curator-test/src/main/java/org/apache/curator/test/BaseClassForTests.java index 1f6503d..a3bc2b5 100644 --- a/curator-test
[5/8] curator git commit: test assumed first latch would be leader. Fixed it so that this is so
test assumed first latch would be leader. Fixed it so that this is so Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/a9271ff6 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/a9271ff6 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/a9271ff6 Branch: refs/heads/CURATOR-3.0 Commit: a9271ff6033557efeda01e4c543d0f840519e64d Parents: f8f13c2 Author: randgalt <randg...@apache.org> Authored: Thu Sep 10 22:37:57 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Thu Sep 10 22:37:57 2015 -0500 -- .../apache/curator/framework/recipes/leader/TestLeaderLatch.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/a9271ff6/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 ecaa433..991e6fc 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 @@ -403,10 +403,9 @@ public class TestLeaderLatch extends BaseClassForTests LeaderLatch latch = new LeaderLatch(client, PATH_NAME); latch.start(); latches.add(latch); +waitForALeader(latches, timing); } -waitForALeader(latches, timing); - //we need to close a Participant that doesn't be actual leader (first Participant) nor the last latches.get(PARTICIPANT_ID).close();
[1/4] curator git commit: interim work - updated APIs make sure old tests work
Repository: curator Updated Branches: refs/heads/CURATOR-265 [created] 36ddd58a3 interim work - updated APIs make sure old tests work Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/2c0fca86 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/2c0fca86 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/2c0fca86 Branch: refs/heads/CURATOR-265 Commit: 2c0fca86ca3c25c9777e1b5c3bb3eea6cb0be8da Parents: ea36769 Author: randgalt <randg...@apache.org> Authored: Fri Sep 25 19:14:53 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 25 19:14:53 2015 -0500 -- .../api/AddStatConfigEnsembleable.java | 5 +- .../framework/api/BackgroundEnsembleable.java | 7 + .../framework/api/BackgroundStatable.java | 24 - .../curator/framework/api/Configurable.java | 31 -- .../framework/api/ConfigureEnsembleable.java| 32 ++ .../curator/framework/api/GetConfigBuilder.java | 9 +- .../api/JoinAddStatConfigEnsembleable.java | 3 +- .../api/JoinLeaveStatConfigEnsembleable.java| 3 +- .../api/JoinStatConfigEnsembleable.java | 5 +- .../framework/api/JoinStatConfigurable.java | 2 +- .../api/LeaveAddStatConfigEnsembleable.java | 3 +- .../api/LeaveStatConfigEnsembleable.java| 5 +- .../curator/framework/api/ReconfigBuilder.java | 4 +- .../framework/api/StatConfigEnsembleable.java | 26 - .../curator/framework/api/StatEnsembleable.java | 26 - .../api/WatchBackgroundEnsembleable.java| 7 + .../framework/imps/GetConfigBuilderImpl.java| 125 - .../framework/imps/ReconfigBuilderImpl.java | 110 +++-- .../framework/imps/TestReconfiguration.java | 474 +-- .../framework/imps/TestReconfigurationX.java| 418 20 files changed, 798 insertions(+), 521 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/2c0fca86/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java index 16f78a2..c60f617 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java @@ -25,8 +25,9 @@ package org.apache.curator.framework.api; * mixing concepts that can't be used together. */ public interface AddStatConfigEnsembleable extends -Addable, -StatConfigEnsembleable +Addable<Statable>, +ConfigureEnsembleable, +Statable { } http://git-wip-us.apache.org/repos/asf/curator/blob/2c0fca86/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java new file mode 100644 index 000..ae2b226 --- /dev/null +++ b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java @@ -0,0 +1,7 @@ +package org.apache.curator.framework.api; + +public interface BackgroundEnsembleable extends +Backgroundable<Ensembleable>, +Ensembleable +{ +} http://git-wip-us.apache.org/repos/asf/curator/blob/2c0fca86/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundStatable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundStatable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundStatable.java deleted file mode 100644 index 77c4e96..000 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundStatable.java +++ /dev/null @@ -1,24 +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 distribute
[4/4] curator git commit: finalized the changes and removed old classes
finalized the changes and removed old classes Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/36ddd58a Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/36ddd58a Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/36ddd58a Branch: refs/heads/CURATOR-265 Commit: 36ddd58a31045addfd9d984353956f7a99c09221 Parents: d42ef17 Author: randgalt <randg...@apache.org> Authored: Fri Sep 25 21:49:12 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 25 21:49:12 2015 -0500 -- .../framework/api/ACLBackgroundPathable.java| 25 -- .../api/ACLVersionBackgroundPathable.java | 25 -- .../api/BackgroundPathableQuietly.java | 23 - .../api/CreateModalPathAndBytesable.java| 25 -- .../api/IncrementalReconfigBuilder.java | 33 -- .../api/JoinAddStatConfigEnsembleable.java | 34 -- .../api/JoinLeaveStatConfigEnsembleable.java| 34 -- .../framework/api/JoinStatConfigurable.java | 30 -- .../api/LeaveAddStatConfigEnsembleable.java | 33 -- .../framework/api/SyncReconfigurable.java | 30 -- .../framework/imps/TestReconfiguration.java | 58 ++- .../framework/imps/TestReconfigurationX.java| 425 --- .../org/apache/curator/test/TestingCluster.java | 16 +- 13 files changed, 49 insertions(+), 742 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/36ddd58a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java deleted file mode 100644 index d63281d..000 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathable.java +++ /dev/null @@ -1,25 +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.framework.api; - -public interface ACLBackgroundPathable extends -ACLable<BackgroundPathable>, -BackgroundPathable -{ -} http://git-wip-us.apache.org/repos/asf/curator/blob/36ddd58a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java -- diff --git a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java b/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java deleted file mode 100644 index bc8e6bf..000 --- a/curator-framework/src/main/java/org/apache/curator/framework/api/ACLVersionBackgroundPathable.java +++ /dev/null @@ -1,25 +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.framework.api; - -public interface ACLVersionBackgroundPathable extends -ACLable<Versionable<BackgroundPathable>>, -Versionable<BackgroundPathable> -{ -} http://git-wip-us.apache.org/repos/asf/curator/blob/36ddd58a/curator-framework/s
[2/4] curator git commit: Merge branch 'CURATOR-3.0' into CURATOR-265
Merge branch 'CURATOR-3.0' into CURATOR-265 Conflicts: curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/4c3c8377 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/4c3c8377 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/4c3c8377 Branch: refs/heads/CURATOR-265 Commit: 4c3c8377935f8ae2c0f24c4a1010cf4bdef4e297 Parents: 2c0fca8 afa8f7a Author: randgalt <randg...@apache.org> Authored: Fri Sep 25 19:22:10 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 25 19:22:10 2015 -0500 -- .../curator/framework/imps/TestFramework.java | 4 +- .../recipes/leader/ChaosMonkeyCnxnFactory.java | 24 +- .../recipes/leader/TestLeaderLatch.java | 3 +- .../locks/TestInterProcessMutexBase.java| 6 +- curator-test/pom.xml| 5 - .../apache/curator/test/BaseClassForTests.java | 1 + .../apache/curator/test/ByteCodeRewrite.java| 131 -- .../curator/test/QuorumConfigBuilder.java | 36 ++- .../org/apache/curator/test/TestingCluster.java | 5 - .../org/apache/curator/test/TestingServer.java | 5 - .../curator/test/TestingZooKeeperMain.java | 236 --- .../curator/test/TestingZooKeeperServer.java| 2 + pom.xml | 21 +- 13 files changed, 218 insertions(+), 261 deletions(-) --
curator git commit: Fixed newMembers test
Repository: curator Updated Branches: refs/heads/CURATOR-265 e09388a5a -> a8ec17ece Fixed newMembers test Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/a8ec17ec Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/a8ec17ec Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/a8ec17ec Branch: refs/heads/CURATOR-265 Commit: a8ec17ece060484e0e6ce723568ed9505f6b82ef Parents: e09388a Author: randgalt <randg...@apache.org> Authored: Fri Sep 25 22:05:43 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Fri Sep 25 22:05:43 2015 -0500 -- .../org/apache/curator/framework/imps/TestReconfiguration.java| 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/a8ec17ec/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java -- diff --git a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java index ca00cba..e7d2229 100644 --- a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java +++ b/curator-framework/src/test/java/org/apache/curator/framework/imps/TestReconfiguration.java @@ -282,7 +282,7 @@ public class TestReconfiguration extends BaseClassForTests client.start(); QuorumVerifier oldConfig = toQuorumVerifier(client.getConfig().forEnsemble()); -Assert.assertEquals(cluster.getInstances().size(), 5); +Assert.assertEquals(oldConfig.getAllMembers().size(), 5); assertConfig(oldConfig, cluster.getInstances()); CountDownLatch latch = setChangeWaiter(client); @@ -291,6 +291,7 @@ public class TestReconfiguration extends BaseClassForTests Assert.assertTrue(timing.awaitLatch(latch)); QuorumVerifier newConfig = toQuorumVerifier(client.getConfig().forEnsemble()); +Assert.assertEquals(newConfig.getAllMembers().size(), 3); assertConfig(newConfig, smallCluster); } }
[2/2] curator git commit: New injectSessionExpiration() operates much faster than previously. It exposes an assumption in the tests. Added a debug hook to work around
New injectSessionExpiration() operates much faster than previously. It exposes an assumption in the tests. Added a debug hook to work around Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/145da217 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/145da217 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/145da217 Branch: refs/heads/CURATOR-253 Commit: 145da217ff35df0178823da784f4dd1618851c5e Parents: d1b4cbf Author: randgalt randg...@apache.org Authored: Fri Aug 28 13:39:54 2015 -0500 Committer: randgalt randg...@apache.org Committed: Fri Aug 28 13:39:54 2015 -0500 -- .../recipes/nodes/PersistentEphemeralNode.java | 16 .../recipes/nodes/TestPersistentEphemeralNode.java | 17 + 2 files changed, 25 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/145da217/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 0b482ef..e889bd8 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 @@ -19,6 +19,7 @@ package org.apache.curator.framework.recipes.nodes; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.curator.framework.CuratorFramework; @@ -111,11 +112,26 @@ public class PersistentEphemeralNode implements Closeable { if ( newState == ConnectionState.RECONNECTED ) { +if ( debugReconnectLatch != null ) +{ +try +{ +debugReconnectLatch.await(); +} +catch ( InterruptedException e ) +{ +Thread.currentThread().interrupt(); +e.printStackTrace(); +} +} createNode(); } } }; +@VisibleForTesting +volatile CountDownLatch debugReconnectLatch = null; + private enum State { LATENT, http://git-wip-us.apache.org/repos/asf/curator/blob/145da217/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 c81cc65..3a0d564 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 @@ -297,6 +297,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests CuratorFramework observer = newCurator(); PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]); +node.debugReconnectLatch = new CountDownLatch(1); node.start(); try { @@ -307,10 +308,11 @@ public class TestPersistentEphemeralNode extends BaseClassForTests Trigger deletedTrigger = Trigger.deleted(); observer.checkExists().usingWatcher(deletedTrigger).forPath(node.getActualPath()); -killSession(curator); +KillSession.kill(curator.getZookeeperClient().getZooKeeper()); // Make sure the node got deleted assertTrue(deletedTrigger.firedWithin(timing.forWaiting().seconds(), TimeUnit.SECONDS)); +node.debugReconnectLatch.countDown(); } finally { @@ -325,6 +327,7 @@ public class TestPersistentEphemeralNode extends BaseClassForTests CuratorFramework observer = newCurator(); PersistentEphemeralNode node = new PersistentEphemeralNode(curator, PersistentEphemeralNode.Mode.EPHEMERAL, PATH, new byte[0]); +node.debugReconnectLatch = new CountDownLatch(1); node.start(); try { @@ -334,10 +337,11 @@ public class TestPersistentEphemeralNode extends BaseClassForTests Trigger deletedTrigger = Trigger.deleted
[1/2] curator git commit: Switch to new injectSessionExpiration()
Repository: curator Updated Branches: refs/heads/CURATOR-253 [created] 145da217f Switch to new injectSessionExpiration() Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/d1b4cbf0 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/d1b4cbf0 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/d1b4cbf0 Branch: refs/heads/CURATOR-253 Commit: d1b4cbf070046ace5c047cc46d99c0ae71f749a5 Parents: 0f5668b Author: randgalt randg...@apache.org Authored: Fri Aug 28 13:39:22 2015 -0500 Committer: randgalt randg...@apache.org Committed: Fri Aug 28 13:39:22 2015 -0500 -- .../curator/TestSessionFailRetryLoop.java | 9 +- .../framework/recipes/cache/TestTreeCache.java | 1 - .../org/apache/curator/test/KillSession.java| 87 3 files changed, 21 insertions(+), 76 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/d1b4cbf0/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java -- diff --git a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java index c17b5bf..937726e 100644 --- a/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java +++ b/curator-client/src/test/java/org/apache/curator/TestSessionFailRetryLoop.java @@ -18,6 +18,7 @@ */ package org.apache.curator; +import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.BaseClassForTests; import org.apache.curator.utils.CloseableUtils; import org.apache.curator.retry.RetryOneTime; @@ -34,7 +35,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests public void testRetry() throws Exception { Timing timing = new Timing(); -final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1)); +final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3)); SessionFailRetryLoopretryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.RETRY); retryLoop.start(); try @@ -103,7 +104,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests public void testRetryStatic() throws Exception { Timing timing = new Timing(); -final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1)); +final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3)); SessionFailRetryLoopretryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.RETRY); retryLoop.start(); try @@ -175,7 +176,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests public void testBasic() throws Exception { Timing timing = new Timing(); -final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1)); +final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3)); SessionFailRetryLoopretryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.FAIL); retryLoop.start(); try @@ -230,7 +231,7 @@ public class TestSessionFailRetryLoop extends BaseClassForTests public void testBasicStatic() throws Exception { Timing timing = new Timing(); -final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new RetryOneTime(1)); +final CuratorZookeeperClientclient = new CuratorZookeeperClient(server.getConnectString(), timing.session(), timing.connection(), null, new ExponentialBackoffRetry(100, 3)); SessionFailRetryLoopretryLoop = client.newSessionFailRetryLoop(SessionFailRetryLoop.Mode.FAIL); retryLoop.start(); try http://git-wip-us.apache.org/repos/asf/curator/blob/d1b4cbf0/curator-recipes/src/test/java/org/apache/curator/framework/recipes/cache/TestTreeCache.java
[1/3] curator git commit: CURATOR-270 createContainers does not work correctly with usingNamespace
Repository: curator Updated Branches: refs/heads/CURATOR-3.0 537156db4 -> df949e7a2 CURATOR-270 createContainers does not work correctly with usingNamespace Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/1556a2fc Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/1556a2fc Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/1556a2fc Branch: refs/heads/CURATOR-3.0 Commit: 1556a2fcb8fd0d43669b057eb51290e3bf5ee5a2 Parents: f8f05be Author: Alexey SerbaAuthored: Tue Oct 6 00:11:35 2015 +0300 Committer: Alexey Serba Committed: Tue Oct 6 00:11:35 2015 +0300 -- .../curator/framework/imps/NamespaceFacade.java | 1 + .../curator/framework/imps/TestFramework.java | 42 2 files changed, 43 insertions(+) -- http://git-wip-us.apache.org/repos/asf/curator/blob/1556a2fc/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-framework/src/main/java/org/apache/curator/framework/imps/NamespaceFacade.java index 60ef647..a4bb2e5 100644 --- a/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceFacade.java +++ b/curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceFacade.java @@ -45,6 +45,7 @@ class NamespaceFacade extends CuratorFrameworkImpl @Override public void createContainers(String path) throws Exception { +path = fixForNamespace(path); client.createContainers(path); } http://git-wip-us.apache.org/repos/asf/curator/blob/1556a2fc/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 811631c..b6a0a40 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 @@ -592,6 +592,48 @@ public class TestFramework extends BaseClassForTests CloseableUtils.closeQuietly(client); } } + +@Test +public void testCreateContainersWithNamespace() throws Exception +{ +final String namespace = "container1"; +CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder(); +CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).namespace(namespace).build(); +client.start(); +try +{ +String path = "/path1/path2"; +client.createContainers(path); +Assert.assertNotNull(client.checkExists().forPath(path)); + Assert.assertNotNull(client.getZookeeperClient().getZooKeeper().exists("/" + namespace + path, false)); +} +finally +{ +CloseableUtils.closeQuietly(client); +} +} + + +@Test +public void testCreateContainersUsingNamespace() throws Exception +{ +final String namespace = "container2"; +CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder(); +CuratorFramework client = builder.connectString(server.getConnectString()).retryPolicy(new RetryOneTime(1)).build(); +client.start(); +CuratorFramework nsClient = client.usingNamespace(namespace); +try +{ +String path = "/path1/path2"; +nsClient.createContainers(path); +Assert.assertNotNull(nsClient.checkExists().forPath(path)); + Assert.assertNotNull(nsClient.getZookeeperClient().getZooKeeper().exists("/" + namespace + path, false)); +} +finally +{ +CloseableUtils.closeQuietly(client); +} +} @Test public void testNamespace() throws Exception
[3/3] curator git commit: Merge branch 'master' into CURATOR-3.0
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/df949e7a Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/df949e7a Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/df949e7a Branch: refs/heads/CURATOR-3.0 Commit: df949e7a27e370d0e7f8013a7c7b0469382d1a2f Parents: 537156d f4f2208 Author: randgalt <randg...@apache.org> Authored: Tue Oct 6 08:23:41 2015 -0500 Committer: randgalt <randg...@apache.org> Committed: Tue Oct 6 08:23:41 2015 -0500 -- .../curator/framework/imps/NamespaceFacade.java | 6 --- .../curator/framework/imps/TestFramework.java | 42 2 files changed, 42 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/curator/blob/df949e7a/curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java --
[2/2] curator git commit: Merge branch 'CURATOR-3.0' of https://git-wip-us.apache.org/repos/asf/curator into CURATOR-3.0
Merge branch 'CURATOR-3.0' of https://git-wip-us.apache.org/repos/asf/curator 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/5c34cb10 Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/5c34cb10 Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/5c34cb10 Branch: refs/heads/CURATOR-3.0 Commit: 5c34cb10bd3e18bfde8b6f71d490bc4325f9d726 Parents: 0ae2950 2d0d2b1 Author: randgalt <randg...@apache.org> Authored: Wed Jan 6 12:52:50 2016 -0800 Committer: randgalt <randg...@apache.org> Committed: Wed Jan 6 12:52:50 2016 -0800 -- .../main/java/org/apache/curator/test/TestingZooKeeperMain.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) --