|
@@ -20,8 +20,12 @@ import java.io.BufferedReader;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStreamReader;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.List;
|
|
|
import java.util.Date;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
|
|
|
import org.apache.log4j.Logger;
|
|
|
|
|
@@ -55,7 +59,6 @@ import com.yahoo.zookeeper.proto.SyncRequest;
|
|
|
import com.yahoo.zookeeper.proto.SyncResponse;
|
|
|
import com.yahoo.zookeeper.proto.WatcherEvent;
|
|
|
import com.yahoo.zookeeper.server.DataTree;
|
|
|
-import com.yahoo.zookeeper.server.ZooKeeperServer;
|
|
|
|
|
|
/**
|
|
|
* This is the main class of ZooKeeper client library. To use a ZooKeeper
|
|
@@ -108,7 +111,145 @@ import com.yahoo.zookeeper.server.ZooKeeperServer;
|
|
|
public class ZooKeeper {
|
|
|
private static final Logger LOG = Logger.getLogger(ZooKeeper.class);
|
|
|
|
|
|
- volatile Watcher watcher;
|
|
|
+ private volatile Watcher defaultWatcher;
|
|
|
+
|
|
|
+ private final Map<String, Set<Watcher>> dataWatches =
|
|
|
+ new HashMap<String, Set<Watcher>>();
|
|
|
+ private final Map<String, Set<Watcher>> childWatches =
|
|
|
+ new HashMap<String, Set<Watcher>>();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Process a WatchEvent.
|
|
|
+ *
|
|
|
+ * Looks up the watch in the set of watches, processes the event
|
|
|
+ * if found, otw uses the default watcher (registered during instance
|
|
|
+ * creation) to process the watch.
|
|
|
+ *
|
|
|
+ * @param event the event to process.
|
|
|
+ */
|
|
|
+ public void processWatchEvent(WatcherEvent event) {
|
|
|
+ // clear the watches if we are not connected
|
|
|
+ if (event.getState() != Watcher.Event.KeeperStateSyncConnected) {
|
|
|
+ synchronized (dataWatches) {
|
|
|
+ for (Set<Watcher> watchers : dataWatches.values()) {
|
|
|
+ for (Watcher watcher : watchers) {
|
|
|
+ watcher.process(event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ dataWatches.clear();
|
|
|
+ }
|
|
|
+ synchronized (childWatches) {
|
|
|
+ for (Set<Watcher> watchers : childWatches.values()) {
|
|
|
+ for (Watcher watcher : watchers) {
|
|
|
+ watcher.process(event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ childWatches.clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<Watcher> watchers = null;
|
|
|
+
|
|
|
+ switch (event.getType()) {
|
|
|
+ case Watcher.Event.EventNone:
|
|
|
+ defaultWatcher.process(event);
|
|
|
+ return;
|
|
|
+ case Watcher.Event.EventNodeDataChanged:
|
|
|
+ case Watcher.Event.EventNodeCreated:
|
|
|
+ synchronized (dataWatches) {
|
|
|
+ watchers = dataWatches.remove(event.getPath());
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ case Watcher.Event.EventNodeChildrenChanged:
|
|
|
+ synchronized (childWatches) {
|
|
|
+ watchers = childWatches.remove(event.getPath());
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ case Watcher.Event.EventNodeDeleted:
|
|
|
+ synchronized (dataWatches) {
|
|
|
+ watchers = dataWatches.remove(event.getPath());
|
|
|
+ }
|
|
|
+ Set<Watcher> cwatches;
|
|
|
+ synchronized (childWatches) {
|
|
|
+ cwatches = childWatches.remove(event.getPath());
|
|
|
+ }
|
|
|
+ if (cwatches != null) {
|
|
|
+ if (watchers == null) {
|
|
|
+ watchers = cwatches;
|
|
|
+ } else {
|
|
|
+ watchers.addAll(cwatches);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ String msg = "Unhandled watch event type " + event.getType();
|
|
|
+ LOG.error(msg);
|
|
|
+ throw new RuntimeException(msg);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (watchers != null) {
|
|
|
+ for (Watcher watcher : watchers) {
|
|
|
+ watcher.process(event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Register a watcher for a particular path.
|
|
|
+ */
|
|
|
+ class WatchRegistration {
|
|
|
+ private Map<String, Set<Watcher>> watches;
|
|
|
+ private Watcher watcher;
|
|
|
+ private String path;
|
|
|
+ public WatchRegistration(Map<String, Set<Watcher>> watches,
|
|
|
+ Watcher watcher, String path)
|
|
|
+ {
|
|
|
+ this.watches = watches;
|
|
|
+ this.watcher = watcher;
|
|
|
+ this.path = path;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Register the watcher with the set of watches on path.
|
|
|
+ * @param rc the result code of the operation that attempted to
|
|
|
+ * add the watch on the path.
|
|
|
+ */
|
|
|
+ public void register(int rc) {
|
|
|
+ if (shouldAddWatch(rc)) {
|
|
|
+ synchronized(watches) {
|
|
|
+ Set<Watcher> watchers = watches.get(path);
|
|
|
+ if (watchers == null) {
|
|
|
+ watchers = new HashSet<Watcher>();
|
|
|
+ watches.put(path, watchers);
|
|
|
+ }
|
|
|
+ watchers.add(watcher);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ /**
|
|
|
+ * Determine whether the watch should be added based on return code.
|
|
|
+ * @param rc the result code of the operation that attempted to add the
|
|
|
+ * watch on the node
|
|
|
+ * @return true if the watch should be added, otw false
|
|
|
+ */
|
|
|
+ protected boolean shouldAddWatch(int rc) {
|
|
|
+ return rc == 0;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Handle the special case of exists watches - they add a watcher
|
|
|
+ * even in the case where NONODE result code is returned.
|
|
|
+ */
|
|
|
+ class ExistsWatchRegistration extends WatchRegistration {
|
|
|
+ public ExistsWatchRegistration(Map<String, Set<Watcher>> watches,
|
|
|
+ Watcher watcher, String path)
|
|
|
+ {
|
|
|
+ super(watches, watcher, path);
|
|
|
+ }
|
|
|
+ protected boolean shouldAddWatch(int rc) {
|
|
|
+ return rc == 0 || rc == KeeperException.Code.NoNode;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
public enum States {
|
|
|
CONNECTING, ASSOCIATING, CONNECTED, CLOSED, AUTH_FAILED;
|
|
@@ -124,13 +265,13 @@ public class ZooKeeper {
|
|
|
|
|
|
public ZooKeeper(String host, int sessionTimeout, Watcher watcher)
|
|
|
throws IOException {
|
|
|
- this.watcher = watcher;
|
|
|
+ this.defaultWatcher = watcher;
|
|
|
cnxn = new ClientCnxn(host, sessionTimeout, this);
|
|
|
}
|
|
|
|
|
|
public ZooKeeper(String host, int sessionTimeout, Watcher watcher,
|
|
|
long sessionId, byte[] sessionPasswd) throws IOException {
|
|
|
- this.watcher = watcher;
|
|
|
+ this.defaultWatcher = watcher;
|
|
|
cnxn = new ClientCnxn(host, sessionTimeout, this, sessionId,
|
|
|
sessionPasswd);
|
|
|
}
|
|
@@ -152,7 +293,7 @@ public class ZooKeeper {
|
|
|
}
|
|
|
|
|
|
public synchronized void register(Watcher watcher) {
|
|
|
- this.watcher = watcher;
|
|
|
+ this.defaultWatcher = watcher;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -169,7 +310,7 @@ public class ZooKeeper {
|
|
|
public synchronized void close() throws InterruptedException {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.closeSession);
|
|
|
- cnxn.submitRequest(h, null, null);
|
|
|
+ cnxn.submitRequest(h, null, null, null);
|
|
|
try {
|
|
|
cnxn.close();
|
|
|
} catch (IOException e) {
|
|
@@ -240,7 +381,7 @@ public class ZooKeeper {
|
|
|
throw new KeeperException.InvalidACLException();
|
|
|
}
|
|
|
request.setAcl(acl);
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, null);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr(), path);
|
|
|
}
|
|
@@ -265,7 +406,7 @@ public class ZooKeeper {
|
|
|
request.setFlags(flags);
|
|
|
request.setPath(path);
|
|
|
request.setAcl(acl);
|
|
|
- cnxn.queuePacket(h, r, request, response, cb, path, ctx);
|
|
|
+ cnxn.queuePacket(h, r, request, response, cb, path, ctx, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -300,7 +441,7 @@ public class ZooKeeper {
|
|
|
DeleteRequest request = new DeleteRequest();
|
|
|
request.setPath(path);
|
|
|
request.setVersion(version);
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, null);
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, null, null);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
@@ -318,93 +459,137 @@ public class ZooKeeper {
|
|
|
DeleteRequest request = new DeleteRequest();
|
|
|
request.setPath(path);
|
|
|
request.setVersion(version);
|
|
|
- cnxn.queuePacket(h, new ReplyHeader(), request, null, cb, path, ctx);
|
|
|
+ cnxn.queuePacket(h, new ReplyHeader(), request, null, cb, path, ctx, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Return the stat of the node of the given path. Return null if no such a
|
|
|
* node exists.
|
|
|
* <p>
|
|
|
- * If the watch is true and the call is successful (no exception is thrown),
|
|
|
+ * If the watch is non-null and the call is successful (no exception is thrown),
|
|
|
* a watch will be left on the node with the given path. The watch will be
|
|
|
* triggered by a successful operation that creates/delete the node or sets
|
|
|
* the data on the node.
|
|
|
*
|
|
|
- * @param path
|
|
|
- * the node path
|
|
|
- * @param watch
|
|
|
- * whether need to watch this node
|
|
|
+ * @param path the node path
|
|
|
+ * @param watcher explicit watcher
|
|
|
* @return the stat of the node of the given path; return null if no such a
|
|
|
* node exists.
|
|
|
* @throws KeeperException If the server signals an error
|
|
|
* @throws InterruptedException If the server transaction is interrupted.
|
|
|
*/
|
|
|
- public Stat exists(String path, boolean watch) throws KeeperException,
|
|
|
- InterruptedException {
|
|
|
+ public Stat exists(String path, Watcher watcher) throws KeeperException,
|
|
|
+ InterruptedException
|
|
|
+ {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.exists);
|
|
|
ExistsRequest request = new ExistsRequest();
|
|
|
request.setPath(path);
|
|
|
- request.setWatch(watch);
|
|
|
+ request.setWatch(watcher != null);
|
|
|
SetDataResponse response = new SetDataResponse();
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ WatchRegistration wcb = null;
|
|
|
+ if (watcher != null) {
|
|
|
+ wcb = new ExistsWatchRegistration(dataWatches, watcher, path);
|
|
|
+ }
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
|
|
|
if (r.getErr() != 0) {
|
|
|
if (r.getErr() == KeeperException.Code.NoNode) {
|
|
|
return null;
|
|
|
}
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
|
+
|
|
|
return response.getStat().getCzxid() == -1 ? null : response.getStat();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Return the stat of the node of the given path. Return null if no such a
|
|
|
+ * node exists.
|
|
|
+ * <p>
|
|
|
+ * If the watch is true and the call is successful (no exception is thrown),
|
|
|
+ * a watch will be left on the node with the given path. The watch will be
|
|
|
+ * triggered by a successful operation that creates/delete the node or sets
|
|
|
+ * the data on the node.
|
|
|
+ *
|
|
|
+ * @param path
|
|
|
+ * the node path
|
|
|
+ * @param watch
|
|
|
+ * whether need to watch this node
|
|
|
+ * @return the stat of the node of the given path; return null if no such a
|
|
|
+ * node exists.
|
|
|
+ * @throws KeeperException If the server signals an error
|
|
|
+ * @throws InterruptedException If the server transaction is interrupted.
|
|
|
+ */
|
|
|
+ public Stat exists(String path, boolean watch) throws KeeperException,
|
|
|
+ InterruptedException
|
|
|
+ {
|
|
|
+ return exists(path, watch ? defaultWatcher : null);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* The Asynchronous version of exists. The request doesn't actually until
|
|
|
* the asynchronous callback is called.
|
|
|
*
|
|
|
* @see #exists(String, boolean)
|
|
|
*/
|
|
|
- public void exists(String path, boolean watch, StatCallback cb, Object ctx) {
|
|
|
+ public void exists(String path, Watcher watcher, StatCallback cb,
|
|
|
+ Object ctx)
|
|
|
+ {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.exists);
|
|
|
ExistsRequest request = new ExistsRequest();
|
|
|
request.setPath(path);
|
|
|
- request.setWatch(watch);
|
|
|
+ request.setWatch(watcher != null);
|
|
|
SetDataResponse response = new SetDataResponse();
|
|
|
- cnxn
|
|
|
- .queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
- ctx);
|
|
|
+ WatchRegistration wcb = null;
|
|
|
+ if (watcher != null) {
|
|
|
+ wcb = new ExistsWatchRegistration(dataWatches, watcher, path);
|
|
|
+ }
|
|
|
+ cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
+ ctx, wcb);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The Asynchronous version of exists. The request doesn't actually until
|
|
|
+ * the asynchronous callback is called.
|
|
|
+ *
|
|
|
+ * @see #exists(String, boolean)
|
|
|
+ */
|
|
|
+ public void exists(String path, boolean watch, StatCallback cb, Object ctx) {
|
|
|
+ exists(path, watch ? defaultWatcher : null, cb, ctx);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Return the data and the stat of the node of the given path.
|
|
|
* <p>
|
|
|
- * If the watch is true and the call is successfull (no exception is
|
|
|
+ * If the watch is non-null and the call is successful (no exception is
|
|
|
* thrown), a watch will be left on the node with the given path. The watch
|
|
|
- * will be triggered by a sucessful operation that sets data on the node, or
|
|
|
+ * will be triggered by a successful operation that sets data on the node, or
|
|
|
* deletes the node.
|
|
|
* <p>
|
|
|
* A KeeperException with error code KeeperException.NoNode will be thrown
|
|
|
* if no node with the given path exists.
|
|
|
*
|
|
|
- * @param path
|
|
|
- * the given path
|
|
|
- * @param watch
|
|
|
- * whether need to watch this node
|
|
|
- * @param stat
|
|
|
- * teh stat of the node
|
|
|
+ * @param path the given path
|
|
|
+ * @param watcher explicit watcher
|
|
|
+ * @param stat the stat of the node
|
|
|
* @return the data of the node
|
|
|
* @throws KeeperException If the server signals an error with a non-zero error code
|
|
|
* @throws InterruptedException If the server transaction is interrupted.
|
|
|
*/
|
|
|
- public byte[] getData(String path, boolean watch, Stat stat)
|
|
|
+ public byte[] getData(String path, Watcher watcher, Stat stat)
|
|
|
throws KeeperException, InterruptedException {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.getData);
|
|
|
GetDataRequest request = new GetDataRequest();
|
|
|
request.setPath(path);
|
|
|
- request.setWatch(watch);
|
|
|
+ request.setWatch(watcher != null);
|
|
|
GetDataResponse response = new GetDataResponse();
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ WatchRegistration wcb = null;
|
|
|
+ if (watcher != null) {
|
|
|
+ wcb = new WatchRegistration(dataWatches, watcher, path);
|
|
|
+ }
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
@@ -414,23 +599,58 @@ public class ZooKeeper {
|
|
|
return response.getData();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Return the data and the stat of the node of the given path.
|
|
|
+ * <p>
|
|
|
+ * If the watch is true and the call is successful (no exception is
|
|
|
+ * thrown), a watch will be left on the node with the given path. The watch
|
|
|
+ * will be triggered by a successful operation that sets data on the node, or
|
|
|
+ * deletes the node.
|
|
|
+ * <p>
|
|
|
+ * A KeeperException with error code KeeperException.NoNode will be thrown
|
|
|
+ * if no node with the given path exists.
|
|
|
+ *
|
|
|
+ * @param path the given path
|
|
|
+ * @param watch whether need to watch this node
|
|
|
+ * @param stat the stat of the node
|
|
|
+ * @return the data of the node
|
|
|
+ * @throws KeeperException If the server signals an error with a non-zero error code
|
|
|
+ * @throws InterruptedException If the server transaction is interrupted.
|
|
|
+ */
|
|
|
+ public byte[] getData(String path, boolean watch, Stat stat)
|
|
|
+ throws KeeperException, InterruptedException {
|
|
|
+ return getData(path, watch ? defaultWatcher : null, stat);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* The Asynchronous version of getData. The request doesn't actually until
|
|
|
* the asynchronous callback is called.
|
|
|
*
|
|
|
- * @see #getData(String, boolean, Stat)
|
|
|
+ * @see #getData(String, Watcher, Stat)
|
|
|
*/
|
|
|
-
|
|
|
- public void getData(String path, boolean watch, DataCallback cb, Object ctx) {
|
|
|
+ public void getData(String path, Watcher watcher, DataCallback cb, Object ctx) {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.getData);
|
|
|
GetDataRequest request = new GetDataRequest();
|
|
|
request.setPath(path);
|
|
|
- request.setWatch(watch);
|
|
|
+ request.setWatch(watcher != null);
|
|
|
GetDataResponse response = new GetDataResponse();
|
|
|
- cnxn
|
|
|
- .queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
- ctx);
|
|
|
+ WatchRegistration wcb = null;
|
|
|
+ if (watcher != null) {
|
|
|
+ wcb = new WatchRegistration(dataWatches, watcher, path);
|
|
|
+ }
|
|
|
+ cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
+ ctx, wcb);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The Asynchronous version of getData. The request doesn't actually until
|
|
|
+ * the asynchronous callback is called.
|
|
|
+ *
|
|
|
+ * @see #getData(String, boolean, Stat)
|
|
|
+ */
|
|
|
+ public void getData(String path, boolean watch, DataCallback cb, Object ctx) {
|
|
|
+ getData(path, watch ? defaultWatcher : null, cb, ctx);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -466,7 +686,7 @@ public class ZooKeeper {
|
|
|
request.setData(data);
|
|
|
request.setVersion(version);
|
|
|
SetDataResponse response = new SetDataResponse();
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, null);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
@@ -490,7 +710,7 @@ public class ZooKeeper {
|
|
|
SetDataResponse response = new SetDataResponse();
|
|
|
cnxn
|
|
|
.queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
- ctx);
|
|
|
+ ctx, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -515,7 +735,7 @@ public class ZooKeeper {
|
|
|
GetACLRequest request = new GetACLRequest();
|
|
|
request.setPath(path);
|
|
|
GetACLResponse response = new GetACLResponse();
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, null);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
@@ -537,7 +757,7 @@ public class ZooKeeper {
|
|
|
GetACLResponse response = new GetACLResponse();
|
|
|
cnxn
|
|
|
.queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
- ctx);
|
|
|
+ ctx, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -571,7 +791,7 @@ public class ZooKeeper {
|
|
|
request.setAcl(acl);
|
|
|
request.setVersion(version);
|
|
|
SetACLResponse response = new SetACLResponse();
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, null);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
@@ -595,57 +815,98 @@ public class ZooKeeper {
|
|
|
SetACLResponse response = new SetACLResponse();
|
|
|
cnxn
|
|
|
.queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
- ctx);
|
|
|
+ ctx, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Return the list of the children of the node of the given path.
|
|
|
* <p>
|
|
|
- * If the watch is true and the call is successful (no exception is thrown),
|
|
|
+ * If the watch is non-null and the call is successful (no exception is thrown),
|
|
|
* a watch will be left on the node with the given path. The watch willbe
|
|
|
- * triggered by a sucessful operation that deletes the node of the given
|
|
|
+ * triggered by a successful operation that deletes the node of the given
|
|
|
* path or creates/delete a child under the node.
|
|
|
* <p>
|
|
|
* A KeeperException with error code KeeperException.NoNode will be thrown
|
|
|
* if no node with the given path exists.
|
|
|
*
|
|
|
* @param path
|
|
|
- * @param watch
|
|
|
+ * @param watcher explicit watcher
|
|
|
* @return an array of children of the node with the given path
|
|
|
* @throws InterruptedException If the server transaction is interrupted.
|
|
|
* @throws KeeperException If the server signals an error with a non-zero error code.
|
|
|
*/
|
|
|
- public List<String> getChildren(String path, boolean watch)
|
|
|
+ public List<String> getChildren(String path, Watcher watcher)
|
|
|
throws KeeperException, InterruptedException {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.getChildren);
|
|
|
GetChildrenRequest request = new GetChildrenRequest();
|
|
|
request.setPath(path);
|
|
|
- request.setWatch(watch);
|
|
|
+ request.setWatch(watcher != null);
|
|
|
GetChildrenResponse response = new GetChildrenResponse();
|
|
|
- ReplyHeader r = cnxn.submitRequest(h, request, response);
|
|
|
+ WatchRegistration wcb = null;
|
|
|
+ if (watcher != null) {
|
|
|
+ wcb = new WatchRegistration(childWatches, watcher, path);
|
|
|
+ }
|
|
|
+ ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
|
|
|
if (r.getErr() != 0) {
|
|
|
throw KeeperException.create(r.getErr());
|
|
|
}
|
|
|
return response.getChildren();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Return the list of the children of the node of the given path.
|
|
|
+ * <p>
|
|
|
+ * If the watch is true and the call is successful (no exception is thrown),
|
|
|
+ * a watch will be left on the node with the given path. The watch willbe
|
|
|
+ * triggered by a successful operation that deletes the node of the given
|
|
|
+ * path or creates/delete a child under the node.
|
|
|
+ * <p>
|
|
|
+ * A KeeperException with error code KeeperException.NoNode will be thrown
|
|
|
+ * if no node with the given path exists.
|
|
|
+ *
|
|
|
+ * @param path
|
|
|
+ * @param watch
|
|
|
+ * @return an array of children of the node with the given path
|
|
|
+ * @throws InterruptedException If the server transaction is interrupted.
|
|
|
+ * @throws KeeperException If the server signals an error with a non-zero error code.
|
|
|
+ */
|
|
|
+ public List<String> getChildren(String path, boolean watch)
|
|
|
+ throws KeeperException, InterruptedException {
|
|
|
+ return getChildren(path, watch ? defaultWatcher : null);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* The Asynchronous version of getChildren. The request doesn't actually
|
|
|
* until the asynchronous callback is called.
|
|
|
*
|
|
|
- * @see #getChildren(String, boolean)
|
|
|
+ * @see #getChildren(String, Watcher)
|
|
|
*/
|
|
|
- public void getChildren(String path, boolean watch, ChildrenCallback cb,
|
|
|
+ public void getChildren(String path, Watcher watcher, ChildrenCallback cb,
|
|
|
Object ctx) {
|
|
|
RequestHeader h = new RequestHeader();
|
|
|
h.setType(ZooDefs.OpCode.getChildren);
|
|
|
GetChildrenRequest request = new GetChildrenRequest();
|
|
|
request.setPath(path);
|
|
|
- request.setWatch(watch);
|
|
|
+ request.setWatch(watcher != null);
|
|
|
GetChildrenResponse response = new GetChildrenResponse();
|
|
|
+ WatchRegistration wcb = null;
|
|
|
+ if (watcher != null) {
|
|
|
+ wcb = new WatchRegistration(childWatches, watcher, path);
|
|
|
+ }
|
|
|
cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
|
|
|
- ctx);
|
|
|
+ ctx, wcb);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The Asynchronous version of getChildren. The request doesn't actually
|
|
|
+ * until the asynchronous callback is called.
|
|
|
+ *
|
|
|
+ * @see #getChildren(String, boolean)
|
|
|
+ */
|
|
|
+ public void getChildren(String path, boolean watch, ChildrenCallback cb,
|
|
|
+ Object ctx) {
|
|
|
+ getChildren(path, watch ? defaultWatcher : null, cb, ctx);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -657,7 +918,8 @@ public class ZooKeeper {
|
|
|
SyncRequest request = new SyncRequest();
|
|
|
SyncResponse response = new SyncResponse();
|
|
|
request.setPath(path);
|
|
|
- cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path, ctx);
|
|
|
+ cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path, ctx,
|
|
|
+ null);
|
|
|
}
|
|
|
|
|
|
public States getState() {
|