Browse Source

[ZOOKEEPER-39] Use Watcher objects rather than boolean on read operations

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@679557 13f79535-47bb-0310-9956-ffa450edef68
Andrew Kornev 17 years ago
parent
commit
2bba284640

+ 17 - 8
src/c/Makefile.am

@@ -10,29 +10,37 @@ LIB_LDFLAGS = -no-undefined -version-info 2
 pkginclude_HEADERS = include/zookeeper.h include/recordio.h generated/zookeeper.jute.h
 pkginclude_HEADERS = include/zookeeper.h include/recordio.h generated/zookeeper.jute.h
 EXTRA_DIST=LICENSE
 EXTRA_DIST=LICENSE
 
 
+HASHTABLE_SRC = src/hashtable/hashtable_itr.h src/hashtable/hashtable_itr.c \
+    src/hashtable/hashtable_private.h src/hashtable/hashtable.h src/hashtable/hashtable.c
+
+noinst_LTLIBRARIES = libhashtable.la
+libhashtable_la_SOURCES = $(HASHTABLE_SRC)
+
 COMMON_SRC = src/zookeeper.c include/zookeeper.h \
 COMMON_SRC = src/zookeeper.c include/zookeeper.h \
     src/recordio.c include/recordio.h include/proto.h \
     src/recordio.c include/recordio.h include/proto.h \
     src/zk_adaptor.h generated/zookeeper.jute.c \
     src/zk_adaptor.h generated/zookeeper.jute.c \
-    src/zk_log.h src/zk_log.c
+    src/zk_log.h src/zk_log.c src/zk_hashtable.h src/zk_hashtable.c
 
 
-noinst_LTLIBRARIES = libzkst.la
+noinst_LTLIBRARIES += libzkst.la
 libzkst_la_SOURCES =$(COMMON_SRC) src/st_adaptor.c
 libzkst_la_SOURCES =$(COMMON_SRC) src/st_adaptor.c
+libzkst_la_LIBADD = -lm
 
 
 lib_LTLIBRARIES = libzookeeper_st.la
 lib_LTLIBRARIES = libzookeeper_st.la
 libzookeeper_st_la_SOURCES =
 libzookeeper_st_la_SOURCES =
-libzookeeper_st_la_LIBADD=libzkst.la
-libzookeeper_st_la_DEPENDENCIES=libzkst.la
+libzookeeper_st_la_LIBADD=libzkst.la libhashtable.la
+libzookeeper_st_la_DEPENDENCIES=libzkst.la libhashtable.la
 libzookeeper_st_la_LDFLAGS = $(LIB_LDFLAGS)
 libzookeeper_st_la_LDFLAGS = $(LIB_LDFLAGS)
 
 
 if WANT_SYNCAPI
 if WANT_SYNCAPI
 noinst_LTLIBRARIES += libzkmt.la
 noinst_LTLIBRARIES += libzkmt.la
 libzkmt_la_SOURCES =$(COMMON_SRC) src/mt_adaptor.c
 libzkmt_la_SOURCES =$(COMMON_SRC) src/mt_adaptor.c
 libzkmt_la_CFLAGS = -DTHREADED
 libzkmt_la_CFLAGS = -DTHREADED
+libzkmt_la_LIBADD = -lm
 
 
 lib_LTLIBRARIES += libzookeeper_mt.la
 lib_LTLIBRARIES += libzookeeper_mt.la
 libzookeeper_mt_la_SOURCES =
 libzookeeper_mt_la_SOURCES =
-libzookeeper_mt_la_LIBADD=libzkmt.la -lpthread
-libzookeeper_mt_la_DEPENDENCIES=libzkmt.la
+libzookeeper_mt_la_LIBADD=libzkmt.la libhashtable.la -lpthread
+libzookeeper_mt_la_DEPENDENCIES=libzkmt.la libhashtable.la
 libzookeeper_mt_la_LDFLAGS = $(LIB_LDFLAGS)
 libzookeeper_mt_la_LDFLAGS = $(LIB_LDFLAGS)
 endif
 endif
 
 
@@ -62,20 +70,21 @@ EXTRA_DIST+=$(wildcard tests/*.cc) $(wildcard tests/*.h) \
 
 
 TEST_SOURCES = tests/TestDriver.cc tests/LibCMocks.cc tests/LibCSymTable.cc \
 TEST_SOURCES = tests/TestDriver.cc tests/LibCMocks.cc tests/LibCSymTable.cc \
     tests/MocksBase.cc  tests/ZKMocks.cc tests/Util.cc tests/ThreadingUtil.cc \
     tests/MocksBase.cc  tests/ZKMocks.cc tests/Util.cc tests/ThreadingUtil.cc \
+    tests/TestWatchers.cc tests/TestHashtable.cc \
     tests/TestOperations.cc tests/TestZookeeperInit.cc tests/TestZookeeperClose.cc
     tests/TestOperations.cc tests/TestZookeeperInit.cc tests/TestZookeeperClose.cc
 
 
 SYMBOL_WRAPPERS=$(shell cat tests/wrappers.opt)
 SYMBOL_WRAPPERS=$(shell cat tests/wrappers.opt)
 
 
 check_PROGRAMS = zktest-st
 check_PROGRAMS = zktest-st
 nodist_zktest_st_SOURCES = $(TEST_SOURCES)
 nodist_zktest_st_SOURCES = $(TEST_SOURCES)
-zktest_st_LDADD = libzkst.la $(CPPUNIT_LIBS)
+zktest_st_LDADD = libzkst.la libhashtable.la $(CPPUNIT_LIBS)
 zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS)
 zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS)
 zktest_st_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS)
 zktest_st_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS)
 
 
 if WANT_SYNCAPI
 if WANT_SYNCAPI
   check_PROGRAMS += zktest-mt
   check_PROGRAMS += zktest-mt
   nodist_zktest_mt_SOURCES = $(TEST_SOURCES) tests/PthreadMocks.cc
   nodist_zktest_mt_SOURCES = $(TEST_SOURCES) tests/PthreadMocks.cc
-  zktest_mt_LDADD = libzkmt.la -lpthread $(CPPUNIT_LIBS)
+  zktest_mt_LDADD = libzkmt.la libhashtable.la -lpthread $(CPPUNIT_LIBS)
   zktest_mt_CXXFLAGS = -DUSE_STATIC_LIB -DTHREADED $(CPPUNIT_CFLAGS)
   zktest_mt_CXXFLAGS = -DUSE_STATIC_LIB -DTHREADED $(CPPUNIT_CFLAGS)
   SYMBOL_WRAPPERS_MT=$(SYMBOL_WRAPPERS) $(shell cat tests/wrappers-mt.opt)
   SYMBOL_WRAPPERS_MT=$(SYMBOL_WRAPPERS) $(shell cat tests/wrappers-mt.opt)
   zktest_mt_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS_MT)
   zktest_mt_LDFLAGS = -static-libtool-libs $(SYMBOL_WRAPPERS_MT)

+ 205 - 13
src/c/include/zookeeper.h

@@ -231,10 +231,32 @@ typedef struct {
 /**
 /**
  * \brief signature of a watch function.
  * \brief signature of a watch function.
  * 
  * 
- * Programs wishing to receive events from ZooKeeper need to implement a function 
- * with this signature and pass a pointer to the function to \ref zookeeper_init.
+ * There are two ways to receive watch notifications: legacy and watcher object.
+ * <p>
+ * The legacy style, an application wishing to receive events from ZooKeeper must 
+ * first implement a function with this signature and pass a pointer to the function 
+ * to \ref zookeeper_init. Next, the application sets a watch by calling one of 
+ * the getter API that accept the watch integer flag (for example, \ref zoo_aexists, 
+ * \ref zoo_get, etc).
+ * <p>
+ * The watcher object style uses an instance of a "watcher object" which in 
+ * the C world is represented by a pair: a pointer to a function implementing this
+ * signature and a pointer to watcher context -- handback user-specific data. 
+ * When a watch is triggered this function will be called along with 
+ * the watcher context. An application wishing to use this style must use
+ * the getter API functions with the "w" prefix in their names (for example, \ref
+ * zoo_awexists, \ref zoo_wget, etc).
+ * 
+ * \param zh zookeeper handle
+ * \param type event type. This is one of the *_EVENT constants. 
+ * \param state connection state. If the type is SESSION_EVENT, the state value 
+ * will be one of the *_STATE constants, otherwise -1.
+ * \param path znode path for which the watcher is triggered. NULL if the event 
+ * type is SESSION_EVENT
+ * \param watcherCtx watcher context.
  */
  */
-typedef void (*watcher_fn)(zhandle_t *, int type, int state, const char *path);
+typedef void (*watcher_fn)(zhandle_t *zh, int type, 
+        int state, const char *path,void *watcherCtx);
 
 
 /**
 /**
  * \brief create a handle to used communicate with zookeeper.
  * \brief create a handle to used communicate with zookeeper.
@@ -243,7 +265,7 @@ typedef void (*watcher_fn)(zhandle_t *, int type, int state, const char *path);
  * to that handle.
  * to that handle.
  * \param host the host name to connect to. This may be a comma separated list
  * \param host the host name to connect to. This may be a comma separated list
  *   of different hosts.
  *   of different hosts.
- * \param fn the watcher callback function. When notifications are triggered
+ * \param fn the global watcher callback function. When notifications are triggered
  *   this function will be invoked.
  *   this function will be invoked.
  * \param clientid the id of a previously established session that this
  * \param clientid the id of a previously established session that this
  *   client will be reconnecting to. Pass 0 if not reconnecting to a previous
  *   client will be reconnecting to. Pass 0 if not reconnecting to a previous
@@ -578,6 +600,39 @@ ZOOAPI int zoo_adelete(zhandle_t *zh, const char *path, int version,
 ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch, 
 ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch, 
         stat_completion_t completion, const void *data);
         stat_completion_t completion, const void *data);
 
 
+/**
+ * \brief checks the existence of a node in zookeeper.
+ * 
+ * This function is similar to \ref zoo_axists except it allows one specify 
+ * a watcher object - a function pointer and associated context. The function
+ * will be called once the watch has fired. The associated context data will be 
+ * passed to the function as the watcher context parameter. 
+ * 
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the name of the node. Expressed as a file name with slashes 
+ * separating ancestors of the node.
+ * \param watcher if non-null a watch will set on the specified znode on the server.
+ * The watch will be set even if the node does not exist. This allows clients 
+ * to watch for nodes to appear.
+ * \param watcherCtx user specific data, will be passed to the watcher callback.
+ * Unlike the global context set by \ref zookeeper_init, this watcher context
+ * is associated with the given instance of the watcher only.
+ * \param completion the routine to invoke when the request completes. The completion
+ * will be triggered with one of the following codes passed in as the rc argument:
+ * ZOK operation completed succesfully
+ * ZNONODE the node does not exist.
+ * ZNOAUTH the client does not have permission.
+ * \param data the data that will be passed to the completion routine when the 
+ * function completes.
+ * \return ZOK on success or one of the following errcodes on failure:
+ * ZBADARGUMENTS - invalid input parameters
+ * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_awexists(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx, 
+        stat_completion_t completion, const void *data);
+
 /**
 /**
  * \brief gets the data associated with a node.
  * \brief gets the data associated with a node.
  * 
  * 
@@ -601,6 +656,36 @@ ZOOAPI int zoo_aexists(zhandle_t *zh, const char *path, int watch,
 ZOOAPI int zoo_aget(zhandle_t *zh, const char *path, int watch, 
 ZOOAPI int zoo_aget(zhandle_t *zh, const char *path, int watch, 
         data_completion_t completion, const void *data);
         data_completion_t completion, const void *data);
 
 
+/**
+ * \brief gets the data associated with a node.
+ * 
+ * This function is similar to \ref zoo_aget except it allows one specify 
+ * a watcher object rather than a boolean watch flag. 
+ *
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the name of the node. Expressed as a file name with slashes 
+ * separating ancestors of the node.
+ * \param watcher if non-null, a watch will be set at the server to notify 
+ * the client if the node changes.
+ * \param watcherCtx user specific data, will be passed to the watcher callback.
+ * Unlike the global context set by \ref zookeeper_init, this watcher context
+ * is associated with the given instance of the watcher only.
+ * \param completion the routine to invoke when the request completes. The completion
+ * will be triggered with one of the following codes passed in as the rc argument:
+ * ZOK operation completed succesfully
+ * ZNONODE the node does not exist.
+ * ZNOAUTH the client does not have permission.
+ * \param data the data that will be passed to the completion routine when 
+ * the function completes.
+ * \return ZOK on success or one of the following errcodes on failure:
+ * ZBADARGUMENTS - invalid input parameters
+ * ZINVALIDSTATE - zhandle state is either in SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_awget(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx, 
+        data_completion_t completion, const void *data);
+
 /**
 /**
  * \brief sets the data associated with a node.
  * \brief sets the data associated with a node.
  * 
  * 
@@ -653,6 +738,36 @@ ZOOAPI int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buf
 ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch, 
 ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch, 
         strings_completion_t completion, const void *data);
         strings_completion_t completion, const void *data);
 
 
+/**
+ * \brief lists the children of a node.
+ * 
+ * This function is similar to \ref zoo_aget_children except it allows one specify 
+ * a watcher object rather than a boolean watch flag.
+ *  
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the name of the node. Expressed as a file name with slashes 
+ * separating ancestors of the node.
+ * \param watcher if non-null, a watch will be set at the server to notify 
+ * the client if the node changes.
+ * \param watcherCtx user specific data, will be passed to the watcher callback.
+ * Unlike the global context set by \ref zookeeper_init, this watcher context
+ * is associated with the given instance of the watcher only.
+ * \param completion the routine to invoke when the request completes. The completion
+ * will be triggered with one of the following codes passed in as the rc argument:
+ * ZOK operation completed succesfully
+ * ZNONODE the node does not exist.
+ * ZNOAUTH the client does not have permission.
+ * \param data the data that will be passed to the completion routine when 
+ * the function completes.
+ * \return ZOK on success or one of the following errcodes on failure:
+ * ZBADARGUMENTS - invalid input parameters
+ * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_awget_children(zhandle_t *zh, const char *path,
+        watcher_fn watcher, void* watcherCtx, 
+        strings_completion_t completion, const void *data);
+
 /**
 /**
  * \brief Flush leader channel.
  * \brief Flush leader channel.
  *
  *
@@ -672,7 +787,8 @@ ZOOAPI int zoo_aget_children(zhandle_t *zh, const char *path, int watch,
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
 
 
-ZOOAPI int zoo_async(zhandle_t *zh, const char *path, string_completion_t completion, const void *data);
+ZOOAPI int zoo_async(zhandle_t *zh, const char *path, 
+        string_completion_t completion, const void *data);
 
 
 
 
 /**
 /**
@@ -825,7 +941,6 @@ ZOOAPI void zoo_deterministic_conn_order(int yesOrNo);
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_create(zhandle_t *zh, const char *path, const char *value,
 ZOOAPI int zoo_create(zhandle_t *zh, const char *path, const char *value,
         int valuelen, const struct ACL_vector *acl, int flags, char *realpath,	 
         int valuelen, const struct ACL_vector *acl, int flags, char *realpath,	 
         int max_realpath_len);
         int max_realpath_len);
@@ -849,7 +964,6 @@ ZOOAPI int zoo_create(zhandle_t *zh, const char *path, const char *value,
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_delete(zhandle_t *zh, const char *path, int version);
 ZOOAPI int zoo_delete(zhandle_t *zh, const char *path, int version);
 
 
 
 
@@ -871,9 +985,35 @@ ZOOAPI int zoo_delete(zhandle_t *zh, const char *path, int version);
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *stat);
 ZOOAPI int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *stat);
 
 
+/**
+ * \brief checks the existence of a node in zookeeper synchronously.
+ * 
+ * This function is similar to \ref zoo_exists except it allows one specify 
+ * a watcher object rather than a boolean watch flag.
+ * 
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the name of the node. Expressed as a file name with slashes 
+ * separating ancestors of the node.
+ * \param watcher if non-null a watch will set on the specified znode on the server.
+ * The watch will be set even if the node does not exist. This allows clients 
+ * to watch for nodes to appear.
+ * \param watcherCtx user specific data, will be passed to the watcher callback.
+ * Unlike the global context set by \ref zookeeper_init, this watcher context
+ * is associated with the given instance of the watcher only.
+ * \param the return stat value of the node.
+ * \return  return code of the function call.
+ * ZOK operation completed succesfully
+ * ZNONODE the node does not exist.
+ * ZNOAUTH the client does not have permission.
+ * ZBADARGUMENTS - invalid input parameters
+ * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_wexists(zhandle_t *zh, const char *path,
+        watcher_fn watcher, void* watcherCtx, struct Stat *stat);
+
 /**
 /**
  * \brief gets the data associated with a node synchronously.
  * \brief gets the data associated with a node synchronously.
  * 
  * 
@@ -894,10 +1034,38 @@ ZOOAPI int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *s
  * ZINVALIDSTATE - zhandle state is either in SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either in SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,   
 ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,   
                    int* buffer_len, struct Stat *stat);
                    int* buffer_len, struct Stat *stat);
 
 
+/**
+ * \brief gets the data associated with a node synchronously.
+ * 
+ * This function is similar to \ref zoo_get except it allows one specify 
+ * a watcher object rather than a boolean watch flag.
+ * 
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the name of the node. Expressed as a file name with slashes 
+ * separating ancestors of the node.
+ * \param watcher if non-null, a watch will be set at the server to notify 
+ * the client if the node changes.
+ * \param watcherCtx user specific data, will be passed to the watcher callback.
+ * Unlike the global context set by \ref zookeeper_init, this watcher context
+ * is associated with the given instance of the watcher only.
+ * \param buffer the buffer holding the node data returned by the server
+ * \param buffer_len is the size of the buffer pointed to by the buffer parameter.
+ * It'll be set to the actual data length upon return.
+ * \param stat if not NULL, will hold the value of stat for the path on return.
+ * \return return value of the function call.
+ * ZOK operation completed succesfully
+ * ZNONODE the node does not exist.
+ * ZNOAUTH the client does not have permission.
+ * ZBADARGUMENTS - invalid input parameters
+ * ZINVALIDSTATE - zhandle state is either in SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_wget(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx, 
+        char *buffer, int* buffer_len, struct Stat *stat);
 
 
 /**
 /**
  * \brief sets the data associated with a node.
  * \brief sets the data associated with a node.
@@ -919,7 +1087,6 @@ ZOOAPI int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_set(zhandle_t *zh, const char *path, const char *buffer, int buflen,
 ZOOAPI int zoo_set(zhandle_t *zh, const char *path, const char *buffer, int buflen,
                    int version);
                    int version);
 
 
@@ -941,9 +1108,36 @@ ZOOAPI int zoo_set(zhandle_t *zh, const char *path, const char *buffer, int bufl
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_get_children(zhandle_t *zh, const char *path, int watch,
 ZOOAPI int zoo_get_children(zhandle_t *zh, const char *path, int watch,
                             struct String_vector *strings);
                             struct String_vector *strings);
+
+/**
+ * \brief lists the children of a node synchronously.
+ * 
+ * This function is similar to \ref zoo_get_children except it allows one specify 
+ * a watcher object rather than a boolean watch flag.
+ * 
+ * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
+ * \param path the name of the node. Expressed as a file name with slashes 
+ * separating ancestors of the node.
+ * \param watcher if non-null, a watch will be set at the server to notify 
+ * the client if the node changes.
+ * \param watcherCtx user specific data, will be passed to the watcher callback.
+ * Unlike the global context set by \ref zookeeper_init, this watcher context
+ * is associated with the given instance of the watcher only.
+ * \param strings return value of children paths.
+ * \return the return code of the function.
+ * ZOK operation completed succesfully
+ * ZNONODE the node does not exist.
+ * ZNOAUTH the client does not have permission.
+ * ZBADARGUMENTS - invalid input parameters
+ * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_wget_children(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx,
+        struct String_vector *strings);
+
 /**
 /**
  * \brief gets the acl associated with a node synchronously.
  * \brief gets the acl associated with a node synchronously.
  * 
  * 
@@ -960,7 +1154,6 @@ ZOOAPI int zoo_get_children(zhandle_t *zh, const char *path, int watch,
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_get_acl(zhandle_t *zh, const char *path, struct ACL_vector *acl,
 ZOOAPI int zoo_get_acl(zhandle_t *zh, const char *path, struct ACL_vector *acl,
                        struct Stat *stat);
                        struct Stat *stat);
 
 
@@ -982,7 +1175,6 @@ ZOOAPI int zoo_get_acl(zhandle_t *zh, const char *path, struct ACL_vector *acl,
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZINVALIDSTATE - zhandle state is either SESSION_EXPIRED_STATE or AUTH_FAILED_STATE
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
  */
-
 ZOOAPI int zoo_set_acl(zhandle_t *zh, const char *path, int version,
 ZOOAPI int zoo_set_acl(zhandle_t *zh, const char *path, int version,
                            const struct ACL_vector *acl);
                            const struct ACL_vector *acl);
 
 

+ 1 - 1
src/c/src/cli.c

@@ -55,7 +55,7 @@ printProfileInfo(struct timeval start, struct timeval end,int thres,const char*
     fprintf(stderr,"%s: execution time=%dms\n",msg,delay);
     fprintf(stderr,"%s: execution time=%dms\n",msg,delay);
 }
 }
 
 
-void watcher(zhandle_t *zzh, int type, int state, const char *path) {
+void watcher(zhandle_t *zzh, int type, int state, const char *path,void* context) {
     fprintf(stderr,"Watcher %d state = %d for %s\n", type, state, (path ? path: "null"));
     fprintf(stderr,"Watcher %d state = %d for %s\n", type, state, (path ? path: "null"));
     if (type == SESSION_EVENT) {
     if (type == SESSION_EVENT) {
         if (state == CONNECTED_STATE) {
         if (state == CONNECTED_STATE) {

+ 30 - 0
src/c/src/hashtable/LICENSE.txt

@@ -0,0 +1,30 @@
+Copyright (c) 2002, 2004, Christopher Clark
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+	* Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+
+	* Redistributions in binary form must reproduce the above copyright
+notice, this list of conditions and the following disclaimer in the
+documentation and/or other materials provided with the distribution.
+
+	* Neither the name of the original author; nor the names of any contributors
+may be used to endorse or promote products derived from this software
+without specific prior written permission.
+
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

+ 274 - 0
src/c/src/hashtable/hashtable.c

@@ -0,0 +1,274 @@
+/* Copyright (C) 2004 Christopher Clark <firstname.lastname@cl.cam.ac.uk> */
+
+#include "hashtable.h"
+#include "hashtable_private.h"
+#include <stdlib.h>
+#include <stdio.h>
+#include <string.h>
+#include <math.h>
+
+/*
+Credit for primes table: Aaron Krowne
+ http://br.endernet.org/~akrowne/
+ http://planetmath.org/encyclopedia/GoodHashTablePrimes.html
+*/
+static const unsigned int primes[] = {
+53, 97, 193, 389,
+769, 1543, 3079, 6151,
+12289, 24593, 49157, 98317,
+196613, 393241, 786433, 1572869,
+3145739, 6291469, 12582917, 25165843,
+50331653, 100663319, 201326611, 402653189,
+805306457, 1610612741
+};
+const unsigned int prime_table_length = sizeof(primes)/sizeof(primes[0]);
+const float max_load_factor = 0.65;
+
+/*****************************************************************************/
+struct hashtable *
+create_hashtable(unsigned int minsize,
+                 unsigned int (*hashf) (void*),
+                 int (*eqf) (void*,void*))
+{
+    struct hashtable *h;
+    unsigned int pindex, size = primes[0];
+    /* Check requested hashtable isn't too large */
+    if (minsize > (1u << 30)) return NULL;
+    /* Enforce size as prime */
+    for (pindex=0; pindex < prime_table_length; pindex++) {
+        if (primes[pindex] > minsize) { size = primes[pindex]; break; }
+    }
+    h = (struct hashtable *)malloc(sizeof(struct hashtable));
+    if (NULL == h) return NULL; /*oom*/
+    h->table = (struct entry **)malloc(sizeof(struct entry*) * size);
+    if (NULL == h->table) { free(h); return NULL; } /*oom*/
+    memset(h->table, 0, size * sizeof(struct entry *));
+    h->tablelength  = size;
+    h->primeindex   = pindex;
+    h->entrycount   = 0;
+    h->hashfn       = hashf;
+    h->eqfn         = eqf;
+    h->loadlimit    = (unsigned int) ceil(size * max_load_factor);
+    return h;
+}
+
+/*****************************************************************************/
+unsigned int
+hash(struct hashtable *h, void *k)
+{
+    /* Aim to protect against poor hash functions by adding logic here
+     * - logic taken from java 1.4 hashtable source */
+    unsigned int i = h->hashfn(k);
+    i += ~(i << 9);
+    i ^=  ((i >> 14) | (i << 18)); /* >>> */
+    i +=  (i << 4);
+    i ^=  ((i >> 10) | (i << 22)); /* >>> */
+    return i;
+}
+
+/*****************************************************************************/
+static int
+hashtable_expand(struct hashtable *h)
+{
+    /* Double the size of the table to accomodate more entries */
+    struct entry **newtable;
+    struct entry *e;
+    struct entry **pE;
+    unsigned int newsize, i, index;
+    /* Check we're not hitting max capacity */
+    if (h->primeindex == (prime_table_length - 1)) return 0;
+    newsize = primes[++(h->primeindex)];
+
+    newtable = (struct entry **)malloc(sizeof(struct entry*) * newsize);
+    if (NULL != newtable)
+    {
+        memset(newtable, 0, newsize * sizeof(struct entry *));
+        /* This algorithm is not 'stable'. ie. it reverses the list
+         * when it transfers entries between the tables */
+        for (i = 0; i < h->tablelength; i++) {
+            while (NULL != (e = h->table[i])) {
+                h->table[i] = e->next;
+                index = indexFor(newsize,e->h);
+                e->next = newtable[index];
+                newtable[index] = e;
+            }
+        }
+        free(h->table);
+        h->table = newtable;
+    }
+    /* Plan B: realloc instead */
+    else 
+    {
+        newtable = (struct entry **)
+                   realloc(h->table, newsize * sizeof(struct entry *));
+        if (NULL == newtable) { (h->primeindex)--; return 0; }
+        h->table = newtable;
+        memset(newtable[h->tablelength], 0, newsize - h->tablelength);
+        for (i = 0; i < h->tablelength; i++) {
+            for (pE = &(newtable[i]), e = *pE; e != NULL; e = *pE) {
+                index = indexFor(newsize,e->h);
+                if (index == i)
+                {
+                    pE = &(e->next);
+                }
+                else
+                {
+                    *pE = e->next;
+                    e->next = newtable[index];
+                    newtable[index] = e;
+                }
+            }
+        }
+    }
+    h->tablelength = newsize;
+    h->loadlimit   = (unsigned int) ceil(newsize * max_load_factor);
+    return -1;
+}
+
+/*****************************************************************************/
+unsigned int
+hashtable_count(struct hashtable *h)
+{
+    return h->entrycount;
+}
+
+/*****************************************************************************/
+int
+hashtable_insert(struct hashtable *h, void *k, void *v)
+{
+    /* This method allows duplicate keys - but they shouldn't be used */
+    unsigned int index;
+    struct entry *e;
+    if (++(h->entrycount) > h->loadlimit)
+    {
+        /* Ignore the return value. If expand fails, we should
+         * still try cramming just this value into the existing table
+         * -- we may not have memory for a larger table, but one more
+         * element may be ok. Next time we insert, we'll try expanding again.*/
+        hashtable_expand(h);
+    }
+    e = (struct entry *)malloc(sizeof(struct entry));
+    if (NULL == e) { --(h->entrycount); return 0; } /*oom*/
+    e->h = hash(h,k);
+    index = indexFor(h->tablelength,e->h);
+    e->k = k;
+    e->v = v;
+    e->next = h->table[index];
+    h->table[index] = e;
+    return -1;
+}
+
+/*****************************************************************************/
+void * /* returns value associated with key */
+hashtable_search(struct hashtable *h, void *k)
+{
+    struct entry *e;
+    unsigned int hashvalue, index;
+    hashvalue = hash(h,k);
+    index = indexFor(h->tablelength,hashvalue);
+    e = h->table[index];
+    while (NULL != e)
+    {
+        /* Check hash value to short circuit heavier comparison */
+        if ((hashvalue == e->h) && (h->eqfn(k, e->k))) return e->v;
+        e = e->next;
+    }
+    return NULL;
+}
+
+/*****************************************************************************/
+void * /* returns value associated with key */
+hashtable_remove(struct hashtable *h, void *k)
+{
+    /* TODO: consider compacting the table when the load factor drops enough,
+     *       or provide a 'compact' method. */
+
+    struct entry *e;
+    struct entry **pE;
+    void *v;
+    unsigned int hashvalue, index;
+
+    hashvalue = hash(h,k);
+    index = indexFor(h->tablelength,hash(h,k));
+    pE = &(h->table[index]);
+    e = *pE;
+    while (NULL != e)
+    {
+        /* Check hash value to short circuit heavier comparison */
+        if ((hashvalue == e->h) && (h->eqfn(k, e->k)))
+        {
+            *pE = e->next;
+            h->entrycount--;
+            v = e->v;
+            freekey(e->k);
+            free(e);
+            return v;
+        }
+        pE = &(e->next);
+        e = e->next;
+    }
+    return NULL;
+}
+
+/*****************************************************************************/
+/* destroy */
+void
+hashtable_destroy(struct hashtable *h, int free_values)
+{
+    unsigned int i;
+    struct entry *e, *f;
+    struct entry **table = h->table;
+    if (free_values)
+    {
+        for (i = 0; i < h->tablelength; i++)
+        {
+            e = table[i];
+            while (NULL != e)
+            { f = e; e = e->next; freekey(f->k); free(f->v); free(f); }
+        }
+    }
+    else
+    {
+        for (i = 0; i < h->tablelength; i++)
+        {
+            e = table[i];
+            while (NULL != e)
+            { f = e; e = e->next; freekey(f->k); free(f); }
+        }
+    }
+    free(h->table);
+    free(h);
+}
+
+/*
+ * Copyright (c) 2002, Christopher Clark
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * 
+ * * Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * 
+ * * Neither the name of the original author; nor the names of any contributors
+ * may be used to endorse or promote products derived from this software
+ * without specific prior written permission.
+ * 
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER
+ * OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+ * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+ * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+ * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+ * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+ * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+ * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/

+ 207 - 0
src/c/src/hashtable/hashtable.h

@@ -0,0 +1,207 @@
+/* Copyright (C) 2002 Christopher Clark <firstname.lastname@cl.cam.ac.uk> */
+
+#ifndef __HASHTABLE_CWC22_H__
+#define __HASHTABLE_CWC22_H__
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+struct hashtable;
+
+/* Example of use:
+ *
+ *      struct hashtable  *h;
+ *      struct some_key   *k;
+ *      struct some_value *v;
+ *
+ *      static unsigned int         hash_from_key_fn( void *k );
+ *      static int                  keys_equal_fn ( void *key1, void *key2 );
+ *
+ *      h = create_hashtable(16, hash_from_key_fn, keys_equal_fn);
+ *      k = (struct some_key *)     malloc(sizeof(struct some_key));
+ *      v = (struct some_value *)   malloc(sizeof(struct some_value));
+ *
+ *      (initialise k and v to suitable values)
+ * 
+ *      if (! hashtable_insert(h,k,v) )
+ *      {     exit(-1);               }
+ *
+ *      if (NULL == (found = hashtable_search(h,k) ))
+ *      {    printf("not found!");                  }
+ *
+ *      if (NULL == (found = hashtable_remove(h,k) ))
+ *      {    printf("Not found\n");                 }
+ *
+ */
+
+/* Macros may be used to define type-safe(r) hashtable access functions, with
+ * methods specialized to take known key and value types as parameters.
+ * 
+ * Example:
+ *
+ * Insert this at the start of your file:
+ *
+ * DEFINE_HASHTABLE_INSERT(insert_some, struct some_key, struct some_value);
+ * DEFINE_HASHTABLE_SEARCH(search_some, struct some_key, struct some_value);
+ * DEFINE_HASHTABLE_REMOVE(remove_some, struct some_key, struct some_value);
+ *
+ * This defines the functions 'insert_some', 'search_some' and 'remove_some'.
+ * These operate just like hashtable_insert etc., with the same parameters,
+ * but their function signatures have 'struct some_key *' rather than
+ * 'void *', and hence can generate compile time errors if your program is
+ * supplying incorrect data as a key (and similarly for value).
+ *
+ * Note that the hash and key equality functions passed to create_hashtable
+ * still take 'void *' parameters instead of 'some key *'. This shouldn't be
+ * a difficult issue as they're only defined and passed once, and the other
+ * functions will ensure that only valid keys are supplied to them.
+ *
+ * The cost for this checking is increased code size and runtime overhead
+ * - if performance is important, it may be worth switching back to the
+ * unsafe methods once your program has been debugged with the safe methods.
+ * This just requires switching to some simple alternative defines - eg:
+ * #define insert_some hashtable_insert
+ *
+ */
+
+/*****************************************************************************
+ * create_hashtable
+   
+ * @name                    create_hashtable
+ * @param   minsize         minimum initial size of hashtable
+ * @param   hashfunction    function for hashing keys
+ * @param   key_eq_fn       function for determining key equality
+ * @return                  newly created hashtable or NULL on failure
+ */
+
+struct hashtable *
+create_hashtable(unsigned int minsize,
+                 unsigned int (*hashfunction) (void*),
+                 int (*key_eq_fn) (void*,void*));
+
+/*****************************************************************************
+ * hashtable_insert
+   
+ * @name        hashtable_insert
+ * @param   h   the hashtable to insert into
+ * @param   k   the key - hashtable claims ownership and will free on removal
+ * @param   v   the value - does not claim ownership
+ * @return      non-zero for successful insertion
+ *
+ * This function will cause the table to expand if the insertion would take
+ * the ratio of entries to table size over the maximum load factor.
+ *
+ * This function does not check for repeated insertions with a duplicate key.
+ * The value returned when using a duplicate key is undefined -- when
+ * the hashtable changes size, the order of retrieval of duplicate key
+ * entries is reversed.
+ * If in doubt, remove before insert.
+ */
+
+int 
+hashtable_insert(struct hashtable *h, void *k, void *v);
+
+#define DEFINE_HASHTABLE_INSERT(fnname, keytype, valuetype) \
+int fnname (struct hashtable *h, keytype *k, valuetype *v) \
+{ \
+    return hashtable_insert(h,k,v); \
+}
+
+/*****************************************************************************
+ * hashtable_search
+   
+ * @name        hashtable_search
+ * @param   h   the hashtable to search
+ * @param   k   the key to search for  - does not claim ownership
+ * @return      the value associated with the key, or NULL if none found
+ */
+
+void *
+hashtable_search(struct hashtable *h, void *k);
+
+#define DEFINE_HASHTABLE_SEARCH(fnname, keytype, valuetype) \
+valuetype * fnname (struct hashtable *h, keytype *k) \
+{ \
+    return (valuetype *) (hashtable_search(h,k)); \
+}
+
+/*****************************************************************************
+ * hashtable_remove
+   
+ * @name        hashtable_remove
+ * @param   h   the hashtable to remove the item from
+ * @param   k   the key to search for  - does not claim ownership
+ * @return      the value associated with the key, or NULL if none found
+ */
+
+void * /* returns value */
+hashtable_remove(struct hashtable *h, void *k);
+
+#define DEFINE_HASHTABLE_REMOVE(fnname, keytype, valuetype) \
+valuetype * fnname (struct hashtable *h, keytype *k) \
+{ \
+    return (valuetype *) (hashtable_remove(h,k)); \
+}
+
+
+/*****************************************************************************
+ * hashtable_count
+   
+ * @name        hashtable_count
+ * @param   h   the hashtable
+ * @return      the number of items stored in the hashtable
+ */
+unsigned int
+hashtable_count(struct hashtable *h);
+
+
+/*****************************************************************************
+ * hashtable_destroy
+   
+ * @name        hashtable_destroy
+ * @param   h   the hashtable
+ * @param       free_values     whether to call 'free' on the remaining values
+ */
+
+void
+hashtable_destroy(struct hashtable *h, int free_values);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* __HASHTABLE_CWC22_H__ */
+
+/*
+ * Copyright (c) 2002, Christopher Clark
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * 
+ * * Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * 
+ * * Neither the name of the original author; nor the names of any contributors
+ * may be used to endorse or promote products derived from this software
+ * without specific prior written permission.
+ * 
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER
+ * OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+ * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+ * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+ * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+ * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+ * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+ * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/

+ 188 - 0
src/c/src/hashtable/hashtable_itr.c

@@ -0,0 +1,188 @@
+/* Copyright (C) 2002, 2004 Christopher Clark  <firstname.lastname@cl.cam.ac.uk> */
+
+#include "hashtable.h"
+#include "hashtable_private.h"
+#include "hashtable_itr.h"
+#include <stdlib.h> /* defines NULL */
+
+/*****************************************************************************/
+/* hashtable_iterator    - iterator constructor */
+
+struct hashtable_itr *
+hashtable_iterator(struct hashtable *h)
+{
+    unsigned int i, tablelength;
+    struct hashtable_itr *itr = (struct hashtable_itr *)
+        malloc(sizeof(struct hashtable_itr));
+    if (NULL == itr) return NULL;
+    itr->h = h;
+    itr->e = NULL;
+    itr->parent = NULL;
+    tablelength = h->tablelength;
+    itr->index = tablelength;
+    if (0 == h->entrycount) return itr;
+
+    for (i = 0; i < tablelength; i++)
+    {
+        if (NULL != h->table[i])
+        {
+            itr->e = h->table[i];
+            itr->index = i;
+            break;
+        }
+    }
+    return itr;
+}
+
+/*****************************************************************************/
+/* key      - return the key of the (key,value) pair at the current position */
+/* value    - return the value of the (key,value) pair at the current position */
+
+void *
+hashtable_iterator_key(struct hashtable_itr *i)
+{ return i->e->k; }
+
+void *
+hashtable_iterator_value(struct hashtable_itr *i)
+{ return i->e->v; }
+
+/*****************************************************************************/
+/* advance - advance the iterator to the next element
+ *           returns zero if advanced to end of table */
+
+int
+hashtable_iterator_advance(struct hashtable_itr *itr)
+{
+    unsigned int j,tablelength;
+    struct entry **table;
+    struct entry *next;
+    if (NULL == itr->e) return 0; /* stupidity check */
+
+    next = itr->e->next;
+    if (NULL != next)
+    {
+        itr->parent = itr->e;
+        itr->e = next;
+        return -1;
+    }
+    tablelength = itr->h->tablelength;
+    itr->parent = NULL;
+    if (tablelength <= (j = ++(itr->index)))
+    {
+        itr->e = NULL;
+        return 0;
+    }
+    table = itr->h->table;
+    while (NULL == (next = table[j]))
+    {
+        if (++j >= tablelength)
+        {
+            itr->index = tablelength;
+            itr->e = NULL;
+            return 0;
+        }
+    }
+    itr->index = j;
+    itr->e = next;
+    return -1;
+}
+
+/*****************************************************************************/
+/* remove - remove the entry at the current iterator position
+ *          and advance the iterator, if there is a successive
+ *          element.
+ *          If you want the value, read it before you remove:
+ *          beware memory leaks if you don't.
+ *          Returns zero if end of iteration. */
+
+int
+hashtable_iterator_remove(struct hashtable_itr *itr)
+{
+    struct entry *remember_e, *remember_parent;
+    int ret;
+
+    /* Do the removal */
+    if (NULL == (itr->parent))
+    {
+        /* element is head of a chain */
+        itr->h->table[itr->index] = itr->e->next;
+    } else {
+        /* element is mid-chain */
+        itr->parent->next = itr->e->next;
+    }
+    /* itr->e is now outside the hashtable */
+    remember_e = itr->e;
+    itr->h->entrycount--;
+    freekey(remember_e->k);
+
+    /* Advance the iterator, correcting the parent */
+    remember_parent = itr->parent;
+    ret = hashtable_iterator_advance(itr);
+    if (itr->parent == remember_e) { itr->parent = remember_parent; }
+    free(remember_e);
+    return ret;
+}
+
+/*****************************************************************************/
+int /* returns zero if not found */
+hashtable_iterator_search(struct hashtable_itr *itr,
+                          struct hashtable *h, void *k)
+{
+    struct entry *e, *parent;
+    unsigned int hashvalue, index;
+
+    hashvalue = hash(h,k);
+    index = indexFor(h->tablelength,hashvalue);
+
+    e = h->table[index];
+    parent = NULL;
+    while (NULL != e)
+    {
+        /* Check hash value to short circuit heavier comparison */
+        if ((hashvalue == e->h) && (h->eqfn(k, e->k)))
+        {
+            itr->index = index;
+            itr->e = e;
+            itr->parent = parent;
+            itr->h = h;
+            return -1;
+        }
+        parent = e;
+        e = e->next;
+    }
+    return 0;
+}
+
+
+/*
+ * Copyright (c) 2002, 2004, Christopher Clark
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * 
+ * * Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * 
+ * * Neither the name of the original author; nor the names of any contributors
+ * may be used to endorse or promote products derived from this software
+ * without specific prior written permission.
+ * 
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER
+ * OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+ * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+ * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+ * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+ * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+ * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+ * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/

+ 119 - 0
src/c/src/hashtable/hashtable_itr.h

@@ -0,0 +1,119 @@
+/* Copyright (C) 2002, 2004 Christopher Clark <firstname.lastname@cl.cam.ac.uk> */
+
+#ifndef __HASHTABLE_ITR_CWC22__
+#define __HASHTABLE_ITR_CWC22__
+#include "hashtable.h"
+#include "hashtable_private.h" /* needed to enable inlining */
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+/*****************************************************************************/
+/* This struct is only concrete here to allow the inlining of two of the
+ * accessor functions. */
+struct hashtable_itr
+{
+    struct hashtable *h;
+    struct entry *e;
+    struct entry *parent;
+    unsigned int index;
+};
+
+
+/*****************************************************************************/
+/* hashtable_iterator
+ */
+
+struct hashtable_itr *
+hashtable_iterator(struct hashtable *h);
+
+/*****************************************************************************/
+/* hashtable_iterator_key
+ * - return the value of the (key,value) pair at the current position */
+
+extern inline void *
+hashtable_iterator_key(struct hashtable_itr *i)
+{
+    return i->e->k;
+}
+
+/*****************************************************************************/
+/* value - return the value of the (key,value) pair at the current position */
+
+extern inline void *
+hashtable_iterator_value(struct hashtable_itr *i)
+{
+    return i->e->v;
+}
+
+/*****************************************************************************/
+/* advance - advance the iterator to the next element
+ *           returns zero if advanced to end of table */
+
+int
+hashtable_iterator_advance(struct hashtable_itr *itr);
+
+/*****************************************************************************/
+/* remove - remove current element and advance the iterator to the next element
+ *          NB: if you need the value to free it, read it before
+ *          removing. ie: beware memory leaks!
+ *          returns zero if advanced to end of table */
+
+int
+hashtable_iterator_remove(struct hashtable_itr *itr);
+
+/*****************************************************************************/
+/* search - overwrite the supplied iterator, to point to the entry
+ *          matching the supplied key.
+            h points to the hashtable to be searched.
+ *          returns zero if not found. */
+int
+hashtable_iterator_search(struct hashtable_itr *itr,
+                          struct hashtable *h, void *k);
+
+#define DEFINE_HASHTABLE_ITERATOR_SEARCH(fnname, keytype) \
+int fnname (struct hashtable_itr *i, struct hashtable *h, keytype *k) \
+{ \
+    return (hashtable_iterator_search(i,h,k)); \
+}
+
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* __HASHTABLE_ITR_CWC22__*/
+
+/*
+ * Copyright (c) 2002, 2004, Christopher Clark
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * 
+ * * Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * 
+ * * Neither the name of the original author; nor the names of any contributors
+ * may be used to endorse or promote products derived from this software
+ * without specific prior written permission.
+ * 
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER
+ * OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+ * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+ * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+ * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+ * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+ * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+ * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/

+ 85 - 0
src/c/src/hashtable/hashtable_private.h

@@ -0,0 +1,85 @@
+/* Copyright (C) 2002, 2004 Christopher Clark <firstname.lastname@cl.cam.ac.uk> */
+
+#ifndef __HASHTABLE_PRIVATE_CWC22_H__
+#define __HASHTABLE_PRIVATE_CWC22_H__
+
+#include "hashtable.h"
+
+/*****************************************************************************/
+struct entry
+{
+    void *k, *v;
+    unsigned int h;
+    struct entry *next;
+};
+
+struct hashtable {
+    unsigned int tablelength;
+    struct entry **table;
+    unsigned int entrycount;
+    unsigned int loadlimit;
+    unsigned int primeindex;
+    unsigned int (*hashfn) (void *k);
+    int (*eqfn) (void *k1, void *k2);
+};
+
+/*****************************************************************************/
+unsigned int
+hash(struct hashtable *h, void *k);
+
+/*****************************************************************************/
+/* indexFor */
+static inline unsigned int
+indexFor(unsigned int tablelength, unsigned int hashvalue) {
+    return (hashvalue % tablelength);
+};
+
+/* Only works if tablelength == 2^N */
+/*static inline unsigned int
+indexFor(unsigned int tablelength, unsigned int hashvalue)
+{
+    return (hashvalue & (tablelength - 1u));
+}
+*/
+
+/*****************************************************************************/
+#define freekey(X) free(X)
+/*define freekey(X) ; */
+
+
+/*****************************************************************************/
+
+#endif /* __HASHTABLE_PRIVATE_CWC22_H__*/
+
+/*
+ * Copyright (c) 2002, Christopher Clark
+ * All rights reserved.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * * Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ * 
+ * * Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in the
+ * documentation and/or other materials provided with the distribution.
+ * 
+ * * Neither the name of the original author; nor the names of any contributors
+ * may be used to endorse or promote products derived from this software
+ * without specific prior written permission.
+ * 
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE COPYRIGHT OWNER
+ * OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+ * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+ * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+ * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+ * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+ * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+ * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+*/

+ 1 - 1
src/c/src/load_gen.c

@@ -68,7 +68,7 @@ void waitCounter(){
     pthread_mutex_unlock(&counterLock);    
     pthread_mutex_unlock(&counterLock);    
 }
 }
 
 
-void listener(zhandle_t *zzh, int type, int state, const char *path) {
+void listener(zhandle_t *zzh, int type, int state, const char *path,void* ctx) {
     if(type == SESSION_EVENT){
     if(type == SESSION_EVENT){
         if(state == CONNECTED_STATE){
         if(state == CONNECTED_STATE){
             pthread_mutex_lock(&lock);
             pthread_mutex_lock(&lock);

+ 22 - 3
src/c/src/zk_adaptor.h

@@ -23,12 +23,28 @@
 #include <pthread.h>
 #include <pthread.h>
 #endif
 #endif
 #include "zookeeper.h"
 #include "zookeeper.h"
+#include "zk_hashtable.h"
 
 
 /* predefined xid's values recognized as special by the server */
 /* predefined xid's values recognized as special by the server */
 #define WATCHER_EVENT_XID -1 
 #define WATCHER_EVENT_XID -1 
 #define PING_XID -2
 #define PING_XID -2
 #define AUTH_XID -4
 #define AUTH_XID -4
 
 
+/* zookeeper state constants */
+#define EXPIRED_SESSION_STATE_DEF -112
+#define AUTH_FAILED_STATE_DEF -113
+#define CONNECTING_STATE_DEF 1
+#define ASSOCIATING_STATE_DEF 2
+#define CONNECTED_STATE_DEF 3
+
+/* zookeeper event type constants */
+#define CREATED_EVENT_DEF 1
+#define DELETED_EVENT_DEF 2
+#define CHANGED_EVENT_DEF 3
+#define CHILD_EVENT_DEF 4
+#define SESSION_EVENT_DEF -1
+#define NOTWATCHING_EVENT_DEF -2
+
 #ifdef __cplusplus
 #ifdef __cplusplus
 extern "C" {
 extern "C" {
 #endif
 #endif
@@ -174,9 +190,12 @@ struct _zhandle {
     volatile int close_requested;
     volatile int close_requested;
     void *adaptor_priv;
     void *adaptor_priv;
     /* Used for debugging only: non-zero value indicates the time when the zookeeper_process
     /* Used for debugging only: non-zero value indicates the time when the zookeeper_process
-     * call returned while there was at least one server response 
-     * unprocessed available in the socket recv buffer */
+     * call returned while there was at least one unprocessed server response 
+     * available in the socket recv buffer */
     struct timeval socket_readable;
     struct timeval socket_readable;
+    
+    zk_hashtable* active_node_watchers;
+    zk_hashtable* active_child_watchers;
 };
 };
 
 
 int adaptor_init(zhandle_t *zh);
 int adaptor_init(zhandle_t *zh);
@@ -209,7 +228,7 @@ int queue_session_event(zhandle_t *zh, int state);
 #define PROCESS_SESSION_EVENT(zh,newstate) queue_session_event(zh,newstate)
 #define PROCESS_SESSION_EVENT(zh,newstate) queue_session_event(zh,newstate)
 #else
 #else
 // in single-threaded mode process session event immediately
 // in single-threaded mode process session event immediately
-#define PROCESS_SESSION_EVENT(zh,newstate) zh->watcher(zh,SESSION_EVENT,newstate,0)
+#define PROCESS_SESSION_EVENT(zh,newstate) deliverWatchers(zh,SESSION_EVENT,newstate,0)
 #endif
 #endif
 
 
 #ifdef __cplusplus
 #ifdef __cplusplus

+ 444 - 0
src/c/src/zk_hashtable.c

@@ -0,0 +1,444 @@
+/**
+ * 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.
+ */
+
+#include "zk_hashtable.h"
+#include "zk_adaptor.h"
+#include "hashtable/hashtable.h"
+#include "hashtable/hashtable_itr.h"
+#include <string.h>
+#include <stdlib.h>
+#include <assert.h>
+
+#ifdef THREADED
+#include <pthread.h>
+#endif
+
+struct _zk_hashtable {
+    struct hashtable* ht;
+#ifdef THREADED
+    pthread_mutex_t lock;
+#endif
+};
+
+hashtable_impl* getImpl(zk_hashtable* ht){
+    return ht->ht;
+}
+
+typedef struct _watcher_object_list_t {
+    watcher_object_t* head;
+} watcher_object_list_t;
+
+watcher_object_t* getFirstWatcher(zk_hashtable* ht,const char* path)
+{
+    watcher_object_list_t* wl=hashtable_search(ht->ht,(void*)path);
+    if(wl!=0)
+        return wl->head;
+    return 0;
+}
+
+watcher_object_t* clone_watcher_object(watcher_object_t* wo)
+{
+    watcher_object_t* res=calloc(1,sizeof(watcher_object_t));
+    res->watcher=wo->watcher;
+    res->context=wo->context;
+    return res;
+}
+
+static unsigned int string_hash_djb2(void *str) 
+{
+    unsigned int hash = 5381;
+    int c;
+
+    while ((c = *(const char*)str++))
+        hash = ((hash << 5) + hash) + c; /* hash * 33 + c */
+
+    return hash;
+}
+
+static int string_equal(void *key1,void *key2)
+{
+    return strcmp((const char*)key1,(const char*)key2)==0;
+}
+
+watcher_object_t* create_watcher_object(watcher_fn watcher,void* ctx)
+{
+    watcher_object_t* wo=calloc(1,sizeof(watcher_object_t));
+    wo->watcher=watcher;
+    wo->context=ctx;
+    return wo;
+}
+
+static watcher_object_list_t* create_watcher_object_list(watcher_object_t* head) 
+{
+    watcher_object_list_t* wl=calloc(1,sizeof(watcher_object_list_t));
+    wl->head=head;
+    return wl;
+}
+
+static void destroy_watcher_object_list(watcher_object_list_t* list)
+{
+    if(list==0)
+        return;
+    watcher_object_t* e=list->head;
+    while(e!=0){
+        watcher_object_t* this=e;
+        e=e->next;
+        free(this);
+    }
+    free(list);
+}
+
+zk_hashtable* create_zk_hashtable()
+{
+    struct _zk_hashtable *ht=calloc(1,sizeof(struct _zk_hashtable));
+#ifdef THREADED
+    pthread_mutex_init(&ht->lock, 0);
+#endif
+    ht->ht=create_hashtable(32,string_hash_djb2,string_equal);
+    return ht;
+}
+
+int get_element_count(zk_hashtable *ht)
+{
+    int res;
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    res=hashtable_count(ht->ht);    
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif
+    return res;
+}
+
+int get_watcher_count(zk_hashtable* ht,const char* path)
+{
+    int res=0;
+    watcher_object_list_t* wl;
+    watcher_object_t* wo;
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    wl=hashtable_search(ht->ht,(void*)path);
+    if(wl==0)
+        goto done;
+    wo=wl->head;
+    while(wo!=0){
+        res++;
+        wo=wo->next;
+    }
+done:
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif
+    return res;    
+}
+
+static void do_clean_hashtable(zk_hashtable* ht)
+{
+    struct hashtable_itr *it;
+    int hasMore;
+    if(hashtable_count(ht->ht)==0)
+        return;
+    it=hashtable_iterator(ht->ht);
+    do{
+        watcher_object_list_t* w=hashtable_iterator_value(it);
+        destroy_watcher_object_list(w);
+        hasMore=hashtable_iterator_remove(it);
+    }while(hasMore);
+    free(it);
+}
+
+void clean_zk_hashtable(zk_hashtable* ht)
+{
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    do_clean_hashtable(ht);    
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif    
+}
+
+void destroy_zk_hashtable(zk_hashtable* ht)
+{
+    if(ht!=0){
+        do_clean_hashtable(ht);
+        hashtable_destroy(ht->ht,0);
+#ifdef THREADED
+        pthread_mutex_destroy(&ht->lock);
+#endif
+        free(ht);
+    }
+}
+
+// searches for a watcher object instance in a watcher object list;
+// two watcher objects are equal if their watcher function and context pointers
+// are equal
+static watcher_object_t* search_watcher(watcher_object_list_t* wl,watcher_object_t* wo)
+{
+    watcher_object_t* wobj=wl->head;
+    while(wobj!=0){
+        if(wobj->watcher==wo->watcher && wobj->context==wo->context)
+            return wobj;
+        wobj=wobj->next;
+    }
+    return 0;
+}
+
+static int do_insert_watcher_object(zk_hashtable *ht, const char *path, watcher_object_t* wo)
+{
+    int res=1;
+    watcher_object_list_t* wl;
+    wl=hashtable_search(ht->ht,(void*)path);
+    if(wl==0){
+        int res;
+        /* inserting a new path element */
+        res=hashtable_insert(ht->ht,strdup(path),create_watcher_object_list(wo));
+        assert(res);
+    }else{
+        /* path already exists; check if the watcher already exists */
+        if(search_watcher(wl,wo)==0){
+            wo->next=wl->head;
+            wl->head=wo; // insert the new watcher at the head
+        }else
+            res=0; // the watcher already exists -- do not insert!
+    }
+    return res;    
+}
+
+int insert_watcher_object(zk_hashtable *ht, const char *path, watcher_object_t* wo)
+{
+    int res;
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    res=do_insert_watcher_object(ht,path,wo);
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif
+    return res;
+}
+
+static void copy_watchers(zk_hashtable* dst,const char* path,watcher_object_list_t* wl)
+{
+    if(wl==0)
+        return;
+    watcher_object_t* wo=wl->head;
+    while(wo!=0){
+        int res;
+        watcher_object_t* cloned=clone_watcher_object(wo);
+        res=do_insert_watcher_object(dst,path,cloned);
+        // was it a duplicate?
+        if(res==0)
+            free(cloned); // yes, didn't get inserted
+        wo=wo->next;
+    }
+}
+
+static void copy_table(zk_hashtable* dst,zk_hashtable* src)
+{
+    struct hashtable_itr *it;
+    int hasMore;
+    if(hashtable_count(src->ht)==0)
+        return;
+    it=hashtable_iterator(src->ht);
+    do{
+        copy_watchers(dst,hashtable_iterator_key(it),hashtable_iterator_value(it));
+        hasMore=hashtable_iterator_advance(it);
+    }while(hasMore);
+    free(it);
+}
+
+zk_hashtable* combine_hashtables(zk_hashtable *ht1,zk_hashtable *ht2)
+{
+    zk_hashtable* newht=create_zk_hashtable();
+#ifdef THREADED
+    pthread_mutex_lock(&ht1->lock);
+    pthread_mutex_lock(&ht2->lock);
+#endif
+    copy_table(newht,ht1);
+    copy_table(newht,ht2);
+#ifdef THREADED
+    pthread_mutex_unlock(&ht2->lock);
+    pthread_mutex_unlock(&ht1->lock);
+#endif    
+    return newht;
+}
+
+zk_hashtable* move_merge_watchers(zk_hashtable *ht1,zk_hashtable *ht2,const char *path)
+{
+    watcher_object_list_t* wl;
+    zk_hashtable* newht=create_zk_hashtable();
+#ifdef THREADED
+    pthread_mutex_lock(&ht1->lock);
+    pthread_mutex_lock(&ht2->lock);
+#endif
+    // copy watchers from table 1
+    wl=hashtable_remove(ht1->ht,(void*)path);
+    copy_watchers(newht,path,wl);
+    destroy_watcher_object_list(wl);
+    // merge all watchers from tabe 2
+    wl=hashtable_remove(ht2->ht,(void*)path);
+    copy_watchers(newht,path,wl);
+    destroy_watcher_object_list(wl);
+    
+#ifdef THREADED
+    pthread_mutex_unlock(&ht2->lock);
+    pthread_mutex_unlock(&ht1->lock);
+#endif    
+    return newht;
+}
+
+int contains_watcher(zk_hashtable *ht,watcher_object_t* wo)
+{
+    struct hashtable_itr *it=0;
+    int res=0;
+    int hasMore;
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    if(hashtable_count(ht->ht)==0)
+        goto done;
+    it=hashtable_iterator(ht->ht);
+    do{
+        watcher_object_list_t* w=hashtable_iterator_value(it);
+        if(search_watcher(w,wo)!=0){
+            res=1;
+            goto done;
+        }
+        hasMore=hashtable_iterator_advance(it);
+    }while(hasMore);
+done:
+    if(it!=0)
+        free(it);
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif
+    return res;
+}
+
+static void do_foreach_watcher(watcher_object_t* wo,zhandle_t* zh,
+        const char* path,int type,int state)
+{
+    while(wo!=0){
+        wo->watcher(zh,type,state,path,wo->context);
+        wo=wo->next;
+    }    
+}
+
+void deliver_session_event(zk_hashtable* ht,zhandle_t* zh,int type,int state)
+{
+    struct hashtable_itr *it;
+    int hasMore;
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    if(hashtable_count(ht->ht)==0)
+        goto done;
+    it=hashtable_iterator(ht->ht);
+    do{
+        watcher_object_t* wo=((watcher_object_list_t*)hashtable_iterator_value(it))->head;
+        // session events are delivered with the path set to null
+        do_foreach_watcher(wo,zh,0,type,state);
+        hasMore=hashtable_iterator_advance(it);
+    }while(hasMore);
+    free(it);
+done:
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif
+    return;
+}
+
+void deliver_znode_event(zk_hashtable* ht,zhandle_t* zh,const char* path,int type,int state)
+{
+    watcher_object_list_t* wl;
+#ifdef THREADED
+    pthread_mutex_lock(&ht->lock);
+#endif
+    wl=hashtable_remove(ht->ht,(void*)path);
+#ifdef THREADED
+    pthread_mutex_unlock(&ht->lock);
+#endif
+    if(wl!=0){
+        do_foreach_watcher(wl->head,zh,path,type,state);
+        destroy_watcher_object_list(wl);
+    }
+}
+
+void deliverWatchers(zhandle_t* zh,int type,int state, const char* path)
+{
+    zk_hashtable *ht;
+    if(type==SESSION_EVENT){
+        watcher_object_t defWatcher;
+        if(state==CONNECTED_STATE){
+            clean_zk_hashtable(zh->active_node_watchers);
+            clean_zk_hashtable(zh->active_child_watchers);
+            // unconditionally call back the default watcher only
+            zh->watcher(zh,type,state,0,zh->context);
+            return;
+        }
+        // process a disconnect/expiration
+        // must merge node and child watchers first
+        ht=combine_hashtables(zh->active_node_watchers,
+                zh->active_child_watchers);
+        // check if the default watcher is already present on the combined map 
+        defWatcher.watcher=zh->watcher;
+        defWatcher.context=zh->context;
+        if(contains_watcher(ht,&defWatcher)==0)
+            insert_watcher_object(ht,"",clone_watcher_object(&defWatcher));
+        // deliver watcher callback to all registered watchers
+        deliver_session_event(ht,zh,type,state);
+        destroy_zk_hashtable(ht);
+        // in anticipation of the watcher auto-reset feature we keep 
+        // the watcher maps intact. 
+        // (for now, we simply clean the maps on reconnect, see above)
+        return;
+    }
+    switch(type){
+    case CREATED_EVENT_DEF:
+    case CHANGED_EVENT_DEF:
+        // look up the watchers for the path and deliver them
+        deliver_znode_event(zh->active_node_watchers,zh,path,type,state);
+        break;
+    case CHILD_EVENT_DEF:
+        // look up the watchers for the path and deliver them
+        deliver_znode_event(zh->active_child_watchers,zh,path,type,state);
+        break;
+    case DELETED_EVENT_DEF:
+        // combine node and child watchers for the path and deliver them
+        ht=move_merge_watchers(zh->active_child_watchers,
+                zh->active_node_watchers,path);
+        deliver_znode_event(ht,zh,path,type,state);
+        destroy_zk_hashtable(ht);
+        break;
+    }
+}
+
+void activateWatcher(watcher_registration_t* reg, int rc)
+{
+    if(reg!=0){
+        /* in multithreaded lib, this code is executed 
+         * by the completion thread */
+        if(reg->checker(rc)){
+            insert_watcher_object(reg->activeMap,reg->path,
+                    create_watcher_object(reg->watcher,reg->context));
+        }
+    }    
+}

+ 115 - 0
src/c/src/zk_hashtable.h

@@ -0,0 +1,115 @@
+/**
+ * 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.
+ */
+
+#ifndef ZK_HASHTABLE_H_
+#define ZK_HASHTABLE_H_
+
+#include <zookeeper.h>
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+typedef struct _zk_hashtable zk_hashtable;
+
+/**
+ * The function must return a non-zero value if the watcher object can be activated
+ * as a result of the server response. Normally, a watch can only be activated
+ * if the server returns a success code (ZOK). However in the case when zoo_exists() 
+ * returns a ZNONODE code the watcher should be activated nevertheless.
+ */
+typedef int (*result_checker_fn)(int rc);
+
+/**
+ * A watcher object gets temporarily stored with the completion entry until 
+ * the server response comes back at which moment the watcher object is moved
+ * to the active watchers map.
+ */
+typedef struct _watcher_registration {
+    watcher_fn watcher;
+    result_checker_fn checker;
+    void* context;
+    zk_hashtable* activeMap; // the map to add the watcher to upon activation
+    const char* path;
+} watcher_registration_t;
+
+
+typedef struct _watcher_object {
+    watcher_fn watcher;
+    void* context;
+    struct _watcher_object* next;
+} watcher_object_t;
+
+watcher_object_t* create_watcher_object(watcher_fn watcher,void* ctx);
+watcher_object_t* clone_watcher_object(watcher_object_t* wo);
+
+zk_hashtable* create_zk_hashtable();
+void clean_zk_hashtable(zk_hashtable* ht);
+void destroy_zk_hashtable(zk_hashtable* ht);
+zk_hashtable* combine_hashtables(zk_hashtable* ht1,zk_hashtable* ht2);
+/**
+ * \brief first, merges all watchers for path from ht1 and ht2 to a new hashtable and 
+ * then removes the path entries from ht1 and ht2 
+ */
+zk_hashtable* move_merge_watchers(zk_hashtable* ht1,zk_hashtable* ht2,const char* path);
+
+/**
+ * The hashtable takes ownership of the watcher object instance.
+ * 
+ * \return 1 if the watcher object was succesfully inserted, 0 otherwise
+ */
+int insert_watcher_object(zk_hashtable* ht, const char* path, watcher_object_t* wo);
+/**
+ * \brief searches the entire hashtable for the watcher object
+ * 
+ * \return 1 if the watcher object found in the table, 0 otherwise
+ */
+int contains_watcher(zk_hashtable* ht,watcher_object_t* wo);
+int get_element_count(zk_hashtable* ht);
+int get_watcher_count(zk_hashtable* ht,const char* path);
+/**
+ * \brief delivers all watchers in the hashtable
+ */
+void deliver_session_event(zk_hashtable* ht,zhandle_t* zh,int type,int state);
+/**
+ * \brief delivers all watchers for path and then removes the path entry 
+ * from the hashtable
+ */
+void deliver_znode_event(zk_hashtable* ht,zhandle_t* zh,const char* path,int type,int state);
+
+/**
+ * zookeeper uses this function to deliver watcher callbacks
+ */
+void deliverWatchers(zhandle_t* zh,int type,int state, const char* path);
+/**
+ * check if the completion has a watcher object associated
+ * with it. If it does, move the watcher object to the map of
+ * active watchers (only if the checker allows to do so)
+ */
+void activateWatcher(watcher_registration_t* reg, int rc);
+
+/* the following functions are for testing only */
+typedef struct hashtable hashtable_impl;
+hashtable_impl* getImpl(zk_hashtable* ht);
+watcher_object_t* getFirstWatcher(zk_hashtable* ht,const char* path);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /*ZK_HASHTABLE_H_*/

+ 183 - 87
src/c/src/zookeeper.c

@@ -29,6 +29,8 @@
 #include <proto.h>
 #include <proto.h>
 #include "zk_adaptor.h"
 #include "zk_adaptor.h"
 #include "zk_log.h"
 #include "zk_log.h"
+#include "zk_hashtable.h"
+
 #include <stdlib.h>
 #include <stdlib.h>
 #include <stdio.h>
 #include <stdio.h>
 #include <string.h>
 #include <string.h>
@@ -55,50 +57,50 @@ const int ZOOKEEPER_READ = 1 << 1;
 const int EPHEMERAL = 1 << 0;
 const int EPHEMERAL = 1 << 0;
 const int SEQUENCE = 1 << 1;
 const int SEQUENCE = 1 << 1;
 
 
-const int EXPIRED_SESSION_STATE = -112;
-const int AUTH_FAILED_STATE = -113;
-const int CONNECTING_STATE = 1;
-const int ASSOCIATING_STATE = 2;
-const int CONNECTED_STATE = 3;
+const int EXPIRED_SESSION_STATE = EXPIRED_SESSION_STATE_DEF;
+const int AUTH_FAILED_STATE = AUTH_FAILED_STATE_DEF;
+const int CONNECTING_STATE = CONNECTING_STATE_DEF;
+const int ASSOCIATING_STATE = ASSOCIATING_STATE_DEF;
+const int CONNECTED_STATE = CONNECTED_STATE_DEF;
 static __attribute__ ((unused)) const char* state2String(int state){
 static __attribute__ ((unused)) const char* state2String(int state){
     switch(state){
     switch(state){
     case 0:
     case 0:
         return "CLOSED_STATE";
         return "CLOSED_STATE";
-    case 1 /*CONNECTING_STATE*/:
+    case CONNECTING_STATE_DEF:
         return "CONNECTING_STATE";
         return "CONNECTING_STATE";
-    case 2 /*ASSOCIATING_STATE*/:
+    case ASSOCIATING_STATE_DEF:
         return "ASSOCIATING_STATE";
         return "ASSOCIATING_STATE";
-    case 3 /*CONNECTED_STATE*/:
+    case CONNECTED_STATE_DEF:
         return "CONNECTED_STATE";
         return "CONNECTED_STATE";
-    case -112 /*EXPIRED_SESSION_STATE*/:
+    case EXPIRED_SESSION_STATE_DEF:
         return "EXPIRED_SESSION_STATE";
         return "EXPIRED_SESSION_STATE";
-    case -113 /*AUTH_FAILED_STATE*/:
+    case AUTH_FAILED_STATE_DEF:
         return "AUTH_FAILED_STATE";
         return "AUTH_FAILED_STATE";
     }
     }
     return "INVALID_STATE";
     return "INVALID_STATE";
 }
 }
 
 
-const int CREATED_EVENT = 1;
-const int DELETED_EVENT = 2;
-const int CHANGED_EVENT = 3;
-const int CHILD_EVENT = 4;
-const int SESSION_EVENT = -1;
-const int NOTWATCHING_EVENT = -2;
+const int CREATED_EVENT = CREATED_EVENT_DEF;
+const int DELETED_EVENT = DELETED_EVENT_DEF;
+const int CHANGED_EVENT = CHANGED_EVENT_DEF;
+const int CHILD_EVENT = CHILD_EVENT_DEF;
+const int SESSION_EVENT = SESSION_EVENT_DEF;
+const int NOTWATCHING_EVENT = NOTWATCHING_EVENT_DEF;
 static __attribute__ ((unused)) const char* watcherEvent2String(int ev){
 static __attribute__ ((unused)) const char* watcherEvent2String(int ev){
     switch(ev){
     switch(ev){
     case 0:
     case 0:
         return "ERROR_EVENT";
         return "ERROR_EVENT";
-    case 1 /*CREATED_EVENT*/:
+    case CREATED_EVENT_DEF:
         return "CREATED_EVENT";
         return "CREATED_EVENT";
-    case 2 /*DELETED_EVENT*/:
+    case DELETED_EVENT_DEF:
         return "DELETED_EVENT";
         return "DELETED_EVENT";
-    case 3 /*CHANGED_EVENT*/:
+    case CHANGED_EVENT_DEF:
         return "CHANGED_EVENT";
         return "CHANGED_EVENT";
-    case 4 /*CHILD_EVENT*/:
+    case CHILD_EVENT_DEF:
         return "CHILD_EVENT";
         return "CHILD_EVENT";
-    case -1 /*SESSION_EVENT*/:
+    case SESSION_EVENT_DEF:
         return "SESSION_EVENT";
         return "SESSION_EVENT";
-    case -2 /*NOTWATCHING_EVENT*/:
+    case NOTWATCHING_EVENT_DEF:
         return "NOTWATCHING_EVENT";
         return "NOTWATCHING_EVENT";
     }
     }
     return "INVALID_EVENT";
     return "INVALID_EVENT";
@@ -126,19 +128,6 @@ struct ACL_vector CREATOR_ALL_ACL = { 1, _CREATOR_ALL_ACL_ACL};
 #define COMPLETION_ACLLIST 4
 #define COMPLETION_ACLLIST 4
 #define COMPLETION_STRING 5
 #define COMPLETION_STRING 5
 
 
-const char*err2string(int err);
-static const char* format_endpoint_info(const struct sockaddr* ep);
-static const char* format_current_endpoint_info(zhandle_t* zh);
-static int add_completion(zhandle_t *zh, int xid, int completion_type, 
-        const void *dc, const void *data, int add_to_front);
-static int handle_socket_error_msg(zhandle_t *zh, int line, int rc,
-    const char* format,...);
-static void cleanup_bufs(zhandle_t *zh,int callCompletion,int rc);
-
-static int disable_conn_permute=0; // permute enabled by default
-
-static void *SYNCHRONOUS_MARKER = (void*)&SYNCHRONOUS_MARKER;
-    
 typedef struct _completion_list {
 typedef struct _completion_list {
     int xid;
     int xid;
     int completion_type; /* one of the COMPLETION_* values */
     int completion_type; /* one of the COMPLETION_* values */
@@ -153,8 +142,30 @@ typedef struct _completion_list {
     const void *data;
     const void *data;
     buffer_list_t *buffer;
     buffer_list_t *buffer;
     struct _completion_list *next;
     struct _completion_list *next;
+    watcher_registration_t* watcher;
 } completion_list_t;
 } completion_list_t;
 
 
+const char*err2string(int err);
+static const char* format_endpoint_info(const struct sockaddr* ep);
+static const char* format_current_endpoint_info(zhandle_t* zh);
+
+/* completion routine forward declarations */
+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);
+static completion_list_t* create_completion_entry(int xid, int completion_type,
+        const void *dc, const void *data,watcher_registration_t* wo);
+static void destroy_completion_entry(completion_list_t* c);
+static void queue_completion(completion_head_t *list, completion_list_t *c,
+        int add_to_front);
+
+static int handle_socket_error_msg(zhandle_t *zh, int line, int rc,
+    const char* format,...);
+static void cleanup_bufs(zhandle_t *zh,int callCompletion,int rc);
+
+static int disable_conn_permute=0; // permute enabled by default
+
+static void *SYNCHRONOUS_MARKER = (void*)&SYNCHRONOUS_MARKER;
+
 const void *zoo_get_context(zhandle_t *zh) 
 const void *zoo_get_context(zhandle_t *zh) 
 {
 {
     return zh->context;
     return zh->context;
@@ -194,6 +205,16 @@ int is_unrecoverable(zhandle_t *zh)
 {
 {
     return (zh->state<0)? ZINVALIDSTATE: ZOK;
     return (zh->state<0)? ZINVALIDSTATE: ZOK;
 }
 }
+
+int exists_result_checker(int rc)
+{
+    return rc==ZOK ||rc == ZNONODE;
+}
+
+int default_result_checker(int rc)
+{
+    return rc==ZOK;
+}
 /**
 /**
  * Frees and closes everything associated with a handle,
  * Frees and closes everything associated with a handle,
  * including the handle itself.
  * including the handle itself.
@@ -219,6 +240,8 @@ static void destroy(zhandle_t *zh)
         zh->addrs = NULL;
         zh->addrs = NULL;
     }
     }
     free_auth_info(&zh->auth);
     free_auth_info(&zh->auth);
+    destroy_zk_hashtable(zh->active_node_watchers);
+    destroy_zk_hashtable(zh->active_child_watchers);
 }
 }
 
 
 static void setup_random()
 static void setup_random()
@@ -359,7 +382,7 @@ const clientid_t *zoo_client_id(zhandle_t *zh)
     return &zh->client_id;
     return &zh->client_id;
 }
 }
 
 
-static void null_watcher_fn(zhandle_t* p1, int p2, int p3,const char* p4){}
+static void null_watcher_fn(zhandle_t* p1, int p2, int p3,const char* p4,void*p5){}
 
 
 watcher_fn zoo_set_watcher(zhandle_t *zh,watcher_fn newFn)
 watcher_fn zoo_set_watcher(zhandle_t *zh,watcher_fn newFn)
 {
 {
@@ -412,9 +435,13 @@ zhandle_t *zookeeper_init(const char *host, watcher_fn watcher,
     zh->last_zxid = 0;
     zh->last_zxid = 0;
     zh->next_deadline.tv_sec=zh->next_deadline.tv_usec=0;
     zh->next_deadline.tv_sec=zh->next_deadline.tv_usec=0;
     zh->socket_readable.tv_sec=zh->socket_readable.tv_usec=0;
     zh->socket_readable.tv_sec=zh->socket_readable.tv_usec=0;
+    zh->active_node_watchers=create_zk_hashtable();
+    zh->active_child_watchers=create_zk_hashtable();
+    
     if (adaptor_init(zh) == -1) {
     if (adaptor_init(zh) == -1) {
         goto abort;
         goto abort;
     }
     }
+    
     return zh;
     return zh;
 abort:
 abort:
     errnosave=errno;
     errnosave=errno;
@@ -675,7 +702,7 @@ void free_completions(zhandle_t *zh,int callCompletion,int rc)
                 break;
                 break;
             }
             }
         }
         }
-        free(cptr);
+        destroy_completion_entry(cptr);
     }
     }
 }
 }
 
 
@@ -1116,11 +1143,6 @@ static __attribute__((unused)) void print_completion_queue(zhandle_t *zh)
     fprintf(LOGSTREAM,"end\n");    
     fprintf(LOGSTREAM,"end\n");    
 }
 }
 
 
-static completion_list_t* create_completion_entry(int xid, int completion_type,
-        const void *dc, const void *data);
-static void queue_completion(completion_head_t *list, completion_list_t *c,
-        int add_to_front);
-
 #ifdef THREADED
 #ifdef THREADED
 // IO thread queues session events to be processed by the completion thread
 // IO thread queues session events to be processed by the completion thread
 int queue_session_event(zhandle_t *zh, int state)
 int queue_session_event(zhandle_t *zh, int state)
@@ -1141,12 +1163,7 @@ int queue_session_event(zhandle_t *zh, int state)
         close_buffer_oarchive(&oa, 1);
         close_buffer_oarchive(&oa, 1);
         goto error;
         goto error;
     }
     }
-    if ((cptr=calloc(1,sizeof(*cptr)))==NULL) {
-        LOG_ERROR(("out of memory"));
-        close_buffer_oarchive(&oa, 1);
-        goto error;
-    }
-    cptr->xid = WATCHER_EVENT_XID;
+    cptr = create_completion_entry(WATCHER_EVENT_XID,-1,0,0,0);
     cptr->buffer = allocate_buffer(get_buffer(oa), get_buffer_len(oa));
     cptr->buffer = allocate_buffer(get_buffer(oa), get_buffer_len(oa));
     cptr->buffer->curr_offset = get_buffer_len(oa);
     cptr->buffer->curr_offset = get_buffer_len(oa);
     if (!cptr->buffer) {
     if (!cptr->buffer) {
@@ -1180,6 +1197,8 @@ completion_list_t *dequeue_completion(completion_head_t *list)
     return cptr;
     return cptr;
 }
 }
 
 
+
+/* handles async completion (both single- and multithreaded) */
 void process_completions(zhandle_t *zh)
 void process_completions(zhandle_t *zh)
 {
 {
     completion_list_t *cptr;
     completion_list_t *cptr;
@@ -1201,7 +1220,7 @@ void process_completions(zhandle_t *zh)
             /* This is a notification so there aren't any pending requests */
             /* This is a notification so there aren't any pending requests */
             LOG_DEBUG(("Calling a watcher for node [%s], event=%s",
             LOG_DEBUG(("Calling a watcher for node [%s], event=%s",
                  (evt.path==NULL?"NULL":evt.path),watcherEvent2String(type)));
                  (evt.path==NULL?"NULL":evt.path),watcherEvent2String(type)));
-            zh->watcher(zh, type, state, evt.path);
+            deliverWatchers(zh,type,state,evt.path);
             deallocate_WatcherEvent(&evt);
             deallocate_WatcherEvent(&evt);
         } else {
         } else {
             int rc = hdr.err;
             int rc = hdr.err;
@@ -1271,9 +1290,9 @@ void process_completions(zhandle_t *zh)
                 }
                 }
                 break;
                 break;
             }
             }
-            free_buffer(cptr->buffer);
-            free(cptr);
+            activateWatcher(cptr->watcher,rc);
         }
         }
+        destroy_completion_entry(cptr);
         close_buffer_iarchive(&ia);
         close_buffer_iarchive(&ia);
     }
     }
 }
 }
@@ -1331,7 +1350,7 @@ int zookeeper_process(zhandle_t *zh, int events)
         zh->last_zxid = hdr.zxid;
         zh->last_zxid = hdr.zxid;
         
         
         if (hdr.xid == WATCHER_EVENT_XID) {
         if (hdr.xid == WATCHER_EVENT_XID) {
-            completion_list_t *c = create_completion_entry(WATCHER_EVENT_XID,-1,0,0);
+            completion_list_t *c = create_completion_entry(WATCHER_EVENT_XID,-1,0,0,0);
             c->buffer = bptr;
             c->buffer = bptr;
             queue_completion(&zh->completions_to_process, c, 0);
             queue_completion(&zh->completions_to_process, c, 0);
         } else if(hdr.xid == AUTH_XID){
         } else if(hdr.xid == AUTH_XID){
@@ -1378,10 +1397,10 @@ int zookeeper_process(zhandle_t *zh, int events)
                 sc->rc = rc;
                 sc->rc = rc;
                 switch(cptr->completion_type) {
                 switch(cptr->completion_type) {
                 case COMPLETION_DATA:
                 case COMPLETION_DATA:
+                    LOG_DEBUG(("Calling COMPLETION_DATA for xid=%x rc=%d",cptr->xid,rc));
                     if (rc==0) {
                     if (rc==0) {
                         struct GetDataResponse res;
                         struct GetDataResponse res;
                         int len;
                         int len;
-                        LOG_DEBUG(("Calling COMPLETION_DATA for xid=%x rc=%d",cptr->xid,rc));
                         deserialize_GetDataResponse(ia, "reply", &res);
                         deserialize_GetDataResponse(ia, "reply", &res);
                         if (res.data.len <= sc->u.data.buff_len) {
                         if (res.data.len <= sc->u.data.buff_len) {
                             len = res.data.len;
                             len = res.data.len;
@@ -1395,18 +1414,18 @@ int zookeeper_process(zhandle_t *zh, int events)
                     }
                     }
                     break;
                     break;
                 case COMPLETION_STAT:
                 case COMPLETION_STAT:
+                    LOG_DEBUG(("Calling COMPLETION_STAT for xid=%x rc=%d",cptr->xid,rc));
                     if (rc == 0) {
                     if (rc == 0) {
                         struct SetDataResponse res;
                         struct SetDataResponse res;
-                        LOG_DEBUG(("Calling COMPLETION_STAT for xid=%x rc=%d",cptr->xid,rc));
                         deserialize_SetDataResponse(ia, "reply", &res);
                         deserialize_SetDataResponse(ia, "reply", &res);
                         sc->u.stat = res.stat;
                         sc->u.stat = res.stat;
                         deallocate_SetDataResponse(&res);
                         deallocate_SetDataResponse(&res);
                     }
                     }
                     break;
                     break;
                 case COMPLETION_STRINGLIST:
                 case COMPLETION_STRINGLIST:
+                    LOG_DEBUG(("Calling COMPLETION_STRINGLIST for xid=%x rc=%d",cptr->xid,rc));
                     if (rc == 0) {
                     if (rc == 0) {
                         struct GetChildrenResponse res;
                         struct GetChildrenResponse res;
-                        LOG_DEBUG(("Calling COMPLETION_STRINGLIST for xid=%x rc=%d",cptr->xid,rc));
                         deserialize_GetChildrenResponse(ia, "reply", &res);
                         deserialize_GetChildrenResponse(ia, "reply", &res);
                         sc->u.strs = res.children;
                         sc->u.strs = res.children;
                         /* We don't deallocate since we are passing it back */
                         /* We don't deallocate since we are passing it back */
@@ -1414,10 +1433,10 @@ int zookeeper_process(zhandle_t *zh, int events)
                     }
                     }
                     break;
                     break;
                 case COMPLETION_STRING:
                 case COMPLETION_STRING:
+                    LOG_DEBUG(("Calling COMPLETION_STRING for xid=%x rc=%d",cptr->xid,rc));
                     if (rc == 0) {
                     if (rc == 0) {
                         struct CreateResponse res;
                         struct CreateResponse res;
                         int len;
                         int len;
-                        LOG_DEBUG(("Calling COMPLETION_STRING for xid=%x rc=%d",cptr->xid,rc));
                         deserialize_CreateResponse(ia, "reply", &res);
                         deserialize_CreateResponse(ia, "reply", &res);
                         if (sc->u.str.str_len > strlen(res.path)) {
                         if (sc->u.str.str_len > strlen(res.path)) {
                             len = strlen(res.path);
                             len = strlen(res.path);
@@ -1430,9 +1449,9 @@ int zookeeper_process(zhandle_t *zh, int events)
                     }
                     }
                     break;
                     break;
                 case COMPLETION_ACLLIST:
                 case COMPLETION_ACLLIST:
+                    LOG_DEBUG(("Calling COMPLETION_ACLLIST for xid=%x rc=%d",cptr->xid,rc));
                     if (rc == 0) {
                     if (rc == 0) {
                         struct GetACLResponse res;
                         struct GetACLResponse res;
-                        LOG_DEBUG(("Calling COMPLETION_ACLLIST for xid=%x rc=%d",cptr->xid,rc));
                         deserialize_GetACLResponse(ia, "reply", &res);
                         deserialize_GetACLResponse(ia, "reply", &res);
                         cptr->c.acl_result(rc, &res.acl, &res.stat, cptr->data);
                         cptr->c.acl_result(rc, &res.acl, &res.stat, cptr->data);
                         sc->u.acl.acl = res.acl;
                         sc->u.acl.acl = res.acl;
@@ -1445,6 +1464,7 @@ int zookeeper_process(zhandle_t *zh, int events)
                     LOG_DEBUG(("Calling COMPLETION_VOID for xid=%x rc=%d",cptr->xid,rc));
                     LOG_DEBUG(("Calling COMPLETION_VOID for xid=%x rc=%d",cptr->xid,rc));
                     break;
                     break;
                 }
                 }
+                activateWatcher(cptr->watcher,rc);
                 notify_sync_completion(sc);
                 notify_sync_completion(sc);
                 free_buffer(bptr);
                 free_buffer(bptr);
                 zh->outstanding_sync--;
                 zh->outstanding_sync--;
@@ -1468,8 +1488,30 @@ int zoo_state(zhandle_t *zh)
     return 0;
     return 0;
 }
 }
 
 
+static watcher_registration_t* create_watcher_registration(const char* path,
+        result_checker_fn checker,watcher_fn watcher,void* ctx,
+        zk_hashtable* activeMap){
+    watcher_registration_t* wo;
+    if(watcher==0)
+        return 0;
+    wo=calloc(1,sizeof(watcher_registration_t));
+    wo->path=strdup(path);
+    wo->watcher=watcher;
+    wo->context=ctx;
+    wo->checker=checker==0?default_result_checker:checker;
+    wo->activeMap=activeMap;
+    return wo;
+}
+
+static void destroy_watcher_registration(watcher_registration_t* wo){
+    if(wo!=0){
+        free((void*)wo->path);
+        free(wo);
+    }
+}
+
 static completion_list_t* create_completion_entry(int xid, int completion_type, 
 static completion_list_t* create_completion_entry(int xid, int completion_type, 
-        const void *dc, const void *data)
+        const void *dc, const void *data,watcher_registration_t* wo)
 {
 {
     completion_list_t *c = calloc(1,sizeof(completion_list_t));
     completion_list_t *c = calloc(1,sizeof(completion_list_t));
     if (!c) {
     if (!c) {
@@ -1499,15 +1541,24 @@ static completion_list_t* create_completion_entry(int xid, int completion_type,
         break;
         break;
     }
     }
     c->xid = xid;
     c->xid = xid;
-    c->next = 0;
+    c->watcher = wo;
 
 
     return c;
     return c;
 }
 }
 
 
+static void destroy_completion_entry(completion_list_t* c){
+    if(c!=0){
+        if(c->buffer!=0)
+            free_buffer(c->buffer);
+        destroy_watcher_registration(c->watcher);
+        free(c);
+    }
+}
+
 static void queue_completion(completion_head_t *list, completion_list_t *c,
 static void queue_completion(completion_head_t *list, completion_list_t *c,
         int add_to_front)
         int add_to_front)
 {
 {
-     c->next = 0;
+    c->next = 0;
     /* appending a new entry to the back of the list */
     /* appending a new entry to the back of the list */
     lock_completion_list(list);
     lock_completion_list(list);
     if (list->last) {
     if (list->last) {
@@ -1530,10 +1581,11 @@ static void queue_completion(completion_head_t *list, completion_list_t *c,
 }
 }
 
 
 static int add_completion(zhandle_t *zh, int xid, int completion_type,
 static int add_completion(zhandle_t *zh, int xid, int completion_type,
-        const void *dc, const void *data, int add_to_front)
+        const void *dc, const void *data, int add_to_front,
+        watcher_registration_t* wo)
 {
 {
     completion_list_t *c =create_completion_entry(xid, completion_type, dc,
     completion_list_t *c =create_completion_entry(xid, completion_type, dc,
-            data);
+            data,wo);
     if (!c) 
     if (!c) 
         return ZSYSTEMERROR;
         return ZSYSTEMERROR;
     queue_completion(&zh->sent_requests, c, add_to_front);
     queue_completion(&zh->sent_requests, c, add_to_front);
@@ -1544,39 +1596,39 @@ static int add_completion(zhandle_t *zh, int xid, int completion_type,
 }
 }
 
 
 static int add_data_completion(zhandle_t *zh, int xid, data_completion_t dc,
 static int add_data_completion(zhandle_t *zh, int xid, data_completion_t dc,
-        const void *data)
+        const void *data,watcher_registration_t* wo)
 {
 {
-    return add_completion(zh, xid, COMPLETION_DATA, dc, data, 0);
+    return add_completion(zh, xid, COMPLETION_DATA, dc, data, 0,wo);
 }
 }
 
 
 static int add_stat_completion(zhandle_t *zh, int xid, stat_completion_t dc,
 static int add_stat_completion(zhandle_t *zh, int xid, stat_completion_t dc,
-        const void *data)
+        const void *data,watcher_registration_t* wo)
 {
 {
-    return add_completion(zh, xid, COMPLETION_STAT, dc, data, 0);
+    return add_completion(zh, xid, COMPLETION_STAT, dc, data, 0,wo);
 }
 }
 
 
 static int add_strings_completion(zhandle_t *zh, int xid,
 static int add_strings_completion(zhandle_t *zh, int xid,
-        strings_completion_t dc, const void *data)
+        strings_completion_t dc, const void *data,watcher_registration_t* wo)
 {
 {
-    return add_completion(zh, xid, COMPLETION_STRINGLIST, dc, data, 0);
+    return add_completion(zh, xid, COMPLETION_STRINGLIST, dc, data, 0,wo);
 }
 }
 
 
 static int add_acl_completion(zhandle_t *zh, int xid, acl_completion_t dc,
 static int add_acl_completion(zhandle_t *zh, int xid, acl_completion_t dc,
         const void *data)
         const void *data)
 {
 {
-    return add_completion(zh, xid, COMPLETION_ACLLIST, dc, data, 0);
+    return add_completion(zh, xid, COMPLETION_ACLLIST, dc, data, 0,0);
 }
 }
 
 
 static int add_void_completion(zhandle_t *zh, int xid, void_completion_t dc,
 static int add_void_completion(zhandle_t *zh, int xid, void_completion_t dc,
         const void *data)
         const void *data)
 {
 {
-    return add_completion(zh, xid, COMPLETION_VOID, dc, data, 0);
+    return add_completion(zh, xid, COMPLETION_VOID, dc, data, 0,0);
 }
 }
 
 
 static int add_string_completion(zhandle_t *zh, int xid,
 static int add_string_completion(zhandle_t *zh, int xid,
         string_completion_t dc, const void *data)
         string_completion_t dc, const void *data)
 {
 {
-    return add_completion(zh, xid, COMPLETION_STRING, dc, data, 0);
+    return add_completion(zh, xid, COMPLETION_STRING, dc, data, 0,0);
 }
 }
 
 
 int zookeeper_close(zhandle_t *zh)
 int zookeeper_close(zhandle_t *zh)
@@ -1624,10 +1676,17 @@ finish:
 
 
 int zoo_aget(zhandle_t *zh, const char *path, int watch, data_completion_t dc,
 int zoo_aget(zhandle_t *zh, const char *path, int watch, data_completion_t dc,
         const void *data)
         const void *data)
+{
+    return zoo_awget(zh,path,watch?zh->watcher:0,zh->context,dc,data);
+}
+
+int zoo_awget(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx, 
+        data_completion_t dc, const void *data)
 {
 {
     struct oarchive *oa; 
     struct oarchive *oa; 
     struct RequestHeader h = { .xid = get_xid(), .type = GETDATA_OP};
     struct RequestHeader h = { .xid = get_xid(), .type = GETDATA_OP};
-    struct GetDataRequest req = { (char*)path, watch };
+    struct GetDataRequest req = { (char*)path, watcher!=0 };
     int rc;
     int rc;
     
     
     if (zh==0 || path==0)
     if (zh==0 || path==0)
@@ -1638,7 +1697,9 @@ int zoo_aget(zhandle_t *zh, const char *path, int watch, data_completion_t dc,
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_GetDataRequest(oa, "req", &req);
     rc = rc < 0 ? rc : serialize_GetDataRequest(oa, "req", &req);
     enter_critical(zh);
     enter_critical(zh);
-    rc = rc < 0 ? rc : add_data_completion(zh, h.xid, dc, data);
+    rc = rc < 0 ? rc : add_data_completion(zh, h.xid, dc, data,
+        create_watcher_registration(path,0,watcher,watcherCtx,
+                zh->active_node_watchers));
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
             get_buffer_len(oa));
             get_buffer_len(oa));
     leave_critical(zh);
     leave_critical(zh);
@@ -1673,7 +1734,7 @@ int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buflen,
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_SetDataRequest(oa, "req", &req);
     rc = rc < 0 ? rc : serialize_SetDataRequest(oa, "req", &req);
     enter_critical(zh);
     enter_critical(zh);
-    rc = rc < 0 ? rc : add_stat_completion(zh, h.xid, dc, data);
+    rc = rc < 0 ? rc : add_stat_completion(zh, h.xid, dc, data,0);
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
             get_buffer_len(oa));
             get_buffer_len(oa));
     leave_critical(zh);
     leave_critical(zh);
@@ -1761,11 +1822,18 @@ int zoo_adelete(zhandle_t *zh, const char *path, int version,
 }
 }
 
 
 int zoo_aexists(zhandle_t *zh, const char *path, int watch,
 int zoo_aexists(zhandle_t *zh, const char *path, int watch,
+        stat_completion_t sc, const void *data)
+{
+    return zoo_awexists(zh,path,watch?zh->watcher:0,zh->context,sc,data);
+}
+
+int zoo_awexists(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx,
         stat_completion_t completion, const void *data)
         stat_completion_t completion, const void *data)
 {
 {
     struct oarchive *oa;
     struct oarchive *oa;
     struct RequestHeader h = { .xid = get_xid(), .type = EXISTS_OP };
     struct RequestHeader h = { .xid = get_xid(), .type = EXISTS_OP };
-    struct ExistsRequest req;
+    struct ExistsRequest req = {(char*)path, watcher!=0 };
     int rc;
     int rc;
     
     
     if (zh==0 || path==0)
     if (zh==0 || path==0)
@@ -1773,12 +1841,12 @@ int zoo_aexists(zhandle_t *zh, const char *path, int watch,
     if (is_unrecoverable(zh))
     if (is_unrecoverable(zh))
         return ZINVALIDSTATE;
         return ZINVALIDSTATE;
     oa = create_buffer_oarchive();
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
-    req.watch = watch;
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_ExistsRequest(oa, "req", &req);
     rc = rc < 0 ? rc : serialize_ExistsRequest(oa, "req", &req);
     enter_critical(zh);
     enter_critical(zh);
-    rc = rc < 0 ? rc : add_stat_completion(zh, h.xid, completion, data);
+    rc = rc < 0 ? rc : add_stat_completion(zh, h.xid, completion, data,
+        create_watcher_registration(path,exists_result_checker,
+                watcher,watcherCtx,zh->active_node_watchers));
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
             get_buffer_len(oa));
             get_buffer_len(oa));
     leave_critical(zh);
     leave_critical(zh);
@@ -1793,11 +1861,18 @@ int zoo_aexists(zhandle_t *zh, const char *path, int watch,
 }
 }
 
 
 int zoo_aget_children(zhandle_t *zh, const char *path, int watch,
 int zoo_aget_children(zhandle_t *zh, const char *path, int watch,
-        strings_completion_t completion, const void *data)
+        strings_completion_t dc, const void *data)
+{
+    return zoo_awget_children(zh,path,watch?zh->watcher:0,zh->context,dc,data);
+}
+
+int zoo_awget_children(zhandle_t *zh, const char *path,
+        watcher_fn watcher, void* watcherCtx, 
+        strings_completion_t dc, const void *data)
 {
 {
     struct oarchive *oa;
     struct oarchive *oa;
     struct RequestHeader h = { .xid = get_xid(), .type = GETCHILDREN_OP};
     struct RequestHeader h = { .xid = get_xid(), .type = GETCHILDREN_OP};
-    struct GetChildrenRequest req;
+    struct GetChildrenRequest req={(char*)path, watcher!=0 };
     int rc;
     int rc;
     
     
     if (zh==0 || path==0)
     if (zh==0 || path==0)
@@ -1805,12 +1880,12 @@ int zoo_aget_children(zhandle_t *zh, const char *path, int watch,
     if (is_unrecoverable(zh))
     if (is_unrecoverable(zh))
         return ZINVALIDSTATE;
         return ZINVALIDSTATE;
     oa = create_buffer_oarchive();
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
-    req.watch = watch;
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_GetChildrenRequest(oa, "req", &req);
     rc = rc < 0 ? rc : serialize_GetChildrenRequest(oa, "req", &req);
     enter_critical(zh);
     enter_critical(zh);
-    rc = rc < 0 ? rc : add_strings_completion(zh, h.xid, completion, data);
+    rc = rc < 0 ? rc : add_strings_completion(zh, h.xid, dc, data,
+            create_watcher_registration(path,0,watcher,watcherCtx,
+                    zh->active_child_watchers));
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
             get_buffer_len(oa));
             get_buffer_len(oa));
     leave_critical(zh);
     leave_critical(zh);
@@ -2130,13 +2205,19 @@ int zoo_delete(zhandle_t *zh, const char *path, int version)
 }
 }
 
 
 int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *stat)
 int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *stat)
+{
+    return zoo_wexists(zh,path,watch?zh->watcher:0,zh->context,stat);
+}
+
+int zoo_wexists(zhandle_t *zh, const char *path,
+        watcher_fn watcher, void* watcherCtx, struct Stat *stat)
 {
 {
     struct sync_completion *sc = alloc_sync_completion();
     struct sync_completion *sc = alloc_sync_completion();
     int rc;
     int rc;
     if (!sc) {
     if (!sc) {
         return ZSYSTEMERROR;
         return ZSYSTEMERROR;
     }
     }
-    rc=zoo_aexists(zh, path, watch, SYNCHRONOUS_MARKER, sc);
+    rc=zoo_awexists(zh,path,watcher,watcherCtx,SYNCHRONOUS_MARKER, sc);
     if(rc==ZOK){
     if(rc==ZOK){
         wait_sync_completion(sc);
         wait_sync_completion(sc);
         rc = sc->rc;
         rc = sc->rc;
@@ -2145,11 +2226,19 @@ int zoo_exists(zhandle_t *zh, const char *path, int watch, struct Stat *stat)
         }
         }
     }
     }
     free_sync_completion(sc);
     free_sync_completion(sc);
-    return rc;
+    return rc;    
 }
 }
 
 
 int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,
 int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,
         int* buffer_len, struct Stat *stat)
         int* buffer_len, struct Stat *stat)
+{
+    return zoo_wget(zh,path,watch?zh->watcher:0,zh->context,
+            buffer,buffer_len,stat);
+}
+
+int zoo_wget(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx, 
+        char *buffer, int* buffer_len, struct Stat *stat)
 {
 {
     struct sync_completion *sc;
     struct sync_completion *sc;
     int rc=0;
     int rc=0;
@@ -2161,7 +2250,7 @@ int zoo_get(zhandle_t *zh, const char *path, int watch, char *buffer,
 
 
     sc->u.data.buffer = buffer;
     sc->u.data.buffer = buffer;
     sc->u.data.buff_len = *buffer_len;
     sc->u.data.buff_len = *buffer_len;
-    rc=zoo_aget(zh, path, watch, SYNCHRONOUS_MARKER, sc);
+    rc=zoo_awget(zh, path, watcher, watcherCtx, SYNCHRONOUS_MARKER, sc);
     if(rc==ZOK){
     if(rc==ZOK){
         wait_sync_completion(sc);
         wait_sync_completion(sc);
         rc = sc->rc;
         rc = sc->rc;
@@ -2194,13 +2283,20 @@ int zoo_set(zhandle_t *zh, const char *path, const char *buffer, int buflen,
 
 
 int zoo_get_children(zhandle_t *zh, const char *path, int watch,
 int zoo_get_children(zhandle_t *zh, const char *path, int watch,
         struct String_vector *strings)
         struct String_vector *strings)
+{
+    return zoo_wget_children(zh,path,watch?zh->watcher:0,zh->context,strings);
+}
+
+int zoo_wget_children(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx,
+        struct String_vector *strings)
 {
 {
     struct sync_completion *sc = alloc_sync_completion();
     struct sync_completion *sc = alloc_sync_completion();
     int rc;
     int rc;
     if (!sc) {
     if (!sc) {
         return ZSYSTEMERROR;
         return ZSYSTEMERROR;
     }
     }
-    rc=zoo_aget_children(zh, path, watch, SYNCHRONOUS_MARKER, sc);
+    rc=zoo_awget_children(zh, path, watcher, watcherCtx, SYNCHRONOUS_MARKER, sc);
     if(rc==ZOK){
     if(rc==ZOK){
         wait_sync_completion(sc);
         wait_sync_completion(sc);
         rc = sc->rc;
         rc = sc->rc;

+ 195 - 0
src/c/tests/CollectionUtil.h

@@ -0,0 +1,195 @@
+/**
+ * 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.
+ */
+#ifndef _COLLECTION_UTIL_H_
+#define _COLLECTION_UTIL_H_
+
+/**
+ * \file
+ * CollectionBuilder and DictionaryBuilder classes and collection utility functions
+ */
+
+namespace Util 
+{
+
+// *********************************************************
+/** A shortcut to use for building collections.
+ * This class is a wrapper around standard STL collection containers such as vector.
+ * It allows one to conveniently build collections at the variable initialization time:
+ * \code
+ * #include "CollectionUtil.h"
+ * #include "Vector.h"  // for ostream << operator overload for STL vector
+ * using Util;
+ * 
+ * int main()
+ * {
+ *   typedef vector<string> MyVector;
+ *   MyVector myVector=CollectionBuilder<MyVector>()("str1")("str2")("str3");
+ *   cout<<myVector;
+ *   // the following output will be produced:
+ *   // [str1,str2,str3]
+ * }
+ * \endcode
+ */
+template <class CONT>
+class CollectionBuilder
+{
+public:
+  /// Type of the collection container. 
+  typedef CONT CollectionType;
+  /// Container's value type.
+  typedef typename CollectionType::value_type value_type;
+  /// Container's constant iterator type.
+  typedef typename CollectionType::const_iterator const_iterator;
+  /// Container's size type.
+  typedef typename CollectionType::size_type size_type;
+
+  /** Operator function call overload to allow call chaining.
+   * \param value the value to be inserted into the container
+   */
+  CollectionBuilder<CONT>& operator()(const value_type& value){
+    return push_back(value);
+  }
+  /** Same as regular STL push_back() but allows call chaining.
+   * \param value the value to be inserted into the container
+   */
+  CollectionBuilder<CONT>& push_back(const value_type& value){
+    collection_.push_back(value);
+    return *this;
+  }
+  /// \name Standard STL container interface
+  /// @{
+  const_iterator begin() const{return collection_.begin();}
+  const_iterator end() const{return collection_.end();}
+  size_type size() const{return collection_.size();}
+  void clear() {collection_.clear();}
+  ///@}
+  /// Explicit typecast operator.
+  operator const CollectionType&() const {return collection_;}
+private:
+  /// \cond PRIVATE
+  CollectionType collection_;
+  /// \endcond
+};
+
+
+// *********************************************************
+/** A shortcut to use for building dictionaries.
+ * This class is a wrapper around standard STL associative containers such as map.
+ * It allows one to conveniently build dictionaries at the variable initialization time:
+ * \code
+ * #include "CollectionUtil.h"
+ * #include "Map.h"  // for ostream << operator overload for STL map
+ * using Util;
+ * 
+ * int main()
+ * {
+ *   typedef map<string,int> MyMap;
+ *   MyMap myMap=DictionaryBuilder<MyMap>()("str1",1)("str2",2)("str3",3);
+ *   cout<<myMap;
+ *   // the following output will be produced:
+ *   // [str1=1,str2=2,str3=3]
+ * }
+ * \endcode
+ */
+template <class CONT>
+class DictionaryBuilder
+{
+public:
+  /// The type of the associative container
+  typedef CONT DictionaryType;
+  /// Container's element type (usually a pair<key_type,mapped_type>)
+  typedef typename DictionaryType::value_type value_type;
+  /// Container's key type
+  typedef typename DictionaryType::key_type key_type;
+  /// Container's value type 
+  typedef typename DictionaryType::mapped_type mapped_type;
+  /// Container's constant iterator type 
+  typedef typename DictionaryType::const_iterator const_iterator;
+  /// Container's writable iterator type   
+  typedef typename DictionaryType::iterator iterator;
+  /// Container's size type
+  typedef typename DictionaryType::size_type size_type;
+ 
+  /** Operator function call overload to allow call chaining.
+   * \param key the value key to be inserted
+   * \param value the value to be inserted into the container
+   * \return a non-const reference to self
+   */
+  DictionaryBuilder<CONT>& operator()(const key_type& key,const mapped_type& value){
+    dict_.insert(value_type(key,value));
+    return *this;
+  }
+  /** Lookup value by key.
+   * \param key the key associated with the value.
+   * \return a non-const iterator pointing to the element whose key matched the \a key parameter
+   */
+  iterator find(const key_type& key){
+    return dict_.find(key);
+  }
+  /** Lookup value by key.
+   * \param key the key associated with the value.
+   * \return a const iterator pointing to the element whose key matched the \a key parameter
+   */
+  const_iterator find(const key_type& key) const{
+    return dict_.find(key);
+  }
+
+  /// \name Standard STL container interface
+  /// @{
+  const_iterator begin() const{return dict_.begin();}
+  const_iterator end() const{return dict_.end();}
+  size_type size() const{return dict_.size();}
+  void clear() {dict_.clear();}
+  ///@}
+  /// Explicit typecast operator.
+  operator const DictionaryType&() const {return dict_;}
+private:
+  DictionaryType dict_;
+};
+
+
+// ***********************************************************
+/** Deletes all dynamically allocated elements of a collection.
+ * C::value_type is expected to be a pointer to a dynamically allocated object, or it won't compile.
+ * The function will iterate over all container elements and call delete for each of them.
+ * \param c a collection (vector,set) whose elements are being deleted.
+ */
+template <class C>
+void clearCollection(C& c){
+  for(typename C::const_iterator it=c.begin();it!=c.end();++it)
+    delete *it;
+  c.clear();
+}
+
+/** Deletes all dynamically allocated values of the assotiative container.
+ * The function expects the M::value_type to be a pair<..., ptr_to_type>, or it won't compile.
+ * It first deletes the objects pointed to by ptr_to_type
+ * and then clears (calls m.clear()) the container.
+ * \param m an associative container (map,hash_map) whose elements are being deleted.
+ */
+template <class M>
+void clearMap(M& m){
+  for(typename M::const_iterator it=m.begin();it!=m.end();++it)
+    delete it->second;
+  m.clear();
+}
+
+} // namespace Util
+
+
+#endif // _COLLECTION_UTIL_H_

+ 268 - 0
src/c/tests/TestHashtable.cc

@@ -0,0 +1,268 @@
+/**
+ * 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.
+ */
+
+#include <cppunit/extensions/HelperMacros.h>
+#include "CppAssertHelper.h"
+
+
+#include "CollectionUtil.h"
+using namespace Util;
+
+#include "Vector.h"
+using namespace std;
+
+#include "src/zk_hashtable.h"
+
+class Zookeeper_hashtable : public CPPUNIT_NS::TestFixture
+{
+    CPPUNIT_TEST_SUITE(Zookeeper_hashtable);
+    CPPUNIT_TEST(testInsertElement1);
+    CPPUNIT_TEST(testInsertElement2);
+    CPPUNIT_TEST(testInsertElement3);
+    CPPUNIT_TEST(testContainsWatcher1);
+    CPPUNIT_TEST(testContainsWatcher2);
+    CPPUNIT_TEST(testCombineHashtable1);
+    CPPUNIT_TEST(testMoveMergeWatchers1);
+    CPPUNIT_TEST(testDeliverSessionEvent1);
+    CPPUNIT_TEST(testDeliverZnodeEvent1);
+    CPPUNIT_TEST_SUITE_END();
+
+    static void watcher(zhandle_t *, int, int, const char *,void*){}
+    zk_hashtable *ht;
+    
+public:
+
+    void setUp()
+    {
+        ht=create_zk_hashtable();
+    }
+    
+    void tearDown()
+    {
+        destroy_zk_hashtable(ht);
+    }
+
+    static vector<int> getWatcherCtxAsVector(zk_hashtable* ht,const char* path){
+        watcher_object_t* wo=getFirstWatcher(ht,path);
+        vector<int> res;
+        while(wo!=0){
+            res.push_back((int)wo->context);
+            wo=wo->next;
+        }
+        return res;
+    }
+    
+    // insert 2 watchers for different paths
+    // verify that hashtable size is 2
+    void testInsertElement1()
+    {
+        CPPUNIT_ASSERT_EQUAL(0,get_element_count(ht));
+        int res=insert_watcher_object(ht,"path1",
+                create_watcher_object(watcher,(void*)1));
+        CPPUNIT_ASSERT_EQUAL(1,res);
+        res=insert_watcher_object(ht,"path2",
+                create_watcher_object(watcher,(void*)1));
+        CPPUNIT_ASSERT_EQUAL(1,res);
+        CPPUNIT_ASSERT_EQUAL(2,get_element_count(ht));
+        vector<int> expWatchers=CollectionBuilder<vector<int> >().push_back(1);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,getWatcherCtxAsVector(ht,"path1"));
+        CPPUNIT_ASSERT_EQUAL(expWatchers,getWatcherCtxAsVector(ht,"path2"));
+        clean_zk_hashtable(ht);
+        CPPUNIT_ASSERT_EQUAL(0,get_element_count(ht));        
+    }
+    
+    // insert 2 different watchers for the same path;
+    // verify: hashtable element count is 1, and the watcher count for the path
+    // is 2
+    void testInsertElement2()
+    {
+        int res=insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)1));
+        CPPUNIT_ASSERT_EQUAL(1,res);
+        res=insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)2));
+        CPPUNIT_ASSERT_EQUAL(1,res);
+        CPPUNIT_ASSERT_EQUAL(1,get_element_count(ht));
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(ht,"path1"));
+        vector<int> expWatchers=CollectionBuilder<vector<int> >().
+            push_back(2).push_back(1);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,getWatcherCtxAsVector(ht,"path1"));
+    }
+
+    // insert 2 identical watchers for the same path;
+    // verify: hashtable element count is 1, the watcher count for the path is 1
+    void testInsertElement3()
+    {
+        watcher_object_t wobject;
+        wobject.watcher=watcher;
+        wobject.context=(void*)1;
+        
+        int res=insert_watcher_object(ht,"path1",clone_watcher_object(&wobject));
+        CPPUNIT_ASSERT_EQUAL(1,res);
+        watcher_object_t* wo=clone_watcher_object(&wobject);
+        res=insert_watcher_object(ht,"path1",wo);
+        CPPUNIT_ASSERT_EQUAL(0,res);
+        CPPUNIT_ASSERT_EQUAL(1,get_element_count(ht));
+        CPPUNIT_ASSERT_EQUAL(1,get_watcher_count(ht,"path1"));
+        vector<int> expWatchers=CollectionBuilder<vector<int> >().push_back(1);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,getWatcherCtxAsVector(ht,"path1"));
+        // must delete the object that wasn't inserted!
+        free(wo);
+    }
+
+    // verify: the watcher is found in the table
+    void testContainsWatcher1()
+    {
+        watcher_object_t expected;
+        expected.watcher=watcher;
+        expected.context=(void*)1;
+        
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)2));
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)3));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)4));
+        insert_watcher_object(ht,"path2",clone_watcher_object(&expected));
+        
+        CPPUNIT_ASSERT_EQUAL(2,get_element_count(ht));
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(ht,"path1"));
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(ht,"path2"));
+
+        int res=contains_watcher(ht,&expected);
+        CPPUNIT_ASSERT(res==1);
+    }
+
+    // verify: the watcher is not found
+    void testContainsWatcher2()
+    {
+        watcher_object_t expected;
+        expected.watcher=watcher;
+        expected.context=(void*)1;
+        
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)2));
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)3));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)4));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)5));
+        
+        CPPUNIT_ASSERT_EQUAL(2,get_element_count(ht));
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(ht,"path1"));
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(ht,"path2"));
+
+        int res=contains_watcher(ht,&expected);
+        CPPUNIT_ASSERT(res==0);
+    }
+
+    void testCombineHashtable1()
+    {
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)2));
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)3));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)4));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)5));
+        
+        zk_hashtable* ht2=create_zk_hashtable();
+
+        insert_watcher_object(ht2,"path1",create_watcher_object(watcher,(void*)2));
+        insert_watcher_object(ht2,"path2",create_watcher_object(watcher,(void*)6));
+        insert_watcher_object(ht2,"path3",create_watcher_object(watcher,(void*)2));
+
+        zk_hashtable* res=combine_hashtables(ht,ht2);
+        
+        CPPUNIT_ASSERT_EQUAL(3,get_element_count(res));
+        // path1 --> 2,3
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(res,"path1"));
+        vector<int> expWatchers1=CollectionBuilder<vector<int> >().
+            push_back(2).push_back(3);
+        CPPUNIT_ASSERT_EQUAL(expWatchers1,getWatcherCtxAsVector(res,"path1"));
+        // path2 --> 4,5,6
+        CPPUNIT_ASSERT_EQUAL(3,get_watcher_count(res,"path2"));
+        vector<int> expWatchers2=CollectionBuilder<vector<int> >().
+            push_back(6).push_back(4).push_back(5);
+        CPPUNIT_ASSERT_EQUAL(expWatchers2,getWatcherCtxAsVector(res,"path2"));
+        // path3 --> 2
+        CPPUNIT_ASSERT_EQUAL(1,get_watcher_count(res,"path3"));
+        vector<int> expWatchers3=CollectionBuilder<vector<int> >().push_back(2);
+        CPPUNIT_ASSERT_EQUAL(expWatchers3,getWatcherCtxAsVector(res,"path3"));
+
+        destroy_zk_hashtable(ht2);
+        destroy_zk_hashtable(res);
+    }
+    
+    void testMoveMergeWatchers1()
+    {
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)2));
+        insert_watcher_object(ht,"path1",create_watcher_object(watcher,(void*)3));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)4));
+        insert_watcher_object(ht,"path2",create_watcher_object(watcher,(void*)5));
+        
+        zk_hashtable* ht2=create_zk_hashtable();
+
+        insert_watcher_object(ht2,"path1",create_watcher_object(watcher,(void*)2));
+        insert_watcher_object(ht2,"path1",create_watcher_object(watcher,(void*)6));
+
+        zk_hashtable* res=move_merge_watchers(ht,ht2,"path1");
+        
+        CPPUNIT_ASSERT_EQUAL(1,get_element_count(res));
+        CPPUNIT_ASSERT_EQUAL(3,get_watcher_count(res,"path1"));
+        vector<int> expWatchers=CollectionBuilder<vector<int> >().
+            push_back(6).push_back(2).push_back(3);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,getWatcherCtxAsVector(res,"path1"));
+
+        // make sure the path entry has been deleted from the source hashtables
+        CPPUNIT_ASSERT_EQUAL(1,get_element_count(ht));
+        CPPUNIT_ASSERT_EQUAL(2,get_watcher_count(ht,"path2"));
+        CPPUNIT_ASSERT_EQUAL(0,get_element_count(ht2));
+        
+        destroy_zk_hashtable(ht2);
+        destroy_zk_hashtable(res);
+    }
+
+    static void iterWatcher(zhandle_t *zh, int type, int state, 
+            const char* path,void* ctx){
+        vector<int>* res=reinterpret_cast<vector<int>*>(zh);
+        res->push_back((int)ctx);
+    }
+    
+    void testDeliverSessionEvent1(){
+        insert_watcher_object(ht,"path1",create_watcher_object(iterWatcher,(void*)2));
+        insert_watcher_object(ht,"path2",create_watcher_object(iterWatcher,(void*)3));
+        insert_watcher_object(ht,"path2",create_watcher_object(iterWatcher,(void*)4));
+        insert_watcher_object(ht,"path3",create_watcher_object(iterWatcher,(void*)5));
+        
+        vector<int> res;
+        deliver_session_event(ht,(zhandle_t*)&res,10,20);
+        vector<int> expWatchers=CollectionBuilder<vector<int> >().
+            push_back(4).push_back(3).push_back(5).push_back(2);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,res);
+    }
+    
+    void testDeliverZnodeEvent1(){
+        insert_watcher_object(ht,"path2",create_watcher_object(iterWatcher,(void*)3));
+        insert_watcher_object(ht,"path2",create_watcher_object(iterWatcher,(void*)4));
+        
+        vector<int> res;
+        deliver_znode_event(ht,(zhandle_t*)&res,"path2",10,20);
+        vector<int> expWatchers=CollectionBuilder<vector<int> >().
+            push_back(4).push_back(3);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,res);
+        expWatchers.clear();
+        res.clear();
+        // non-existent path
+        deliver_znode_event(ht,(zhandle_t*)&res,"path100",10,20);
+        CPPUNIT_ASSERT_EQUAL(expWatchers,res);
+        // make sure the path entry has been deleted from the source hashtable
+        CPPUNIT_ASSERT_EQUAL(0,get_element_count(ht));
+    }
+};
+
+CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_hashtable);

+ 24 - 5
src/c/tests/TestOperations.cc

@@ -41,7 +41,7 @@ class Zookeeper_operations : public CPPUNIT_NS::TestFixture
     CPPUNIT_TEST_SUITE_END();
     CPPUNIT_TEST_SUITE_END();
     zhandle_t *zh;
     zhandle_t *zh;
 
 
-    static void watcher(zhandle_t *, int, int, const char *){}
+    static void watcher(zhandle_t *, int, int, const char *,void*){}
 public: 
 public: 
     void setUp()
     void setUp()
     {
     {
@@ -256,7 +256,7 @@ public:
         zkServer.addRecvResponse(new PingResponse);
         zkServer.addRecvResponse(new PingResponse);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         CPPUNIT_ASSERT_EQUAL(ZOK,rc);
         CPPUNIT_ASSERT_EQUAL(ZOK,rc);
-        // sleep for a short while (10 ms)
+        // pseudo-sleep for a short while (10 ms)
         timeMock.millitick(10);
         timeMock.millitick(10);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
         CPPUNIT_ASSERT_EQUAL(ZOK,rc);
         CPPUNIT_ASSERT_EQUAL(ZOK,rc);
@@ -575,12 +575,26 @@ public:
             changed_=true;
             changed_=true;
             if(path!=0) path_=path;
             if(path!=0) path_=path;
         }
         }
+        // this predicate checks if CHANGE_EVENT event type was triggered, unlike
+        // the isWatcherTriggered() that returns true whenever a watcher is triggered
+        // regardless of the event type
         SyncedBoolCondition isNodeChangedTriggered() const{
         SyncedBoolCondition isNodeChangedTriggered() const{
             return SyncedBoolCondition(changed_,mx_);
             return SyncedBoolCondition(changed_,mx_);
         }
         }
         bool changed_;
         bool changed_;
         string path_;
         string path_;
     };
     };
+    
+    class AsyncWatcherCompletion: public AsyncCompletion{
+    public:
+        AsyncWatcherCompletion(ZookeeperServer& zkServer):zkServer_(zkServer){}
+        virtual void statCompl(int rc, const Stat *stat){
+            // we received a server response, now enqueue a watcher event
+            // to trigger the watcher
+            zkServer_.addRecvResponse(new ZNodeEvent(CHANGED_EVENT,"/x/y/z"));
+        }
+        ZookeeperServer& zkServer_;
+    };
     // verify that async watcher is called for znode events (CREATED, DELETED etc.)
     // verify that async watcher is called for znode events (CREATED, DELETED etc.)
     void testAsyncWatcher1(){
     void testAsyncWatcher1(){
         Mock_gettimeofday timeMock;
         Mock_gettimeofday timeMock;
@@ -596,9 +610,14 @@ public:
         CPPUNIT_ASSERT(zh!=0);
         CPPUNIT_ASSERT(zh!=0);
         // make sure the client has connected
         // make sure the client has connected
         CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
-
-        // trigger the watcher
-        zkServer.addRecvResponse(new ZNodeEvent(CHANGED_EVENT,"/x/y/z"));
+        
+        // set the watcher
+        AsyncWatcherCompletion completion(zkServer);
+        // prepare a response for the zoo_aexists() request
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_aexists(zh,"/x/y/z",1,asyncCompletion,&completion);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
         CPPUNIT_ASSERT(ensureCondition(action.isNodeChangedTriggered(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(action.isNodeChangedTriggered(),1000)<1000);
         CPPUNIT_ASSERT_EQUAL(string("/x/y/z"),action.path_);                
         CPPUNIT_ASSERT_EQUAL(string("/x/y/z"),action.path_);                
     }
     }

+ 745 - 0
src/c/tests/TestWatchers.cc

@@ -0,0 +1,745 @@
+/**
+ * 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.
+ */
+
+#include <cppunit/extensions/HelperMacros.h>
+#include "CppAssertHelper.h"
+
+#include "ZKMocks.h"
+#include "CollectionUtil.h"
+
+class Zookeeper_watchers : public CPPUNIT_NS::TestFixture
+{
+    CPPUNIT_TEST_SUITE(Zookeeper_watchers);
+    CPPUNIT_TEST(testDefaultSessionWatcher1);
+    CPPUNIT_TEST(testDefaultSessionWatcher2);
+    CPPUNIT_TEST(testObjectSessionWatcher1);
+    CPPUNIT_TEST(testObjectSessionWatcher2);
+    CPPUNIT_TEST(testNodeWatcher1);
+    CPPUNIT_TEST(testChildWatcher1);
+    CPPUNIT_TEST(testChildWatcher2);
+    CPPUNIT_TEST_SUITE_END();
+
+    static void watcher(zhandle_t *, int, int, const char *,void*){}
+    zhandle_t *zh;
+    
+public:
+
+    void setUp()
+    {
+        zoo_set_debug_level((ZooLogLevel)0); // disable logging
+        zoo_deterministic_conn_order(0);
+        zh=0;
+    }
+    
+    void tearDown()
+    {
+        zookeeper_close(zh);
+    }
+    
+    class ConnectionWatcher: public WatcherAction{
+    public:
+        ConnectionWatcher():connected_(false),counter_(0){}
+        virtual void onConnectionEstablished(zhandle_t*){
+            synchronized(mx_);
+            counter_++;
+            connected_=true;
+        }
+        SyncedBoolCondition isConnectionEstablished() const{
+            return SyncedBoolCondition(connected_,mx_);
+        }
+        bool connected_;
+        int counter_;
+    };
+
+    class DisconnectWatcher: public WatcherAction{
+    public:
+        DisconnectWatcher():disconnected_(false),counter_(0){}
+        virtual void onConnectionLost(zhandle_t*){
+            synchronized(mx_);
+            counter_++;
+            disconnected_=true;
+        }
+        SyncedBoolCondition isDisconnected() const{
+            return SyncedBoolCondition(disconnected_,mx_);
+        }
+        bool disconnected_;
+        int counter_;
+    };
+
+    class CountingDataWatcher: public WatcherAction{
+    public:
+        CountingDataWatcher():disconnected_(false),counter_(0){}
+        virtual void onNodeValueChanged(zhandle_t*,const char* path){
+            synchronized(mx_);
+            counter_++;
+        }
+        virtual void onConnectionLost(zhandle_t*){
+            synchronized(mx_);
+            counter_++;
+            disconnected_=true;
+        }
+        bool disconnected_;
+        int counter_;
+    };
+
+    class DeletionCountingDataWatcher: public WatcherAction{
+    public:
+        DeletionCountingDataWatcher():counter_(0){}
+        virtual void onNodeDeleted(zhandle_t*,const char* path){
+            synchronized(mx_);
+            counter_++;
+        }
+        int counter_;
+    };
+
+    class ChildEventCountingWatcher: public WatcherAction{
+    public:
+        ChildEventCountingWatcher():counter_(0){}
+        virtual void onChildChanged(zhandle_t*,const char* path){
+            synchronized(mx_);
+            counter_++;
+        }
+        int counter_;
+    };
+
+#ifndef THREADED
+    
+    // verify: the default watcher is called once for a session event
+    void testDefaultSessionWatcher1(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        ConnectionWatcher watcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &watcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        
+        int fd=0;
+        int interest=0;
+        timeval tv;
+        // open the socket
+        int rc=zookeeper_interest(zh,&fd,&interest,&tv);        
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL(CONNECTING_STATE,zoo_state(zh));
+        // send the handshake packet to the server
+        rc=zookeeper_process(zh,interest);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL(ASSOCIATING_STATE,zoo_state(zh));
+        // receive the server handshake response
+        rc=zookeeper_process(zh,interest);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // verify connected
+        CPPUNIT_ASSERT_EQUAL(CONNECTED_STATE,zoo_state(zh));
+        CPPUNIT_ASSERT(watcher.connected_);
+        CPPUNIT_ASSERT_EQUAL(1,watcher.counter_);
+    }
+    
+    // test case: connect to server, set a default watcher, disconnect from the server
+    // verify: the default watcher is called once
+    void testDefaultSessionWatcher2(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        DisconnectWatcher watcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &watcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // simulate connected state
+        forceConnected(zh);
+        
+        // first operation
+        AsyncCompletion ignored;
+        zkServer.addOperationResponse(new ZooGetResponse("1",1));
+        int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // this will process the response and activate the watcher
+        rc=zookeeper_process(zh,ZOOKEEPER_READ);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        // now, disconnect
+        zkServer.setConnectionLost();
+        rc=zookeeper_process(zh,ZOOKEEPER_READ);
+        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
+        // verify disconnected
+        CPPUNIT_ASSERT(watcher.disconnected_);
+        CPPUNIT_ASSERT_EQUAL(1,watcher.counter_);
+    }
+    
+    // testcase: connect to the server, set a watcher object on a node, 
+    //           disconnect from the server
+    // verify: the watcher object as well as the default watcher are called
+    void testObjectSessionWatcher1(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        DisconnectWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // simulate connected state
+        forceConnected(zh);
+        
+        AsyncCompletion ignored;
+        CountingDataWatcher wobject;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_awexists(zh,"/x/y/1",activeWatcher,&wobject,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // this will process the response and activate the watcher
+        rc=zookeeper_process(zh,ZOOKEEPER_READ);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        // now, disconnect
+        zkServer.setConnectionLost();
+        rc=zookeeper_process(zh,ZOOKEEPER_READ);
+        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
+
+        // verify the default watcher has been triggered
+        CPPUNIT_ASSERT(defWatcher.disconnected_);
+        // and triggered only once
+        CPPUNIT_ASSERT_EQUAL(1,defWatcher.counter_);
+        
+        // the path-specific watcher has been triggered as well
+        CPPUNIT_ASSERT(wobject.disconnected_);
+        // only once!
+        CPPUNIT_ASSERT_EQUAL(1,wobject.counter_);
+    }
+
+    // testcase: connect to the server, set a watcher object on a node, 
+    //           set a def watcher on another node,disconnect from the server
+    // verify: the watcher object as well as the default watcher are called
+    void testObjectSessionWatcher2(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        DisconnectWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // simulate connected state
+        forceConnected(zh);
+        
+        // set the default watcher
+        AsyncCompletion ignored;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_aexists(zh,"/a/b/c",1,asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        CountingDataWatcher wobject;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        // this will process the response and activate the watcher
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+
+        // disconnect now
+        zkServer.setConnectionLost();
+        rc=zookeeper_process(zh,ZOOKEEPER_READ);
+        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
+
+        // verify the default watcher has been triggered
+        CPPUNIT_ASSERT(defWatcher.disconnected_);
+        // and triggered only once
+        CPPUNIT_ASSERT_EQUAL(1,defWatcher.counter_);
+        
+        // the path-specific watcher has been triggered as well
+        CPPUNIT_ASSERT(wobject.disconnected_);
+        // only once!
+        CPPUNIT_ASSERT_EQUAL(1,wobject.counter_);
+    }
+
+    // testcase: register 2 node watches for different paths, trigger the watches
+    // verify: the data watchers are processed, the default watcher is not called
+    void testNodeWatcher1(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        DisconnectWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // simulate connected state
+        forceConnected(zh);
+        
+        AsyncCompletion ignored;
+        CountingDataWatcher wobject1;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_awexists(zh,"/a/b/c",activeWatcher,&wobject1,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        CountingDataWatcher wobject2;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject2,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        // this will process the response and activate the watcher
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+
+        // we are all set now; let's trigger the watches
+        zkServer.addRecvResponse(new ZNodeEvent(CHANGED_EVENT,"/a/b/c"));
+        zkServer.addRecvResponse(new ZNodeEvent(CHANGED_EVENT,"/x/y/z"));
+        // make sure all watchers have been processed
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        
+        CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
+        CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
+        CPPUNIT_ASSERT_EQUAL(0,defWatcher.counter_);
+    }
+
+    // testcase: set up both a children and a data watchers on the node /a, then
+    //           delete the node by sending a DELETE_EVENT event
+    // verify: both watchers are triggered
+    void testChildWatcher1(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        DeletionCountingDataWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // simulate connected state
+        forceConnected(zh);
+        
+        AsyncCompletion ignored;
+        DeletionCountingDataWatcher wobject1;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_awexists(zh,"/a",activeWatcher,&wobject1,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        typedef ZooGetChildrenResponse::StringVector ZooVector;
+        zkServer.addOperationResponse(new ZooGetChildrenResponse(
+                Util::CollectionBuilder<ZooVector>()("/a/1")("/a/2")
+                ));
+        DeletionCountingDataWatcher wobject2;
+        rc=zoo_awget_children(zh,"/a",activeWatcher,
+                &wobject2,asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        // this will process the response and activate the watcher
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+
+        // we are all set now; let's trigger the watches
+        zkServer.addRecvResponse(new ZNodeEvent(DELETED_EVENT,"/a"));
+        // make sure the watchers have been processed
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+
+        CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
+        CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
+        CPPUNIT_ASSERT_EQUAL(0,defWatcher.counter_);
+    }
+
+    // testcase: create both a child and data watch on the node /a, send a CHILD_EVENT
+    // verify: only the child watch triggered
+    void testChildWatcher2(){
+        Mock_gettimeofday timeMock;
+        ZookeeperServer zkServer;
+        // must call zookeeper_close() while all the mocks are in scope
+        CloseFinally guard(&zh);
+
+        ChildEventCountingWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // simulate connected state
+        forceConnected(zh);
+        
+        AsyncCompletion ignored;
+        ChildEventCountingWatcher wobject1;
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_awexists(zh,"/a",activeWatcher,&wobject1,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        typedef ZooGetChildrenResponse::StringVector ZooVector;
+        zkServer.addOperationResponse(new ZooGetChildrenResponse(
+                Util::CollectionBuilder<ZooVector>()("/a/1")("/a/2")
+                ));
+        ChildEventCountingWatcher wobject2;
+        rc=zoo_awget_children(zh,"/a",activeWatcher,
+                &wobject2,asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        // this will process the response and activate the watcher
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+
+        // we are all set now; let's trigger the watches
+        zkServer.addRecvResponse(new ZNodeEvent(CHILD_EVENT,"/a"));
+        // make sure the watchers have been processed
+        while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
+        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+
+        CPPUNIT_ASSERT_EQUAL(0,wobject1.counter_);
+        CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
+        CPPUNIT_ASSERT_EQUAL(0,defWatcher.counter_);
+    }
+
+#else
+    // verify: the default watcher is called once for a session event
+    void testDefaultSessionWatcher1(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(SESSION_EVENT,CONNECTED_STATE);
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        ConnectionWatcher watcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &watcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // wait till watcher proccessing has completed (the connection 
+        // established event)
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.isWatcherProcessingCompleted(),1000)<1000);
+        
+        // verify the watcher has been triggered
+        CPPUNIT_ASSERT(ensureCondition(watcher.isConnectionEstablished(),1000)<1000);
+        // triggered only once
+        CPPUNIT_ASSERT_EQUAL(1,watcher.counter_);
+    }
+
+    // test case: connect to server, set a default watcher, disconnect from the server
+    // verify: the default watcher is called once
+    void testDefaultSessionWatcher2(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(SESSION_EVENT,CONNECTING_STATE);
+        DisconnectWatcher watcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &watcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // make sure the client has connected
+        CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
+        // set a default watch
+        AsyncCompletion ignored;
+        // a successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        int rc=zoo_aexists(zh,"/x/y/z",1,asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        // now, initiate a disconnect
+        zkServer.setConnectionLost();
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.isWatcherProcessingCompleted(),1000)<1000);
+        
+        // verify the watcher has been triggered
+        CPPUNIT_ASSERT(watcher.disconnected_);
+        // triggered only once
+        CPPUNIT_ASSERT_EQUAL(1,watcher.counter_);
+    }
+
+    // testcase: connect to the server, set a watcher object on a node, 
+    //           disconnect from the server
+    // verify: the watcher object as well as the default watcher are called
+    void testObjectSessionWatcher1(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(SESSION_EVENT,CONNECTING_STATE);
+        DisconnectWatcher defWatcher;
+        // use the tracker to find out when the watcher has been activated
+        WatcherActivationTracker activationTracker;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // make sure the client has connected
+        CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
+
+        AsyncCompletion ignored;
+        // this successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        CountingDataWatcher wobject;
+        activationTracker.track(&wobject);
+        // set a path-specific watcher
+        int rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // make sure the watcher gets activated before we continue
+        CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
+        
+        // now, initiate a disconnect
+        zkServer.setConnectionLost();
+        // make sure all watchers have been processed
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.isWatcherProcessingCompleted(),1000)<1000);
+        
+        // verify the default watcher has been triggered
+        CPPUNIT_ASSERT(defWatcher.disconnected_);
+        // and triggered only once
+        CPPUNIT_ASSERT_EQUAL(1,defWatcher.counter_);
+        
+        // the path-specific watcher has been triggered as well
+        CPPUNIT_ASSERT(wobject.disconnected_);
+        // only once!
+        CPPUNIT_ASSERT_EQUAL(1,wobject.counter_);
+    }
+
+    // testcase: connect to the server, set a watcher object on a node, 
+    //           set a def watcher on another node,disconnect from the server
+    // verify: the watcher object as well as the default watcher are called
+    void testObjectSessionWatcher2(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(SESSION_EVENT,CONNECTING_STATE);
+        DisconnectWatcher defWatcher;
+        // use the tracker to find out when the watcher has been activated
+        WatcherActivationTracker activationTracker;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // make sure the client has connected
+        CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
+
+        // set a default watch
+        AsyncCompletion ignored;
+        // a successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        activationTracker.track(&defWatcher);
+        int rc=zoo_aexists(zh,"/a/b/c",1,asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // make sure the watcher gets activated before we continue
+        CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
+
+        // this successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        CountingDataWatcher wobject;
+        activationTracker.track(&wobject);
+        // set a path-specific watcher
+        rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // make sure the watcher gets activated before we continue
+        CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
+        
+        // now, initiate a disconnect
+        zkServer.setConnectionLost();
+        // make sure all watchers have been processed
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.isWatcherProcessingCompleted(),1000)<1000);
+        
+        // verify the default watcher has been triggered
+        CPPUNIT_ASSERT(defWatcher.disconnected_);
+        // and triggered only once
+        CPPUNIT_ASSERT_EQUAL(1,defWatcher.counter_);
+        
+        // the path-specific watcher has been triggered as well
+        CPPUNIT_ASSERT(wobject.disconnected_);
+        // only once!
+        CPPUNIT_ASSERT_EQUAL(1,wobject.counter_);
+    }
+
+    // testcase: register 2 node watches for different paths, trigger the watches
+    // verify: the data watchers are processed, the default watcher is not called
+    void testNodeWatcher1(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(CHANGED_EVENT,0,false);
+        CountingDataWatcher defWatcher;
+        // use the tracker to find out when the watcher has been activated
+        WatcherActivationTracker activationTracker;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // make sure the client has connected
+        CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
+
+        // don't care about completions
+        AsyncCompletion ignored;
+        // set a one-shot watch
+        // a successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        CountingDataWatcher wobject1;
+        activationTracker.track(&wobject1);
+        int rc=zoo_awexists(zh,"/a/b/c",activeWatcher,&wobject1,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // make sure the watcher gets activated before we continue
+        CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
+
+        // this successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        CountingDataWatcher wobject2;
+        activationTracker.track(&wobject2);
+        // set a path-specific watcher
+        rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject2,
+                asyncCompletion,&ignored);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        // make sure the watcher gets activated before we continue
+        CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
+
+        // we are all set now; let's trigger the watches
+        zkServer.addRecvResponse(new ZNodeEvent(CHANGED_EVENT,"/a/b/c"));
+        zkServer.addRecvResponse(new ZNodeEvent(CHANGED_EVENT,"/x/y/z"));
+        // make sure all watchers have been processed
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.deliveryCounterEquals(2),1000)<1000);
+        
+        CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
+        CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
+        CPPUNIT_ASSERT_EQUAL(0,defWatcher.counter_);
+    }
+
+    // testcase: set up both a children and a data watchers on the node /a, then
+    //           delete the node (that is, send a DELETE_EVENT)
+    // verify: both watchers are triggered
+    void testChildWatcher1(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(DELETED_EVENT,0);
+        DeletionCountingDataWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // make sure the client has connected
+        CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
+        
+        // a successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        DeletionCountingDataWatcher wobject1;
+        Stat stat;
+        // add a node watch
+        int rc=zoo_wexists(zh,"/a",activeWatcher,&wobject1,&stat);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        typedef ZooGetChildrenResponse::StringVector ZooVector;
+        zkServer.addOperationResponse(new ZooGetChildrenResponse(
+                Util::CollectionBuilder<ZooVector>()("/a/1")("/a/2")
+                ));
+        DeletionCountingDataWatcher wobject2;
+        String_vector children;
+        rc=zoo_wget_children(zh,"/a",activeWatcher,&wobject2,&children);
+        deallocate_String_vector(&children);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        // we are all set now; let's trigger the watches
+        zkServer.addRecvResponse(new ZNodeEvent(DELETED_EVENT,"/a"));
+        // make sure the watchers have been processed
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.isWatcherProcessingCompleted(),1000)<1000);
+
+        CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
+        CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
+        CPPUNIT_ASSERT_EQUAL(0,defWatcher.counter_);
+    }
+    
+    // testcase: create both a child and data watch on the node /a, send a CHILD_EVENT
+    // verify: only the child watch triggered
+    void testChildWatcher2(){
+        Mock_gettimeofday timeMock;
+        // zookeeper simulator
+        ZookeeperServer zkServer;
+        Mock_poll pollMock(&zkServer,ZookeeperServer::FD);
+        // must call zookeeper_close() while all the mocks are in the scope!
+        CloseFinally guard(&zh);
+        
+        // detects when all watchers have been delivered
+        WatcherDeliveryTracker deliveryTracker(CHILD_EVENT,0);
+        ChildEventCountingWatcher defWatcher;
+        zh=zookeeper_init("localhost:2121",activeWatcher,10000,TEST_CLIENT_ID,
+                &defWatcher,0);
+        CPPUNIT_ASSERT(zh!=0);
+        // make sure the client has connected
+        CPPUNIT_ASSERT(ensureCondition(ClientConnected(zh),1000)<1000);
+        
+        // a successful server response will activate the watcher 
+        zkServer.addOperationResponse(new ZooStatResponse);
+        ChildEventCountingWatcher wobject1;
+        Stat stat;
+        // add a node watch
+        int rc=zoo_wexists(zh,"/a",activeWatcher,&wobject1,&stat);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        
+        typedef ZooGetChildrenResponse::StringVector ZooVector;
+        zkServer.addOperationResponse(new ZooGetChildrenResponse(
+                Util::CollectionBuilder<ZooVector>()("/a/1")("/a/2")
+                ));
+        ChildEventCountingWatcher wobject2;
+        String_vector children;
+        rc=zoo_wget_children(zh,"/a",activeWatcher,&wobject2,&children);
+        deallocate_String_vector(&children);
+        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+
+        // we are all set now; let's trigger the watches
+        zkServer.addRecvResponse(new ZNodeEvent(CHILD_EVENT,"/a"));
+        // make sure the watchers have been processed
+        CPPUNIT_ASSERT(ensureCondition(
+                deliveryTracker.isWatcherProcessingCompleted(),1000)<1000);
+
+        CPPUNIT_ASSERT_EQUAL(0,wobject1.counter_);
+        CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
+        CPPUNIT_ASSERT_EQUAL(0,defWatcher.counter_);
+    }
+
+#endif //THREADED
+};
+
+CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_watchers);

+ 6 - 4
src/c/tests/TestZookeeperClose.cc

@@ -38,7 +38,7 @@ class Zookeeper_close : public CPPUNIT_NS::TestFixture
     CPPUNIT_TEST(testCloseFromWatcher1);
     CPPUNIT_TEST(testCloseFromWatcher1);
     CPPUNIT_TEST_SUITE_END();
     CPPUNIT_TEST_SUITE_END();
     zhandle_t *zh;
     zhandle_t *zh;
-    static void watcher(zhandle_t *, int, int, const char *){}
+    static void watcher(zhandle_t *, int, int, const char *,void*){}
 public: 
 public: 
     void setUp()
     void setUp()
     {
     {
@@ -59,7 +59,7 @@ public:
         virtual void onSessionExpired(zhandle_t* zh){
         virtual void onSessionExpired(zhandle_t* zh){
             memcpy(&lzh,zh,sizeof(lzh));
             memcpy(&lzh,zh,sizeof(lzh));
             if(callClose_)
             if(callClose_)
-                rc=zookeeper_close(zh);           
+                rc=zookeeper_close(zh);
         }
         }
         zhandle_t lzh;
         zhandle_t lzh;
         bool callClose_;
         bool callClose_;
@@ -88,7 +88,7 @@ public:
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.addrs));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.addrs));
-        CPPUNIT_ASSERT_EQUAL(3,freeMock.callCounter);
+        CPPUNIT_ASSERT_EQUAL(9,freeMock.callCounter);
     }
     }
     void testCloseUnconnected1()
     void testCloseUnconnected1()
     {
     {
@@ -236,7 +236,7 @@ public:
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.addrs));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.addrs));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(adaptor));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(adaptor));
-        CPPUNIT_ASSERT_EQUAL(4,freeMock.callCounter);
+        CPPUNIT_ASSERT_EQUAL(10,freeMock.callCounter);
         // threads
         // threads
         CPPUNIT_ASSERT_EQUAL(1,MockPthreadsNull::getDestroyCounter(adaptor->io));
         CPPUNIT_ASSERT_EQUAL(1,MockPthreadsNull::getDestroyCounter(adaptor->io));
         CPPUNIT_ASSERT_EQUAL(1,MockPthreadsNull::getDestroyCounter(adaptor->completion));
         CPPUNIT_ASSERT_EQUAL(1,MockPthreadsNull::getDestroyCounter(adaptor->completion));
@@ -410,6 +410,8 @@ public:
             CPPUNIT_ASSERT(zh!=0);
             CPPUNIT_ASSERT(zh!=0);
             CPPUNIT_ASSERT(ensureCondition(SessionExpired(zh),1000)<1000);
             CPPUNIT_ASSERT(ensureCondition(SessionExpired(zh),1000)<1000);
             CPPUNIT_ASSERT(ensureCondition(IOThreadStopped(zh),1000)<1000);
             CPPUNIT_ASSERT(ensureCondition(IOThreadStopped(zh),1000)<1000);
+            // make sure the watcher has been processed
+            CPPUNIT_ASSERT(ensureCondition(closeAction.isWatcherTriggered(),1000)<1000);
             // make sure the threads have not been destroyed yet
             // make sure the threads have not been destroyed yet
             adaptor_threads* adaptor=(adaptor_threads*)zh->adaptor_priv;
             adaptor_threads* adaptor=(adaptor_threads*)zh->adaptor_priv;
             CPPUNIT_ASSERT_EQUAL(0,CheckedPthread::getDestroyCounter(adaptor->io));
             CPPUNIT_ASSERT_EQUAL(0,CheckedPthread::getDestroyCounter(adaptor->io));

+ 1 - 1
src/c/tests/TestZookeeperInit.cc

@@ -49,7 +49,7 @@ class Zookeeper_init : public CPPUNIT_NS::TestFixture
 	CPPUNIT_TEST_SUITE_END();
 	CPPUNIT_TEST_SUITE_END();
     zhandle_t *zh;
     zhandle_t *zh;
     MockPthreadsNull* pthreadMock;   
     MockPthreadsNull* pthreadMock;   
-    static void watcher(zhandle_t *, int , int , const char *){}
+    static void watcher(zhandle_t *, int , int , const char *,void*){}
 public: 
 public: 
     Zookeeper_init():zh(0),pthreadMock(0){}
     Zookeeper_init():zh(0),pthreadMock(0){}
     
     

+ 37 - 0
src/c/tests/Vector.h

@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+#ifndef _VECTOR_UTIL_H
+#define _VECTOR_UTIL_H
+
+#include <vector>
+
+// function to conveniently stream vectors
+template <class U>
+std::ostream& operator<<(std::ostream& os,const std::vector<U>& c){
+  typedef std::vector<U> V;
+  os<<"[";
+  if(c.size()>0){
+      for(typename V::const_iterator it=c.begin();it!=c.end();++it)
+          os<<*it<<",";
+      os.seekp(-1,std::ios::cur);
+  }
+  os<<"]";
+  return os;
+}
+
+#endif // _VECTOR_UTIL_H

+ 222 - 8
src/c/tests/ZKMocks.cc

@@ -66,17 +66,26 @@ HandshakeRequest* HandshakeRequest::parse(const std::string& buf){
 
 
 // *****************************************************************************
 // *****************************************************************************
 // watcher action implementation
 // watcher action implementation
-void activeWatcher(zhandle_t *zh, int type, int state, const char *path){
-    if(zh==0 || zoo_get_context(zh)==0) return;
-    WatcherAction* action=(WatcherAction*)zoo_get_context(zh);
-    action->setWatcherTriggered();    
+void activeWatcher(zhandle_t *zh, int type, int state, const char *path,void* ctx){
+    if(zh==0 || ctx==0) return;
+    WatcherAction* action=(WatcherAction*)ctx;
     
     
-    if(type==SESSION_EVENT && state==EXPIRED_SESSION_STATE)
-        action->onSessionExpired(zh);
-    if(type==CHANGED_EVENT)
+    if(type==SESSION_EVENT){
+        if(state==EXPIRED_SESSION_STATE)
+            action->onSessionExpired(zh);
+        else if(state==CONNECTING_STATE)
+            action->onConnectionLost(zh);
+        else if(state==CONNECTED_STATE)
+            action->onConnectionEstablished(zh);
+    }else if(type==CHANGED_EVENT)
         action->onNodeValueChanged(zh,path);
         action->onNodeValueChanged(zh,path);
+    else if(type==DELETED_EVENT)
+        action->onNodeDeleted(zh,path);
+    else if(type==CHILD_EVENT)
+        action->onChildChanged(zh,path);
     // TODO: implement for the rest of the event types
     // TODO: implement for the rest of the event types
     // ...
     // ...
+    action->setWatcherTriggered();    
 }
 }
 SyncedBoolCondition WatcherAction::isWatcherTriggered() const{
 SyncedBoolCondition WatcherAction::isWatcherTriggered() const{
     return SyncedBoolCondition(triggered_,mx_);
     return SyncedBoolCondition(triggered_,mx_);
@@ -144,6 +153,167 @@ DECLARE_WRAPPER(int32_t,get_xid,())
 
 
 Mock_get_xid* Mock_get_xid::mock_=0;
 Mock_get_xid* Mock_get_xid::mock_=0;
 
 
+//******************************************************************************
+// activateWatcher mock
+
+DECLARE_WRAPPER(void,activateWatcher,(watcher_registration_t* reg, int rc))
+{
+    if(!Mock_activateWatcher::mock_){
+        CALL_REAL(activateWatcher,(reg,rc));
+    }else{
+        Mock_activateWatcher::mock_->call(reg,rc);
+    }
+}
+Mock_activateWatcher* Mock_activateWatcher::mock_=0;
+
+class ActivateWatcherWrapper: public Mock_activateWatcher{
+public:
+    ActivateWatcherWrapper():ctx_(0),activated_(false){}
+    
+    virtual void call(watcher_registration_t* reg, int rc){
+        CALL_REAL(activateWatcher,(reg,rc));
+        synchronized(mx_);
+        if(reg->context==ctx_){
+            activated_=true;
+            ctx_=0;
+        }
+    }
+    
+    void setContext(void* ctx){
+        synchronized(mx_);
+        ctx_=ctx;
+        activated_=false;
+    }
+    
+    SyncedBoolCondition isActivated() const{
+        return SyncedBoolCondition(activated_,mx_);
+    }
+    mutable Mutex mx_;
+    void* ctx_;
+    bool activated_;
+};
+
+WatcherActivationTracker::WatcherActivationTracker():
+    wrapper_(new ActivateWatcherWrapper)
+{    
+}
+
+WatcherActivationTracker::~WatcherActivationTracker(){
+    delete wrapper_;
+}
+
+void WatcherActivationTracker::track(void* ctx){
+    wrapper_->setContext(ctx);
+}
+
+SyncedBoolCondition WatcherActivationTracker::isWatcherActivated() const{
+    return wrapper_->isActivated();
+}
+
+//******************************************************************************
+//
+DECLARE_WRAPPER(void,deliverWatchers,(zhandle_t* zh,int type,int state, const char* path))
+{
+    if(!Mock_deliverWatchers::mock_){
+        CALL_REAL(deliverWatchers,(zh,type,state,path));
+    }else{
+        Mock_deliverWatchers::mock_->call(zh,type,state,path);
+    }
+}
+
+Mock_deliverWatchers* Mock_deliverWatchers::mock_=0;
+
+struct RefCounterValue{
+    RefCounterValue(zhandle_t* const& zh,int32_t expectedCounter,Mutex& mx):
+        zh_(zh),expectedCounter_(expectedCounter),mx_(mx){}
+    bool operator()() const{
+        {
+            synchronized(mx_);
+            if(zh_==0)
+                return false;
+        }
+        return inc_ref_counter(zh_,0)==expectedCounter_;
+    }
+    zhandle_t* const& zh_;
+    int32_t expectedCounter_;
+    Mutex& mx_;
+};
+
+
+class DeliverWatchersWrapper: public Mock_deliverWatchers{
+public:
+    DeliverWatchersWrapper(int type,int state,bool terminate):
+        type_(type),state_(state),
+        allDelivered_(false),terminate_(terminate),zh_(0),deliveryCounter_(0){}
+    virtual void call(zhandle_t* zh,int type,int state, const char* path){
+        {
+            synchronized(mx_);
+            zh_=zh;
+            allDelivered_=false;
+        }
+        CALL_REAL(deliverWatchers,(zh,type,state,path));
+        if(type_==type && state_==state){
+            if(terminate_){
+                // prevent zhandle_t from being prematurely distroyed;
+                // this will also ensure that zookeeper_close() cleanups the thread
+                // resources by calling finish_adaptor()
+                inc_ref_counter(zh,1);
+                terminateZookeeperThreads(zh);
+            }
+            synchronized(mx_);
+            allDelivered_=true;
+            deliveryCounter_++;
+        }
+    }
+    SyncedBoolCondition isDelivered() const{
+        if(terminate_){
+            int i=ensureCondition(RefCounterValue(zh_,1,mx_),1000);
+            assert(i<1000);
+        }
+        return SyncedBoolCondition(allDelivered_,mx_);
+    }
+    void resetDeliveryCounter(){
+        synchronized(mx_);
+        deliveryCounter_=0;
+    }
+    SyncedIntegerEqual deliveryCounterEquals(int expected) const{
+        if(terminate_){
+            int i=ensureCondition(RefCounterValue(zh_,1,mx_),1000);
+            assert(i<1000);
+        }
+        return SyncedIntegerEqual(deliveryCounter_,expected,mx_);
+    }
+    int type_;
+    int state_;
+    mutable Mutex mx_;
+    bool allDelivered_;
+    bool terminate_;
+    zhandle_t* zh_;
+    int deliveryCounter_;
+};
+
+WatcherDeliveryTracker::WatcherDeliveryTracker(
+        int type,int state,bool terminateCompletionThread):
+    deliveryWrapper_(new DeliverWatchersWrapper(
+            type,state,terminateCompletionThread)){
+}
+
+WatcherDeliveryTracker::~WatcherDeliveryTracker(){
+    delete deliveryWrapper_;
+}
+
+SyncedBoolCondition WatcherDeliveryTracker::isWatcherProcessingCompleted() const{
+    return deliveryWrapper_->isDelivered();
+}
+
+void WatcherDeliveryTracker::resetDeliveryCounter(){
+    deliveryWrapper_->resetDeliveryCounter();
+}
+
+SyncedIntegerEqual WatcherDeliveryTracker::deliveryCounterEquals(int expected) const{
+    return deliveryWrapper_->deliveryCounterEquals(expected);
+}
+
 //******************************************************************************
 //******************************************************************************
 //
 //
 string HandshakeResponse::toString() const {
 string HandshakeResponse::toString() const {
@@ -185,6 +355,45 @@ string ZooGetResponse::toString() const{
     return res;
     return res;
 }
 }
 
 
+string ZooStatResponse::toString() const{
+    oarchive* oa=create_buffer_oarchive();
+    
+    ReplyHeader h = {xid_,1,rc_};
+    serialize_ReplyHeader(oa, "hdr", &h);
+    
+    SetDataResponse resp;
+    resp.stat=stat_;
+    serialize_SetDataResponse(oa, "reply", &resp);
+    int32_t len=htonl(get_buffer_len(oa));
+    string res((char*)&len,sizeof(len));
+    res.append(get_buffer(oa),get_buffer_len(oa));
+    
+    close_buffer_oarchive(&oa,1);
+    return res;
+}
+
+string ZooGetChildrenResponse::toString() const{
+    oarchive* oa=create_buffer_oarchive();
+    
+    ReplyHeader h = {xid_,1,rc_};
+    serialize_ReplyHeader(oa, "hdr", &h);
+ 
+    GetChildrenResponse resp;
+    // populate the string vector
+    allocate_String_vector(&resp.children,strings_.size());
+    for(int i=0;i<(int)strings_.size();++i)
+        resp.children.data[i]=strdup(strings_[i].c_str());
+    serialize_GetChildrenResponse(oa, "reply", &resp);
+    deallocate_GetChildrenResponse(&resp);
+    
+    int32_t len=htonl(get_buffer_len(oa));
+    string res((char*)&len,sizeof(len));
+    res.append(get_buffer(oa),get_buffer_len(oa));
+    
+    close_buffer_oarchive(&oa,1);
+    return res;
+}
+
 string ZNodeEvent::toString() const{
 string ZNodeEvent::toString() const{
     oarchive* oa=create_buffer_oarchive();
     oarchive* oa=create_buffer_oarchive();
     struct WatcherEvent evt = {type_,0,(char*)path_.c_str()};
     struct WatcherEvent evt = {type_,0,(char*)path_.c_str()};
@@ -219,7 +428,7 @@ string PingResponse::toString() const{
 // Zookeeper server simulator
 // Zookeeper server simulator
 // 
 // 
 bool ZookeeperServer::hasMoreRecv() const{
 bool ZookeeperServer::hasMoreRecv() const{
-  return recvHasMore.get()!=0;
+  return recvHasMore.get()!=0  || connectionLost;
 }
 }
 
 
 ssize_t ZookeeperServer::callRecv(int s,void *buf,size_t len,int flags){
 ssize_t ZookeeperServer::callRecv(int s,void *buf,size_t len,int flags){
@@ -295,3 +504,8 @@ void forceConnected(zhandle_t* zh){
     gettimeofday(&zh->last_recv,0);    
     gettimeofday(&zh->last_recv,0);    
     gettimeofday(&zh->last_send,0);    
     gettimeofday(&zh->last_send,0);    
 }
 }
+
+void terminateZookeeperThreads(zhandle_t* zh){
+    // this will cause the zookeeper threads to terminate
+    zh->close_requested=1;
+}

+ 116 - 6
src/c/tests/ZKMocks.h

@@ -32,6 +32,11 @@
 // Async API tests!
 // Async API tests!
 void forceConnected(zhandle_t* zh); 
 void forceConnected(zhandle_t* zh); 
 
 
+/**
+ * Gracefully terminates zookeeper I/O and completion threads. 
+ */
+void terminateZookeeperThreads(zhandle_t* zh);
+
 // *****************************************************************************
 // *****************************************************************************
 // Abstract watcher action
 // Abstract watcher action
 struct SyncedBoolCondition;
 struct SyncedBoolCondition;
@@ -42,7 +47,11 @@ public:
     virtual ~WatcherAction(){}
     virtual ~WatcherAction(){}
     
     
     virtual void onSessionExpired(zhandle_t*){}
     virtual void onSessionExpired(zhandle_t*){}
+    virtual void onConnectionEstablished(zhandle_t*){}
+    virtual void onConnectionLost(zhandle_t*){}
     virtual void onNodeValueChanged(zhandle_t*,const char* path){}
     virtual void onNodeValueChanged(zhandle_t*,const char* path){}
+    virtual void onNodeDeleted(zhandle_t*,const char* path){}
+    virtual void onChildChanged(zhandle_t*,const char* path){}
     
     
     SyncedBoolCondition isWatcherTriggered() const;
     SyncedBoolCondition isWatcherTriggered() const;
     void setWatcherTriggered(){
     void setWatcherTriggered(){
@@ -57,7 +66,7 @@ protected:
 // zh->context is a pointer to a WatcherAction instance
 // zh->context is a pointer to a WatcherAction instance
 // based on the event type and state, the watcher calls a specific watcher 
 // based on the event type and state, the watcher calls a specific watcher 
 // action method
 // action method
-void activeWatcher(zhandle_t *zh, int type, int state, const char *path);
+void activeWatcher(zhandle_t *zh, int type, int state, const char *path,void* ctx);
 
 
 // *****************************************************************************
 // *****************************************************************************
 // a set of async completion signatures
 // a set of async completion signatures
@@ -104,6 +113,7 @@ struct IOThreadStopped{
     zhandle_t* zh_;
     zhandle_t* zh_;
 };
 };
 
 
+// a synchronized boolean condition
 struct SyncedBoolCondition{
 struct SyncedBoolCondition{
     SyncedBoolCondition(const bool& cond,Mutex& mx):cond_(cond),mx_(mx){}
     SyncedBoolCondition(const bool& cond,Mutex& mx):cond_(cond),mx_(mx){}
     bool operator()() const{
     bool operator()() const{
@@ -113,6 +123,20 @@ struct SyncedBoolCondition{
     const bool& cond_;
     const bool& cond_;
     Mutex& mx_;
     Mutex& mx_;
 };
 };
+
+// a synchronized integer comparison
+struct SyncedIntegerEqual{
+    SyncedIntegerEqual(const int& cond,int expected,Mutex& mx):
+        cond_(cond),expected_(expected),mx_(mx){}
+    bool operator()() const{
+        synchronized(mx_);
+        return cond_==expected_;
+    }
+    const int& cond_;
+    const int expected_;
+    Mutex& mx_;
+};
+
 // *****************************************************************************
 // *****************************************************************************
 // make sure to call zookeeper_close() even in presence of exceptions 
 // make sure to call zookeeper_close() even in presence of exceptions 
 struct CloseFinally{
 struct CloseFinally{
@@ -158,7 +182,6 @@ struct HandshakeRequest: public connect_req
 
 
 // *****************************************************************************
 // *****************************************************************************
 // flush_send_queue
 // flush_send_queue
-
 class Mock_flush_send_queue: public Mock
 class Mock_flush_send_queue: public Mock
 {
 {
 public:
 public:
@@ -177,7 +200,6 @@ public:
 
 
 // *****************************************************************************
 // *****************************************************************************
 // get_xid
 // get_xid
-
 class Mock_get_xid: public Mock
 class Mock_get_xid: public Mock
 {
 {
 public:
 public:
@@ -193,6 +215,57 @@ public:
     static Mock_get_xid* mock_;
     static Mock_get_xid* mock_;
 };
 };
 
 
+// *****************************************************************************
+// activateWatcher
+class Mock_activateWatcher: public Mock{
+public:
+    Mock_activateWatcher(){mock_=this;}
+    virtual ~Mock_activateWatcher(){mock_=0;}
+    
+    virtual void call(watcher_registration_t* reg, int rc){}
+    static Mock_activateWatcher* mock_;
+};
+
+class ActivateWatcherWrapper;
+class WatcherActivationTracker{
+public:
+    WatcherActivationTracker();
+    ~WatcherActivationTracker();
+    
+    void track(void* ctx);
+    SyncedBoolCondition isWatcherActivated() const;
+private:
+    ActivateWatcherWrapper* wrapper_;
+};
+
+// *****************************************************************************
+// deliverWatchers
+class Mock_deliverWatchers: public Mock{
+public:
+    Mock_deliverWatchers(){mock_=this;}
+    virtual ~Mock_deliverWatchers(){mock_=0;}
+    
+    virtual void call(zhandle_t* zh,int type,int state, const char* path){}
+    static Mock_deliverWatchers* mock_;
+};
+
+class DeliverWatchersWrapper;
+class WatcherDeliveryTracker{
+public:
+    // filters deliveries by state and type
+    WatcherDeliveryTracker(int type,int state,bool terminateCompletionThread=true);
+    ~WatcherDeliveryTracker();
+    
+    // if the thread termination requested (see the ctor params)
+    // this function will wait for the I/O and completion threads to 
+    // terminate before returning a SyncBoolCondition instance
+    SyncedBoolCondition isWatcherProcessingCompleted() const;
+    void resetDeliveryCounter();
+    SyncedIntegerEqual deliveryCounterEquals(int expected) const;
+private:
+    DeliverWatchersWrapper* deliveryWrapper_;
+};
+
 // *****************************************************************************
 // *****************************************************************************
 // a zookeeper Stat wrapper
 // a zookeeper Stat wrapper
 struct NodeStat: public Stat
 struct NodeStat: public Stat
@@ -220,6 +293,8 @@ public:
     virtual ~Response(){}
     virtual ~Response(){}
     
     
     virtual void setXID(int32_t xid){}
     virtual void setXID(int32_t xid){}
+    // this method is used by the ZookeeperServer class to serialize 
+    // the instance of Response
     virtual std::string toString() const =0;
     virtual std::string toString() const =0;
 };
 };
 
 
@@ -259,6 +334,41 @@ private:
     Stat stat_;
     Stat stat_;
 };
 };
 
 
+// zoo_exists(), zoo_set() response
+class ZooStatResponse: public Response
+{
+public:
+    ZooStatResponse(int32_t xid=0,int rc=ZOK,const Stat& stat=NodeStat())
+        :xid_(xid),rc_(rc),stat_(stat)
+    {
+    }
+    virtual std::string toString() const;
+    virtual void setXID(int32_t xid) {xid_=xid;}
+    
+private:
+    int32_t xid_;
+    int rc_;
+    Stat stat_;
+};
+
+// zoo_get_children()
+class ZooGetChildrenResponse: public Response
+{
+public:
+    typedef std::vector<std::string> StringVector;
+    ZooGetChildrenResponse(const StringVector& v,int rc=ZOK):
+        xid_(0),strings_(v),rc_(rc)
+    {
+    }
+    
+    virtual std::string toString() const;
+    virtual void setXID(int32_t xid) {xid_=xid;}
+
+    int32_t xid_;
+    StringVector strings_;
+    int rc_;
+};
+
 // PING response
 // PING response
 class PingResponse: public Response
 class PingResponse: public Response
 {
 {
@@ -326,12 +436,12 @@ public:
     // this is a trigger that gets reset back to false
     // this is a trigger that gets reset back to false
     // a connect request will return a non-matching session id thus causing 
     // a connect request will return a non-matching session id thus causing 
     // the client throw SESSION_EXPIRED
     // the client throw SESSION_EXPIRED
-    bool sessionExpired;
+    volatile bool sessionExpired;
     void returnSessionExpired(){ sessionExpired=true; }
     void returnSessionExpired(){ sessionExpired=true; }
     
     
-    // this is a trigger that gets reset back to false
+    // this is a one shot trigger that gets reset back to false
     // next recv call will return 0 length, thus simulating a connecton loss
     // next recv call will return 0 length, thus simulating a connecton loss
-    bool connectionLost;
+    volatile bool connectionLost;
     void setConnectionLost() {connectionLost=true;}
     void setConnectionLost() {connectionLost=true;}
     
     
     // recv
     // recv

+ 2 - 0
src/c/tests/wrappers.opt

@@ -2,3 +2,5 @@
 -Wl,--wrap -Wl,free
 -Wl,--wrap -Wl,free
 -Wl,--wrap -Wl,flush_send_queue
 -Wl,--wrap -Wl,flush_send_queue
 -Wl,--wrap -Wl,get_xid
 -Wl,--wrap -Wl,get_xid
+-Wl,--wrap -Wl,deliverWatchers
+-Wl,--wrap -Wl,activateWatcher