Return-Path: X-Original-To: apmail-curator-commits-archive@minotaur.apache.org Delivered-To: apmail-curator-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 58FB411944 for ; Wed, 20 Aug 2014 21:14:04 +0000 (UTC) Received: (qmail 68809 invoked by uid 500); 20 Aug 2014 21:14:03 -0000 Delivered-To: apmail-curator-commits-archive@curator.apache.org Received: (qmail 68757 invoked by uid 500); 20 Aug 2014 21:14:03 -0000 Mailing-List: contact commits-help@curator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@curator.apache.org Delivered-To: mailing list commits@curator.apache.org Received: (qmail 68742 invoked by uid 99); 20 Aug 2014 21:14:03 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Aug 2014 21:14:03 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 282539827C3; Wed, 20 Aug 2014 21:14:03 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: randgalt@apache.org To: commits@curator.apache.org Date: Wed, 20 Aug 2014 21:14:03 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/2] git commit: CURATOR-136 Consistent relpath handling in recipes Repository: curator Updated Branches: refs/heads/master e6fc38cf5 -> 7bec246de CURATOR-136 Consistent relpath handling in recipes LeaderLatch would previously normalize a relative path in some places but not others, causing inconsistent behavior. Make it check for an absolute path on startup. Additionally, scrub through the rest of the recipes and validate that the path given is also valid before processing so that clients will see failures early. Modify PathUtils.validatePath to return the path given, for fluent- style invocation, modelled on guava's Preconditions. This closes #39 Project: http://git-wip-us.apache.org/repos/asf/curator/repo Commit: http://git-wip-us.apache.org/repos/asf/curator/commit/05254afe Tree: http://git-wip-us.apache.org/repos/asf/curator/tree/05254afe Diff: http://git-wip-us.apache.org/repos/asf/curator/diff/05254afe Branch: refs/heads/master Commit: 05254afea628a1018e1058d230a568746f98df5a Parents: 0d1397a Author: Mike Drob Authored: Mon Aug 11 15:12:53 2014 -0500 Committer: Mike Drob Committed: Wed Aug 20 16:08:40 2014 -0500 ---------------------------------------------------------------------- .../src/main/java/org/apache/curator/utils/PathUtils.java | 7 +++++-- .../framework/recipes/atomic/DistributedAtomicValue.java | 4 +++- .../curator/framework/recipes/atomic/PromotedToLock.java | 3 ++- .../framework/recipes/barriers/DistributedBarrier.java | 3 ++- .../framework/recipes/barriers/DistributedDoubleBarrier.java | 3 ++- .../apache/curator/framework/recipes/cache/ChildData.java | 3 ++- .../curator/framework/recipes/cache/GetDataOperation.java | 4 +++- .../apache/curator/framework/recipes/cache/NodeCache.java | 3 ++- .../curator/framework/recipes/cache/PathChildrenCache.java | 3 ++- .../apache/curator/framework/recipes/cache/TreeCache.java | 3 ++- .../apache/curator/framework/recipes/leader/LeaderLatch.java | 5 +++-- .../curator/framework/recipes/leader/LeaderSelector.java | 3 ++- .../apache/curator/framework/recipes/locks/ChildReaper.java | 3 ++- .../framework/recipes/locks/InterProcessMultiLock.java | 1 + .../curator/framework/recipes/locks/InterProcessMutex.java | 3 ++- .../framework/recipes/locks/InterProcessSemaphore.java | 1 + .../framework/recipes/locks/InterProcessSemaphoreV2.java | 2 ++ .../curator/framework/recipes/locks/LockInternals.java | 3 +-- .../curator/framework/recipes/locks/PredicateResults.java | 4 +++- .../framework/recipes/nodes/PersistentEphemeralNode.java | 3 ++- .../curator/framework/recipes/queue/ChildrenCache.java | 3 ++- .../curator/framework/recipes/queue/DistributedQueue.java | 6 +++--- .../apache/curator/framework/recipes/queue/QueueBuilder.java | 5 +++-- .../apache/curator/framework/recipes/queue/QueueSafety.java | 5 +++-- .../framework/recipes/queue/SimpleDistributedQueue.java | 3 ++- .../apache/curator/framework/recipes/shared/SharedValue.java | 3 ++- .../curator/framework/recipes/leader/TestLeaderLatch.java | 8 ++++++++ 27 files changed, 67 insertions(+), 30 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-client/src/main/java/org/apache/curator/utils/PathUtils.java ---------------------------------------------------------------------- diff --git a/curator-client/src/main/java/org/apache/curator/utils/PathUtils.java b/curator-client/src/main/java/org/apache/curator/utils/PathUtils.java index c491874..4fcf05b 100644 --- a/curator-client/src/main/java/org/apache/curator/utils/PathUtils.java +++ b/curator-client/src/main/java/org/apache/curator/utils/PathUtils.java @@ -40,9 +40,10 @@ public class PathUtils { /** * Validate the provided znode path string * @param path znode path string + * @return The given path if it was valid, for fluent chaining * @throws IllegalArgumentException if the path is invalid */ - public static void validatePath(String path) throws IllegalArgumentException { + public static String validatePath(String path) throws IllegalArgumentException { if (path == null) { throw new IllegalArgumentException("Path cannot be null"); } @@ -54,7 +55,7 @@ public class PathUtils { "Path must start with / character"); } if (path.length() == 1) { // done checking - it's the root - return; + return path; } if (path.charAt(path.length() - 1) == '/') { throw new IllegalArgumentException( @@ -101,5 +102,7 @@ public class PathUtils { throw new IllegalArgumentException( "Invalid path string \"" + path + "\" caused by " + reason); } + + return path; } } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/DistributedAtomicValue.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/DistributedAtomicValue.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/DistributedAtomicValue.java index 4ba3097..c90fb2b 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/DistributedAtomicValue.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/DistributedAtomicValue.java @@ -26,6 +26,8 @@ import org.apache.curator.utils.EnsurePath; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; import java.util.Arrays; +import org.apache.curator.utils.PathUtils; +import org.apache.zookeeper.ZKUtil; /** *

A distributed value that attempts atomic sets. It first tries uses optimistic locking. If that fails, @@ -69,7 +71,7 @@ public class DistributedAtomicValue public DistributedAtomicValue(CuratorFramework client, String path, RetryPolicy retryPolicy, PromotedToLock promotedToLock) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); this.retryPolicy = retryPolicy; this.promotedToLock = promotedToLock; mutex = (promotedToLock != null) ? new InterProcessMutex(client, promotedToLock.getPath()) : null; http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/PromotedToLock.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/PromotedToLock.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/PromotedToLock.java index 04ef3d1..caad66e 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/PromotedToLock.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/PromotedToLock.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import org.apache.curator.RetryPolicy; import org.apache.curator.retry.RetryNTimes; import java.util.concurrent.TimeUnit; +import org.apache.curator.utils.PathUtils; /** * Abstraction of arguments for mutex promotion. Use {@link #builder()} to create. @@ -68,7 +69,7 @@ public class PromotedToLock */ public Builder lockPath(String path) { - instance = new PromotedToLock(path, instance.maxLockTime, instance.maxLockTimeUnit, instance.retryPolicy); + instance = new PromotedToLock(PathUtils.validatePath(path), instance.maxLockTime, instance.maxLockTimeUnit, instance.retryPolicy); return this; } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedBarrier.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedBarrier.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedBarrier.java index f978a7c..b891b2b 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedBarrier.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedBarrier.java @@ -23,6 +23,7 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import java.util.concurrent.TimeUnit; +import org.apache.curator.utils.PathUtils; /** *

@@ -54,7 +55,7 @@ public class DistributedBarrier public DistributedBarrier(CuratorFramework client, String barrierPath) { this.client = client; - this.barrierPath = barrierPath; + this.barrierPath = PathUtils.validatePath(barrierPath); } /** http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedDoubleBarrier.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedDoubleBarrier.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedDoubleBarrier.java index 6b204f1..5034b0a 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedDoubleBarrier.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/barriers/DistributedDoubleBarrier.java @@ -36,6 +36,7 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.curator.utils.PathUtils; /** *

@@ -85,7 +86,7 @@ public class DistributedDoubleBarrier Preconditions.checkState(memberQty > 0, "memberQty cannot be 0"); this.client = client; - this.barrierPath = barrierPath; + this.barrierPath = PathUtils.validatePath(barrierPath); this.memberQty = memberQty; ourPath = ZKPaths.makePath(barrierPath, UUID.randomUUID().toString()); readyPath = ZKPaths.makePath(barrierPath, READY_NODE); http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/ChildData.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/ChildData.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/ChildData.java index d385d4d..21e0bc4 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/ChildData.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/ChildData.java @@ -21,6 +21,7 @@ package org.apache.curator.framework.recipes.cache; import org.apache.zookeeper.data.Stat; import java.util.Arrays; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; public class ChildData implements Comparable { @@ -30,7 +31,7 @@ public class ChildData implements Comparable public ChildData(String path, Stat stat, byte[] data) { - this.path = path; + this.path = PathUtils.validatePath(path); this.stat = stat; this.data = new AtomicReference(data); } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/GetDataOperation.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/GetDataOperation.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/GetDataOperation.java index a6581b2..f789d10 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/GetDataOperation.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/GetDataOperation.java @@ -18,6 +18,8 @@ */ package org.apache.curator.framework.recipes.cache; +import org.apache.curator.utils.PathUtils; + class GetDataOperation implements Operation { private final PathChildrenCache cache; @@ -26,7 +28,7 @@ class GetDataOperation implements Operation GetDataOperation(PathChildrenCache cache, String fullPath) { this.cache = cache; - this.fullPath = fullPath; + this.fullPath = PathUtils.validatePath(fullPath); } @Override http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/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 bfa3505..fa0df51 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 @@ -40,6 +40,7 @@ import java.io.IOException; import java.util.concurrent.Exchanger; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

A utility that attempts to keep the data from a node locally cached. This class @@ -129,7 +130,7 @@ public class NodeCache implements Closeable public NodeCache(CuratorFramework client, String path, boolean dataIsCompressed) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); this.dataIsCompressed = dataIsCompressed; ensurePath = client.newNamespaceAwareEnsurePath(path).excludingLast(); } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java index 57c6e92..2d2f586 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCache.java @@ -54,6 +54,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

A utility that attempts to keep all data from all children of a ZK path locally cached. This class @@ -216,7 +217,7 @@ public class PathChildrenCache implements Closeable public PathChildrenCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, final CloseableExecutorService executorService) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); this.cacheData = cacheData; this.dataIsCompressed = dataIsCompressed; this.executorService = executorService; http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java index 617d652..a4c252b 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCache.java @@ -50,6 +50,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

A utility that attempts to keep all data from all children of a ZK path locally cached. This class @@ -411,7 +412,7 @@ public class TreeCache implements Closeable */ public TreeCache(CuratorFramework client, String path, boolean cacheData, boolean dataIsCompressed, final CloseableExecutorService executorService) { - this.root = new TreeNode(path, null); + this.root = new TreeNode(PathUtils.validatePath(path), null); this.client = client; this.cacheData = cacheData; this.dataIsCompressed = dataIsCompressed; http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/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 af3f895..489cd7e 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 @@ -50,6 +50,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

@@ -143,7 +144,7 @@ public class LeaderLatch implements Closeable public LeaderLatch(CuratorFramework client, String latchPath, String id, CloseMode closeMode) { this.client = Preconditions.checkNotNull(client, "client cannot be null"); - this.latchPath = Preconditions.checkNotNull(latchPath, "mutexPath cannot be null"); + this.latchPath = PathUtils.validatePath(latchPath); this.id = Preconditions.checkNotNull(id, "id cannot be null"); this.closeMode = Preconditions.checkNotNull(closeMode, "closeMode cannot be null"); } @@ -582,7 +583,7 @@ public class LeaderLatch implements Closeable } } }; - client.getChildren().inBackground(callback).forPath(latchPath); + client.getChildren().inBackground(callback).forPath(ZKPaths.makePath(latchPath, null)); } private void handleStateChange(ConnectionState newState) http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java index 3be941a..9c09b4c 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java @@ -46,6 +46,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

@@ -135,7 +136,7 @@ public class LeaderSelector implements Closeable public LeaderSelector(CuratorFramework client, String leaderPath, CloseableExecutorService executorService, LeaderSelectorListener listener) { Preconditions.checkNotNull(client, "client cannot be null"); - Preconditions.checkNotNull(leaderPath, "leaderPath cannot be null"); + PathUtils.validatePath(leaderPath); Preconditions.checkNotNull(listener, "listener cannot be null"); this.client = client; http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/ChildReaper.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/ChildReaper.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/ChildReaper.java index 2035ccc..5f28f82 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/ChildReaper.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/ChildReaper.java @@ -34,6 +34,7 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** * Utility to reap empty child nodes of a parent node. Periodically calls getChildren on @@ -103,7 +104,7 @@ public class ChildReaper implements Closeable public ChildReaper(CuratorFramework client, String path, Reaper.Mode mode, ScheduledExecutorService executor, int reapingThresholdMs, String leaderPath) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); this.mode = mode; this.executor = new CloseableScheduledExecutorService(executor); this.reapingThresholdMs = reapingThresholdMs; http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMultiLock.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMultiLock.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMultiLock.java index ca06b34..f400a1a 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMultiLock.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMultiLock.java @@ -43,6 +43,7 @@ public class InterProcessMultiLock implements InterProcessLock */ public InterProcessMultiLock(CuratorFramework client, List paths) { + // paths get checked in each individual InterProcessMutex, so trust them here this(makeLocks(client, paths)); } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMutex.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMutex.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMutex.java index ffdeb70..4c647da 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMutex.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/InterProcessMutex.java @@ -27,6 +27,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.curator.utils.PathUtils; /** * A re-entrant mutex that works across JVMs. Uses Zookeeper to hold the lock. All processes in all JVMs that @@ -186,7 +187,7 @@ public class InterProcessMutex implements InterProcessLock, Revocable @@ -115,6 +116,7 @@ public class InterProcessSemaphoreV2 private InterProcessSemaphoreV2(CuratorFramework client, String path, int maxLeases, SharedCountReader count) { this.client = client; + path = PathUtils.validatePath(path); lock = new InterProcessMutex(client, ZKPaths.makePath(path, LOCK_PARENT)); this.maxLeases = (count != null) ? count.getCount() : maxLeases; leasesPath = ZKPaths.makePath(path, LEASE_PARENT); http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java index 706b242..20402db 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternals.java @@ -99,10 +99,9 @@ public class LockInternals this.driver = driver; this.lockName = lockName; this.maxLeases = maxLeases; - PathUtils.validatePath(path); this.client = client; - this.basePath = path; + this.basePath = PathUtils.validatePath(path); this.path = ZKPaths.makePath(path, lockName); } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/PredicateResults.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/PredicateResults.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/PredicateResults.java index 0f098ea..02454d2 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/PredicateResults.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/PredicateResults.java @@ -18,6 +18,8 @@ */ package org.apache.curator.framework.recipes.locks; +import org.apache.curator.utils.PathUtils; + class PredicateResults { private final boolean getsTheLock; @@ -25,7 +27,7 @@ class PredicateResults PredicateResults(String pathToWatch, boolean getsTheLock) { - this.pathToWatch = pathToWatch; + this.pathToWatch = PathUtils.validatePath(pathToWatch); this.getsTheLock = getsTheLock; } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/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 2b07586..1f8be75 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 @@ -39,6 +39,7 @@ import java.util.Arrays; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

@@ -190,7 +191,7 @@ public class PersistentEphemeralNode implements Closeable public PersistentEphemeralNode(CuratorFramework client, Mode mode, String basePath, byte[] data) { this.client = Preconditions.checkNotNull(client, "client cannot be null"); - this.basePath = Preconditions.checkNotNull(basePath, "basePath cannot be null"); + this.basePath = PathUtils.validatePath(basePath); this.mode = Preconditions.checkNotNull(mode, "mode cannot be null"); data = Preconditions.checkNotNull(data, "data cannot be null"); http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java index 5619c59..032dc7a 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ChildrenCache.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; class ChildrenCache implements Closeable { @@ -79,7 +80,7 @@ class ChildrenCache implements Closeable ChildrenCache(CuratorFramework client, String path) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); } void start() throws Exception http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java index ca53a4e..671bea9 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/DistributedQueue.java @@ -47,6 +47,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** *

An implementation of the Distributed Queue ZK recipe. Items put into the queue @@ -120,20 +121,19 @@ public class DistributedQueue implements QueueBase { Preconditions.checkNotNull(client, "client cannot be null"); Preconditions.checkNotNull(serializer, "serializer cannot be null"); - Preconditions.checkNotNull(queuePath, "queuePath cannot be null"); Preconditions.checkNotNull(threadFactory, "threadFactory cannot be null"); Preconditions.checkNotNull(executor, "executor cannot be null"); Preconditions.checkArgument(maxItems > 0, "maxItems must be a positive number"); isProducerOnly = (consumer == null); - this.lockPath = lockPath; + this.lockPath = PathUtils.validatePath(lockPath); this.putInBackground = putInBackground; this.consumer = consumer; this.minItemsBeforeRefresh = minItemsBeforeRefresh; this.refreshOnWatch = refreshOnWatch; this.client = client; this.serializer = serializer; - this.queuePath = queuePath; + this.queuePath = PathUtils.validatePath(queuePath); this.executor = executor; this.maxItems = maxItems; this.finalFlushMs = finalFlushMs; http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueBuilder.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueBuilder.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueBuilder.java index fe005f6..a571578 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueBuilder.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueBuilder.java @@ -26,6 +26,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import org.apache.curator.utils.PathUtils; /** * The builder for both {@link DistributedQueue} and {@link DistributedPriorityQueue} @@ -213,7 +214,7 @@ public class QueueBuilder */ public QueueBuilder lockPath(String path) { - lockPath = path; + lockPath = PathUtils.validatePath(path); return this; } @@ -265,7 +266,7 @@ public class QueueBuilder this.client = client; this.consumer = consumer; this.serializer = serializer; - this.queuePath = queuePath; + this.queuePath = PathUtils.validatePath(queuePath); factory = defaultThreadFactory; executor = MoreExecutors.sameThreadExecutor(); http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSafety.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSafety.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSafety.java index a312c77..3de5ba5 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSafety.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSafety.java @@ -19,6 +19,7 @@ package org.apache.curator.framework.recipes.queue; import java.util.concurrent.BlockingQueue; +import org.apache.curator.utils.PathUtils; /** * Parameter block for specifying queue safety with either {@link DistributedQueue} or @@ -36,14 +37,14 @@ public class QueueSafety */ public QueueSafety(String lockPath, QueueConsumer consumer) { - this.lockPath = lockPath; + this.lockPath = PathUtils.validatePath(lockPath); this.consumer = consumer; this.queue = null; } QueueSafety(String lockPath, BlockingQueue queue) { - this.lockPath = lockPath; + this.lockPath = PathUtils.validatePath(lockPath); this.consumer = null; this.queue = queue; } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java index 5b00855..0c386cd 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/SimpleDistributedQueue.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import org.apache.curator.utils.PathUtils; /** *

@@ -60,7 +61,7 @@ public class SimpleDistributedQueue public SimpleDistributedQueue(CuratorFramework client, String path) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); ensurePath = client.newNamespaceAwareEnsurePath(path); } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java ---------------------------------------------------------------------- diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java index 9524650..5c4b53b 100644 --- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java +++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/shared/SharedValue.java @@ -34,6 +34,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.Arrays; import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.utils.PathUtils; /** * Manages a shared value. All clients watching the same path will have the up-to-date @@ -88,7 +89,7 @@ public class SharedValue implements Closeable, SharedValueReader public SharedValue(CuratorFramework client, String path, byte[] seedValue) { this.client = client; - this.path = path; + this.path = PathUtils.validatePath(path); this.seedValue = Arrays.copyOf(seedValue, seedValue.length); value = seedValue; } http://git-wip-us.apache.org/repos/asf/curator/blob/05254afe/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 9fd2f10..96e6d45 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 @@ -739,4 +739,12 @@ public class TestLeaderLatch extends BaseClassForTests } return leaders; } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void testRelativePath() throws Exception + { + Timing timing = new Timing(); + CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), timing.connection(), new RetryOneTime(1)); + LeaderLatch latch = new LeaderLatch(client, "parent"); + } }