Author: michim
Date: Wed Apr 16 06:15:28 2014
New Revision: 1587812
URL: http://svn.apache.org/r1587812
Log:
ZOOKEEPER-1887. C implementation of removeWatches (Raul Gutierrez Segales via michim)
Modified:
zookeeper/trunk/CHANGES.txt
zookeeper/trunk/src/c/include/proto.h
zookeeper/trunk/src/c/include/zookeeper.h
zookeeper/trunk/src/c/src/zk_hashtable.c
zookeeper/trunk/src/c/src/zk_hashtable.h
zookeeper/trunk/src/c/src/zookeeper.c
zookeeper/trunk/src/c/tests/TestClient.cc
Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Wed Apr 16 06:15:28 2014
@@ -26,6 +26,9 @@ NEW FEATURES:
ZOOKEEPER-1830. Support command line shell for removing watches
(Rakesh R via michim)
+ ZOOKEEPER-1887. C implementation of removeWatches (Raul Gutierrez Segales via
+ michim)
+
BUGFIXES:
ZOOKEEPER-786. Exception in ZooKeeper.toString
Modified: zookeeper/trunk/src/c/include/proto.h
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/include/proto.h?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/src/c/include/proto.h (original)
+++ zookeeper/trunk/src/c/include/proto.h Wed Apr 16 06:15:28 2014
@@ -38,6 +38,7 @@ extern "C" {
#define ZOO_MULTI_OP 14
#define ZOO_CREATE2_OP 15
#define ZOO_RECONFIG_OP 16
+#define ZOO_REMOVE_WATCHES 17
#define ZOO_CLOSE_OP -11
#define ZOO_SETAUTH_OP 100
#define ZOO_SETWATCHES_OP 101
Modified: zookeeper/trunk/src/c/include/zookeeper.h
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/include/zookeeper.h?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ zookeeper/trunk/src/c/include/zookeeper.h Wed Apr 16 06:15:28 2014
@@ -118,7 +118,8 @@ enum ZOO_ERRORS {
invoking reconfiguration after new servers are connected
and synced */
ZRECONFIGINPROGRESS = -121, /*!< Reconfiguration requested while another reconfiguration
is currently in progress. This is currently
not supported. Please retry. */
- ZEPHEMERALONLOCALSESSION = -122 /*!< Attempt to create ephemeral node on a local session
*/
+ ZEPHEMERALONLOCALSESSION = -122, /*!< Attempt to create ephemeral node on a local session
*/
+ ZNOWATCHER = -123 /*!< The watcher couldn't be found */
};
#ifdef __cplusplus
@@ -1991,6 +1992,62 @@ ZOOAPI int zoo_set_acl(zhandle_t *zh, co
*/
ZOOAPI int zoo_multi(zhandle_t *zh, int count, const zoo_op_t *ops, zoo_op_result_t *results);
+/**
+ * Type of watchers: used to select which type of watchers should be removed
+ */
+typedef enum {
+ ZWATCHERTYPE_CHILDREN = 1,
+ ZWATCHERTYPE_DATA = 2,
+ ZWATCHERTYPE_ANY = 3
+} ZooWatcherType;
+
+/**
+ * \brief removes the watchers for the given path and watcher type.
+ *
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the path for which watchers will be removed
+ * \param wtype the watcher type to be removed
+ * \param watcher the watcher to be removed, if null all watchers for that
+ * path (and watcher type) will be removed
+ * \param watcherCtx the contex associated with the watcher to be removed
+ * \param local whether the watchers will be removed locally even if there is
+ * no server connection
+ * \return the return code for the function call.
+ * ZOK - operation completed successfully
+ * ZNOWATCHER - the watcher couldn't be found.
+ * ZINVALIDSTATE - if !local, zhandle state is either ZOO_SESSION_EXPIRED_STATE
+ * or ZOO_AUTH_FAILED_STATE
+ * ZBADARGUMENTS - invalid input parameters
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ * ZSYSTEMERROR - a system error occured
+ */
+ZOOAPI int zoo_remove_watchers(zhandle_t *zh, const char *path,
+ ZooWatcherType wtype, watcher_fn watcher, void *watcherCtx, int local);
+
+/**
+ * \brief removes the watchers for the given path and watcher type.
+ *
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the path for which watchers will be removed
+ * \param wtype the watcher type to be removed
+ * \param watcher the watcher to be removed, if null all watchers for that
+ * path (and watcher type) will be removed
+ * \param watcherCtx the contex associated with the watcher to be removed
+ * \param local whether the watchers will be removed locally even if there is
+ * no server connection
+ * \return the return code for the function call.
+ * ZOK - operation completed successfully
+ * ZNOWATCHER - the watcher couldn't be found.
+ * ZINVALIDSTATE - if !local, zhandle state is either ZOO_SESSION_EXPIRED_STATE
+ * or ZOO_AUTH_FAILED_STATE
+ * ZBADARGUMENTS - invalid input parameters
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ * ZSYSTEMERROR - a system error occured
+ */
+ZOOAPI int zoo_aremove_watchers(zhandle_t *zh, const char *path,
+ ZooWatcherType wtype, watcher_fn watcher, void *watcherCtx, int local,
+ void_completion_t *completion, const void *data);
+
#ifdef __cplusplus
}
#endif
Modified: zookeeper/trunk/src/c/src/zk_hashtable.c
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/src/zk_hashtable.c?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/src/c/src/zk_hashtable.c (original)
+++ zookeeper/trunk/src/c/src/zk_hashtable.c Wed Apr 16 06:15:28 2014
@@ -335,3 +335,142 @@ void activateWatcher(zhandle_t *zh, watc
}
}
}
+
+/* If watcher is NULL, we return TRUE since we consider it a match */
+static int containsWatcher(zk_hashtable *watchers, const char *path,
+ watcher_fn watcher, void *watcherCtx)
+{
+ watcher_object_list_t *wl;
+ watcher_object_t e;
+
+ if (!watcher)
+ return 1;
+
+ wl = hashtable_search(watchers->ht, (void *)path);
+ if (!wl)
+ return 0;
+
+ e.watcher = watcher;
+ e.context = watcherCtx;
+
+ return search_watcher(&wl, &e) ? 1 : 0;
+}
+
+/**
+ * remove any watcher_object that has a matching (watcher, watcherCtx)
+ */
+static void removeWatcherFromList(watcher_object_list_t *wl, watcher_fn watcher,
+ void *watcherCtx)
+{
+ watcher_object_t *e = NULL;
+
+ if (!wl || (wl && !wl->head))
+ return;
+
+ e = wl->head;
+ while (e){
+ if (e->next &&
+ e->next->watcher == watcher &&
+ e->context == watcherCtx) {
+ watcher_object_t *this = e->next;
+ e->next = e->next->next;
+ free(this);
+ break;
+ }
+ e = e->next;
+ }
+
+ if (wl->head &&
+ wl->head->watcher == watcher && wl->head->context == watcherCtx)
{
+ watcher_object_t *this = wl->head;
+ wl->head = wl->head->next;
+ free(this);
+ }
+}
+
+static void removeWatcher(zk_hashtable *watchers, const char *path,
+ watcher_fn watcher, void *watcherCtx)
+{
+ watcher_object_list_t *wl = hashtable_search(watchers->ht, (void *)path);
+
+ if (!wl)
+ return;
+
+ if (!watcher) {
+ wl = (watcher_object_list_t *) hashtable_remove(watchers->ht,
+ (void *)path);
+ destroy_watcher_object_list(wl);
+ return;
+ }
+
+ removeWatcherFromList(wl, watcher, watcherCtx);
+
+ if (!wl->head) {
+ wl = (watcher_object_list_t *) hashtable_remove(watchers->ht,
+ (void *)path);
+ destroy_watcher_object_list(wl);
+ }
+}
+
+void deactivateWatcher(zhandle_t *zh, watcher_deregistration_t *dereg, int rc)
+{
+ if (rc != ZOK || !dereg)
+ return;
+
+ removeWatchers(zh, dereg->path, dereg->type, dereg->watcher,
+ dereg->context);
+}
+
+void removeWatchers(zhandle_t *zh, const char* path, ZooWatcherType type,
+ watcher_fn watcher, void *watcherCtx)
+{
+ switch (type) {
+ case ZWATCHERTYPE_CHILDREN:
+ removeWatcher(zh->active_child_watchers, path, watcher, watcherCtx);
+ break;
+ case ZWATCHERTYPE_DATA:
+ removeWatcher(zh->active_node_watchers, path, watcher, watcherCtx);
+ removeWatcher(zh->active_exist_watchers, path, watcher, watcherCtx);
+ break;
+ case ZWATCHERTYPE_ANY:
+ removeWatcher(zh->active_child_watchers, path, watcher, watcherCtx);
+ removeWatcher(zh->active_node_watchers, path, watcher, watcherCtx);
+ removeWatcher(zh->active_exist_watchers, path, watcher, watcherCtx);
+ break;
+ }
+}
+
+int pathHasWatcher(zhandle_t *zh, const char *path, int wtype,
+ watcher_fn watcher, void *watcherCtx)
+{
+ int watcher_found = 0;
+
+ switch (wtype) {
+ case ZWATCHERTYPE_CHILDREN:
+ watcher_found = containsWatcher(zh->active_child_watchers,
+ path, watcher, watcherCtx);
+ break;
+ case ZWATCHERTYPE_DATA:
+ watcher_found = containsWatcher(zh->active_node_watchers, path,
+ watcher, watcherCtx);
+ if (!watcher_found) {
+ watcher_found = containsWatcher(zh->active_exist_watchers, path,
+ watcher, watcherCtx);
+ }
+ break;
+ case ZWATCHERTYPE_ANY:
+ watcher_found = containsWatcher(zh->active_child_watchers, path,
+ watcher, watcherCtx);
+ if (!watcher_found) {
+ watcher_found = containsWatcher(zh->active_node_watchers, path,
+ watcher, watcherCtx);
+ }
+ if (!watcher_found) {
+ watcher_found = containsWatcher(zh->active_exist_watchers, path,
+ watcher, watcherCtx);
+ }
+ break;
+ }
+
+ return watcher_found;
+}
Modified: zookeeper/trunk/src/c/src/zk_hashtable.h
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/src/zk_hashtable.h?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/src/c/src/zk_hashtable.h (original)
+++ zookeeper/trunk/src/c/src/zk_hashtable.h Wed Apr 16 06:15:28 2014
@@ -48,6 +48,18 @@ typedef struct _watcher_registration {
const char* path;
} watcher_registration_t;
+/**
+ * A watcher deregistration gets temporarily stored with the completion entry until
+ * the server response comes back at which moment we can remove the watchers from
+ * the active watchers map.
+ */
+typedef struct _watcher_deregistration {
+ watcher_fn watcher;
+ void* context;
+ ZooWatcherType type;
+ const char* path;
+} watcher_deregistration_t;
+
zk_hashtable* create_zk_hashtable();
void destroy_zk_hashtable(zk_hashtable* ht);
@@ -59,8 +71,13 @@ char **collect_keys(zk_hashtable *ht, in
* active watchers (only if the checker allows to do so)
*/
void activateWatcher(zhandle_t *zh, watcher_registration_t* reg, int rc);
+ void deactivateWatcher(zhandle_t *zh, watcher_deregistration_t *dereg, int rc);
watcher_object_list_t *collectWatchers(zhandle_t *zh,int type, char *path);
void deliverWatchers(zhandle_t *zh, int type, int state, char *path, struct watcher_object_list
**list);
+ void removeWatchers(zhandle_t *zh, const char* path, ZooWatcherType type,
+ watcher_fn watcher, void *watcherCtx);
+ int pathHasWatcher(zhandle_t *zh, const char *path, int wtype,
+ watcher_fn watcher, void *watcherCtx);
#ifdef __cplusplus
}
Modified: zookeeper/trunk/src/c/src/zookeeper.c
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/src/zookeeper.c?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ zookeeper/trunk/src/c/src/zookeeper.c Wed Apr 16 06:15:28 2014
@@ -177,6 +177,7 @@ typedef struct _completion_list {
buffer_list_t *buffer;
struct _completion_list *next;
watcher_registration_t* watcher;
+ watcher_deregistration_t* watcher_deregistration;
} completion_list_t;
const char*err2string(int err);
@@ -191,9 +192,23 @@ static int deserialize_multi(zhandle_t *
static int add_completion(zhandle_t *zh, int xid, int completion_type,
const void *dc, const void *data, int add_to_front,
watcher_registration_t* wo, completion_head_t *clist);
+static int add_completion_deregistration(zhandle_t *zh, int xid,
+ int completion_type, const void *dc, const void *data,
+ int add_to_front, watcher_deregistration_t* wo,
+ completion_head_t *clist);
+static int do_add_completion(zhandle_t *zh, const void *dc, completion_list_t *c,
+ int add_to_front);
+
static completion_list_t* create_completion_entry(zhandle_t *zh, int xid, int completion_type,
const void *dc, const void *data, watcher_registration_t* wo,
completion_head_t *clist);
+static completion_list_t* create_completion_entry_deregistration(zhandle_t *zh,
+ int xid, int completion_type, const void *dc, const void *data,
+ watcher_deregistration_t* wo, completion_head_t *clist);
+static completion_list_t* do_create_completion_entry(zhandle_t *zh,
+ int xid, int completion_type, const void *dc, const void *data,
+ watcher_registration_t* wo, completion_head_t *clist,
+ watcher_deregistration_t* wdo);
static void destroy_completion_entry(completion_list_t* c);
static void queue_completion_nolock(completion_head_t *list, completion_list_t *c,
int add_to_front);
@@ -2670,6 +2685,7 @@ int zookeeper_process(zhandle_t *zh, int
zh->last_zxid = hdr.zxid;
}
activateWatcher(zh, cptr->watcher, rc);
+ deactivateWatcher(zh, cptr->watcher_deregistration, rc);
if (cptr->c.void_result != SYNCHRONOUS_MARKER) {
if(hdr.xid == PING_XID){
@@ -2730,6 +2746,21 @@ static watcher_registration_t* create_wa
return wo;
}
+static watcher_deregistration_t* create_watcher_deregistration(const char* path,
+ watcher_fn watcher, void *watcherCtx, ZooWatcherType wtype) {
+ watcher_deregistration_t *wdo;
+
+ wdo = calloc(1, sizeof(watcher_deregistration_t));
+ if (!wdo) {
+ return NULL;
+ }
+ wdo->path = strdup(path);
+ wdo->watcher = watcher;
+ wdo->context = watcherCtx;
+ wdo->type = wtype;
+ return wdo;
+}
+
static void destroy_watcher_registration(watcher_registration_t* wo){
if(wo!=0){
free((void*)wo->path);
@@ -2737,10 +2768,34 @@ static void destroy_watcher_registration
}
}
+static void destroy_watcher_deregistration(watcher_deregistration_t *wdo) {
+ if (wdo) {
+ free((void *)wdo->path);
+ free(wdo);
+ }
+}
+
static completion_list_t* create_completion_entry(zhandle_t *zh, int xid, int completion_type,
const void *dc, const void *data,watcher_registration_t* wo, completion_head_t *clist)
{
- completion_list_t *c = calloc(1,sizeof(completion_list_t));
+ return do_create_completion_entry(zh, xid, completion_type, dc, data, wo,
+ clist, NULL);
+}
+
+static completion_list_t* create_completion_entry_deregistration(zhandle_t *zh,
+ int xid, int completion_type, const void *dc, const void *data,
+ watcher_deregistration_t* wdo, completion_head_t *clist)
+{
+ return do_create_completion_entry(zh, xid, completion_type, dc, data, NULL,
+ clist, wdo);
+}
+
+static completion_list_t* do_create_completion_entry(zhandle_t *zh, int xid,
+ int completion_type, const void *dc, const void *data,
+ watcher_registration_t* wo, completion_head_t *clist,
+ watcher_deregistration_t* wdo)
+{
+ completion_list_t *c = calloc(1, sizeof(completion_list_t));
if (!c) {
LOG_ERROR(LOGCALLBACK(zh), "out of memory");
return 0;
@@ -2779,6 +2834,7 @@ static completion_list_t* create_complet
}
c->xid = xid;
c->watcher = wo;
+ c->watcher_deregistration = wdo;
return c;
}
@@ -2786,6 +2842,7 @@ static completion_list_t* create_complet
static void destroy_completion_entry(completion_list_t* c){
if(c!=0){
destroy_watcher_registration(c->watcher);
+ destroy_watcher_deregistration(c->watcher_deregistration);
if(c->buffer!=0)
free_buffer(c->buffer);
free(c);
@@ -2831,6 +2888,21 @@ static int add_completion(zhandle_t *zh,
{
completion_list_t *c =create_completion_entry(zh, xid, completion_type, dc,
data, wo, clist);
+ return do_add_completion(zh, dc, c, add_to_front);
+}
+
+static int add_completion_deregistration(zhandle_t *zh, int xid,
+ int completion_type, const void *dc, const void *data, int add_to_front,
+ watcher_deregistration_t* wdo, completion_head_t *clist)
+{
+ completion_list_t *c = create_completion_entry_deregistration(zh, xid,
+ completion_type, dc, data, wdo, clist);
+ return do_add_completion(zh, dc, c, add_to_front);
+}
+
+static int do_add_completion(zhandle_t *zh, const void *dc,
+ completion_list_t *c, int add_to_front)
+{
int rc = 0;
if (!c)
return ZSYSTEMERROR;
@@ -4374,3 +4446,94 @@ int zoo_set_acl(zhandle_t *zh, const cha
free_sync_completion(sc);
return rc;
}
+
+int zoo_remove_watchers(zhandle_t *zh, const char *path, ZooWatcherType wtype,
+ watcher_fn watcher, void *watcherCtx, int local)
+{
+ struct sync_completion *sc;
+ int rc = 0;
+
+ if (!path)
+ return ZBADARGUMENTS;
+
+ sc = alloc_sync_completion();
+ if (!sc)
+ return ZSYSTEMERROR;
+
+ rc = zoo_aremove_watchers(zh, path, wtype, watcher, watcherCtx, local,
+ SYNCHRONOUS_MARKER, sc);
+ if (rc == ZOK) {
+ wait_sync_completion(sc);
+ rc = sc->rc;
+ }
+ free_sync_completion(sc);
+ return rc;
+}
+
+int zoo_aremove_watchers(zhandle_t *zh, const char *path, ZooWatcherType wtype,
+ watcher_fn watcher, void *watcherCtx, int local,
+ void_completion_t *completion, const void *data)
+{
+ char *server_path = prepend_string(zh, path);
+ int rc;
+ struct oarchive *oa;
+ struct RequestHeader h = { get_xid(), ZOO_REMOVE_WATCHES };
+ struct RemoveWatchesRequest req = { (char*)server_path, wtype };
+ watcher_deregistration_t *wdo;
+
+ if (!zh || !isValidPath(server_path, 0)) {
+ rc = ZBADARGUMENTS;
+ goto done;
+ }
+
+ if (!local && is_unrecoverable(zh)) {
+ rc = ZINVALIDSTATE;
+ goto done;
+ }
+
+ if (!pathHasWatcher(zh, server_path, wtype, watcher, watcherCtx)) {
+ rc = ZNOWATCHER;
+ goto done;
+ }
+
+ if (local) {
+ removeWatchers(zh, server_path, wtype, watcher, watcherCtx);
+ notify_sync_completion((struct sync_completion *)data);
+ rc = ZOK;
+ goto done;
+ }
+
+ oa = create_buffer_oarchive();
+ rc = serialize_RequestHeader(oa, "header", &h);
+ rc = rc < 0 ? rc : serialize_RemoveWatchesRequest(oa, "req", &req);
+ if (rc < 0) {
+ goto done;
+ }
+
+ wdo = create_watcher_deregistration(server_path, watcher, watcherCtx,
+ wtype);
+ if (!wdo) {
+ rc = ZSYSTEMERROR;
+ goto done;
+ }
+
+ enter_critical(zh);
+ rc = add_completion_deregistration(zh, h.xid, COMPLETION_VOID,
+ completion, data, 0, wdo, 0);
+ rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
+ get_buffer_len(oa));
+ rc = rc < 0 ? ZMARSHALLINGERROR : ZOK;
+ leave_critical(zh);
+
+ /* We queued the buffer, so don't free it */
+ close_buffer_oarchive(&oa, 0);
+
+ LOG_DEBUG(LOGCALLBACK(zh), "Sending request xid=%#x for path [%s] to %s",
+ h.xid, path, zoo_get_current_server(zh));
+
+ adaptor_send_queue(zh, 0);
+
+done:
+ free_duplicate_path(server_path, path);
+ return rc;
+}
Modified: zookeeper/trunk/src/c/tests/TestClient.cc
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/tests/TestClient.cc?rev=1587812&r1=1587811&r2=1587812&view=diff
==============================================================================
--- zookeeper/trunk/src/c/tests/TestClient.cc (original)
+++ zookeeper/trunk/src/c/tests/TestClient.cc Wed Apr 16 06:15:28 2014
@@ -217,6 +217,7 @@ class Zookeeper_simpleSystem : public CP
CPPUNIT_TEST(testWatcherAutoResetWithLocal);
CPPUNIT_TEST(testGetChildren2);
CPPUNIT_TEST(testLastZxid);
+ CPPUNIT_TEST(testRemoveWatchers);
#endif
CPPUNIT_TEST_SUITE_END();
@@ -1298,6 +1299,82 @@ public:
// zk2's last zxid should remain the same
CPPUNIT_ASSERT_EQUAL(updated, (int64_t) zk2->last_zxid);
}
+
+ static void watcher_remove_watchers(zhandle_t *zh, int type,
+ int state, const char *path,void *watcherCtx) {
+ count++;
+ }
+
+ void testRemoveWatchers() {
+ int rc;
+ watchctx_t ctx;
+ zhandle_t *zk = createClient(&ctx);
+ CPPUNIT_ASSERT(zk);
+
+ count = 0;
+
+ rc = zoo_create(zk, "/something", "", 0,
+ &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+ char buf[1024];
+ int blen = sizeof(buf);
+ rc = zoo_get(zk, "/something", 1, buf, &blen, NULL);
+
+ /* remove all watchers */
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_DATA,
+ NULL, NULL, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+ /* remove a specific watcher before it's added (should fail) */
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_DATA,
+ watcher_remove_watchers, NULL, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZNOWATCHER, rc);
+
+ /* now add a specific watcher and then remove it */
+ rc = zoo_wget(zk, "/something", watcher_remove_watchers, NULL,
+ buf, &blen, NULL);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_DATA,
+ watcher_remove_watchers, NULL, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+ /* ditto for children watcher */
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_CHILDREN,
+ watcher_remove_watchers, NULL, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZNOWATCHER, rc);
+
+ struct String_vector str_vec = {0, NULL};
+ rc = zoo_wget_children(zk, "/something", watcher_remove_watchers, NULL,
+ &str_vec);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_CHILDREN,
+ watcher_remove_watchers, NULL, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+ /* add a watch, stop the server, and have remove fail */
+ rc = zoo_wget(zk, "/something", watcher_remove_watchers, NULL,
+ buf, &blen, NULL);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+ stopServer();
+ ctx.waitForDisconnected(zk);
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_DATA,
+ watcher_remove_watchers, NULL, 0);
+ CPPUNIT_ASSERT_EQUAL((int)ZCONNECTIONLOSS, rc);
+
+ /* bring the server back */
+ startServer();
+ zk = createClient(&ctx);
+
+ /* add a watch, stop the server, and remove it locally */
+ rc = zoo_wget(zk, "/something", watcher_remove_watchers, NULL,
+ buf, &blen, NULL);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+ stopServer();
+ rc = zoo_remove_watchers(zk, "/something", ZWATCHERTYPE_DATA,
+ watcher_remove_watchers, NULL, 1);
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+ }
};
volatile int Zookeeper_simpleSystem::count;
|