You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2009/10/31 00:09:28 UTC

svn commit: r831486 - in /hadoop/zookeeper/trunk: ./ src/ src/c/include/ src/c/src/ src/c/tests/ src/java/main/org/apache/zookeeper/ src/java/main/org/apache/zookeeper/server/ src/java/test/org/apache/zookeeper/test/

Author: mahadev
Date: Fri Oct 30 23:09:26 2009
New Revision: 831486

URL: http://svn.apache.org/viewvc?rev=831486&view=rev
Log:
ZOOKEEPER-555. Add stat information to GetChildrenResponse. (Arni Jonson and phunt via mahadev)

Added:
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GetChildren2Test.java
Modified:
    hadoop/zookeeper/trunk/CHANGES.txt
    hadoop/zookeeper/trunk/src/c/include/proto.h
    hadoop/zookeeper/trunk/src/c/include/zookeeper.h
    hadoop/zookeeper/trunk/src/c/src/cli.c
    hadoop/zookeeper/trunk/src/c/src/zk_adaptor.h
    hadoop/zookeeper/trunk/src/c/src/zk_log.c
    hadoop/zookeeper/trunk/src/c/src/zookeeper.c
    hadoop/zookeeper/trunk/src/c/tests/TestClient.cc
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/TraceFormatter.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
    hadoop/zookeeper/trunk/src/zookeeper.jute

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Oct 30 23:09:26 2009
@@ -121,6 +121,9 @@
 NEW FEATURES:
   ZOOKEEPER-539. generate eclipse project via ant target. (phunt via mahadev)
 
+  ZOOKEEPER-555. Add stat information to GetChildrenResponse. (Arni Jonson and
+  phunt via mahadev)
+
 Release 3.2.0 - 2009-06-30
 
 Non-backward compatible changes:

Modified: hadoop/zookeeper/trunk/src/c/include/proto.h
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/include/proto.h?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/include/proto.h (original)
+++ hadoop/zookeeper/trunk/src/c/include/proto.h Fri Oct 30 23:09:26 2009
@@ -33,6 +33,7 @@
 static const int GETCHILDREN_OP=8;
 static const int SYNC_OP=9;
 static const int PING_OP=11;
+static const int GETCHILDREN2_OP=12;
 static const int CLOSE_OP=-11;
 static const int SETAUTH_OP=100;
 static const int SETWATCHES_OP=101;

Modified: hadoop/zookeeper/trunk/src/c/include/zookeeper.h
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/include/zookeeper.h?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ hadoop/zookeeper/trunk/src/c/include/zookeeper.h Fri Oct 30 23:09:26 2009
@@ -19,6 +19,7 @@
 #ifndef ZOOKEEPER_H_
 #define ZOOKEEPER_H_
 
+#include <stdlib.h>
 #include <sys/time.h>
 #include <stdio.h>
 
@@ -211,7 +212,7 @@
  * \brief a change as occurred in the list of children.
  * 
  * This is only generated by watches on the child list of a node. These watches
- * are set using \ref zoo_get_children.
+ * are set using \ref zoo_get_children or \ref zoo_get_children2.
  */
 extern ZOOAPI const int ZOO_CHILD_EVENT;
 /**
@@ -490,6 +491,35 @@
         const struct String_vector *strings, const void *data);
 
 /**
+ * \brief signature of a completion function that returns a list of strings and stat.
+ * .
+ * 
+ * This method will be invoked at the end of a asynchronous call and also as 
+ * a result of connection loss or timeout.
+ * \param rc the error code of the call. Connection loss/timeout triggers 
+ * the completion with one of the following error codes:
+ * ZCONNECTIONLOSS -- lost connection to the server
+ * ZOPERATIONTIMEOUT -- connection timed out
+ * Data related events trigger the completion with error codes listed the 
+ * Exceptions section of the documentation of the function that initiated the
+ * call. (Zero indicates call was successful.)
+ * \param strings a pointer to the structure containng the list of strings of the
+ *   names of the children of a node. If a non zero error code is returned,
+ *   the content of strings is undefined. The programmer is NOT responsible
+ *   for freeing strings.
+ * \param stat a pointer to the stat information for the node involved in
+ *   this function. If a non zero error code is returned, the content of
+ *   stat is undefined. The programmer is NOT responsible for freeing stat.
+ * \param data the pointer that was passed by the caller when the function
+ *   that this completion corresponds to was invoked. The programmer
+ *   is responsible for any memory freeing associated with the data
+ *   pointer.
+ */
+typedef void (*strings_stat_completion_t)(int rc,
+        const struct String_vector *strings, const struct Stat *stat,
+        const void *data);
+
+/**
  * \brief signature of a completion function that returns a list of strings.
  * 
  * This method will be invoked at the end of a asynchronous call and also as 
@@ -798,6 +828,59 @@
         strings_completion_t completion, const void *data);
 
 /**
+ * \brief lists the children of a node, and get the parent stat.
+ * 
+ * \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 watch if nonzero, a watch will be set at the server to notify 
+ * the client if the node changes.
+ * \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 ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_aget_children2(zhandle_t *zh, const char *path, int watch, 
+        strings_stat_completion_t completion, const void *data);
+
+/**
+ * \brief lists the children of a node, and get the parent stat.
+ * 
+ * This function is similar to \ref zoo_aget_children2 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 ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_awget_children2(zhandle_t *zh, const char *path,
+        watcher_fn watcher, void* watcherCtx, 
+        strings_stat_completion_t completion, const void *data);
+
+/**
  * \brief Flush leader channel.
  *
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
@@ -1203,6 +1286,55 @@
         struct String_vector *strings);
 
 /**
+ * \brief lists the children of a node and get its stat synchronously.
+ * 
+ * \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 watch if nonzero, a watch will be set at the server to notify 
+ * the client if the node changes.
+ * \param strings return value of children paths.
+ * \param stat return value of node stat.
+ * \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 ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_get_children2(zhandle_t *zh, const char *path, int watch,
+                            struct String_vector *strings, struct Stat *stat);
+
+/**
+ * \brief lists the children of a node and get its stat 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.
+ * \param stat return value of node stat.
+ * \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 ZOO_SESSION_EXPIRED_STATE or ZOO_AUTH_FAILED_STATE
+ * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
+ */
+ZOOAPI int zoo_wget_children2(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx,
+        struct String_vector *strings, struct Stat *stat);
+
+/**
  * \brief gets the acl associated with a node synchronously.
  * 
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init

Modified: hadoop/zookeeper/trunk/src/c/src/cli.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/cli.c?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/cli.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/cli.c Fri Oct 30 23:09:26 2009
@@ -209,6 +209,14 @@
       shutdownThisThing=1;
 }
 
+void my_strings_stat_completion(int rc, const struct String_vector *strings,
+        const struct Stat *stat, const void *data) {
+    my_strings_completion(rc, strings, data);
+    dumpStat(stat);
+    if(batchMode)
+      shutdownThisThing=1;
+}
+
 void my_void_completion(int rc, const void *data) {
     fprintf(stderr, "%s: rc = %d\n", (char*)data, rc);
     free((void*)data);
@@ -273,6 +281,7 @@
       fprintf(stderr, "    set <path> <data>\n");
       fprintf(stderr, "    get <path>\n");
       fprintf(stderr, "    ls <path>\n");
+      fprintf(stderr, "    ls2 <path>\n");
       fprintf(stderr, "    sync <path>\n");
       fprintf(stderr, "    exists <path>\n");
       fprintf(stderr, "    myid\n");
@@ -339,6 +348,17 @@
         if (rc) {
             fprintf(stderr, "Error %d for %s\n", rc, line);
         }
+    } else if (startsWith(line, "ls2 ")) {
+        line += 4;
+        if (line[0] != '/') {
+            fprintf(stderr, "Path must start with /, found: %s\n", line);
+            return;
+        }
+        gettimeofday(&startTime, 0);
+        rc= zoo_aget_children2(zh, line, 1, my_strings_stat_completion, strdup(line));
+        if (rc) {
+            fprintf(stderr, "Error %d for %s\n", rc, line);
+        }
     } else if (startsWith(line, "create ")) {
         int flags = 0;
         line += 7;
@@ -453,7 +473,7 @@
 
     if (argc < 2) {
         fprintf(stderr,
-                "USAGE %s zookeeper_host_list [clientid_file|cmd:(ls|create|od|...)]\n", 
+                "USAGE %s zookeeper_host_list [clientid_file|cmd:(ls|ls2|create|od|...)]\n", 
                 argv[0]);
         fprintf(stderr,
                 "Version: ZooKeeper cli (c client) version %d.%d.%d\n", 

Modified: hadoop/zookeeper/trunk/src/c/src/zk_adaptor.h
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zk_adaptor.h?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/zk_adaptor.h (original)
+++ hadoop/zookeeper/trunk/src/c/src/zk_adaptor.h Fri Oct 30 23:09:26 2009
@@ -92,7 +92,11 @@
             struct ACL_vector acl;
             struct Stat stat;
         } acl;
-        struct String_vector strs;
+        struct String_vector strs2;
+        struct {
+            struct String_vector strs2;
+            struct Stat stat2;
+        } strs_stat;
     } u;
     int complete;
 #ifdef THREADED

Modified: hadoop/zookeeper/trunk/src/c/src/zk_log.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zk_log.c?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/zk_log.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/zk_log.c Fri Oct 30 23:09:26 2009
@@ -128,8 +128,8 @@
     fprintf(LOGSTREAM, "%s:%d:%s@%s@%d: %s\n", time_now(),pid,
             dbgLevelStr[curLevel],funcName,line,message);
 #else
-    fprintf(LOGSTREAM, "%s:%d(0x%x):%s@%s@%d: %s\n", time_now(),pid,
-            (unsigned long)pthread_self(),
+    fprintf(LOGSTREAM, "%s:%d(0x%lx):%s@%s@%d: %s\n", time_now(),pid,
+            (unsigned long int)pthread_self(),
             dbgLevelStr[curLevel],funcName,line,message);
 #endif
     fflush(LOGSTREAM);

Modified: hadoop/zookeeper/trunk/src/c/src/zookeeper.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zookeeper.c?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/zookeeper.c Fri Oct 30 23:09:26 2009
@@ -136,8 +136,9 @@
 #define COMPLETION_STAT 1
 #define COMPLETION_DATA 2
 #define COMPLETION_STRINGLIST 3
-#define COMPLETION_ACLLIST 4
-#define COMPLETION_STRING 5
+#define COMPLETION_STRINGLIST_STAT 4
+#define COMPLETION_ACLLIST 5
+#define COMPLETION_STRING 6
 
 typedef struct _auth_completion_list {
     void_completion_t completion;
@@ -153,6 +154,7 @@
         stat_completion_t stat_result;
         data_completion_t data_result;
         strings_completion_t strings_result;
+        strings_stat_completion_t strings_stat_result;
         acl_completion_t acl_result;
         string_completion_t string_result;
         struct watcher_object_list *watcher_result;
@@ -1670,6 +1672,17 @@
                     deallocate_GetChildrenResponse(&res);
                 }
                 break;
+            case COMPLETION_STRINGLIST_STAT:
+                LOG_DEBUG(("Calling COMPLETION_STRINGLIST_STAT for xid=%#x rc=%d",cptr->xid,rc));
+                if (rc) {
+                    cptr->c.strings_stat_result(rc, 0, 0, cptr->data);
+                } else {
+                    struct GetChildren2Response res;
+                    deserialize_GetChildren2Response(ia, "reply", &res);
+                    cptr->c.strings_stat_result(rc, &res.children, &res.stat, cptr->data);
+                    deallocate_GetChildren2Response(&res);
+                }
+                break;
             case COMPLETION_STRING:
                 LOG_DEBUG(("Calling COMPLETION_STRING for xid=%#x rc=%d",cptr->xid,rc));
                 if (rc) {
@@ -1882,11 +1895,23 @@
                     if (rc == 0) {
                         struct GetChildrenResponse res;
                         deserialize_GetChildrenResponse(ia, "reply", &res);
-                        sc->u.strs = res.children;
+                        sc->u.strs2 = res.children;
                         /* We don't deallocate since we are passing it back */
                         // deallocate_GetChildrenResponse(&res);
                     }
                     break;
+                case COMPLETION_STRINGLIST_STAT:
+                    LOG_DEBUG(("Calling COMPLETION_STRINGLIST_STAT for xid=%#x rc=%d",
+                               cptr->xid, rc));
+                    if (rc == 0) {
+                        struct GetChildren2Response res;
+                        deserialize_GetChildren2Response(ia, "reply", &res);
+                        sc->u.strs_stat.strs2 = res.children;
+                        sc->u.strs_stat.stat2 = res.stat;
+                        /* We don't deallocate since we are passing it back */
+                        // deallocate_GetChildren2Response(&res);
+                    }
+                    break;
                 case COMPLETION_STRING:
                     LOG_DEBUG(("Calling COMPLETION_STRING for xid=%#x rc=%d",
                                cptr->xid, rc));
@@ -1994,6 +2019,9 @@
     case COMPLETION_STRINGLIST:
         c->c.strings_result = (strings_completion_t)dc;
         break;
+    case COMPLETION_STRINGLIST_STAT:
+        c->c.strings_stat_result = (strings_stat_completion_t)dc;
+        break;
     case COMPLETION_ACLLIST:
         c->c.acl_result = (acl_completion_t)dc;
         break;
@@ -2071,6 +2099,12 @@
     return add_completion(zh, xid, COMPLETION_STRINGLIST, dc, data, 0,wo);
 }
 
+static int add_strings_stat_completion(zhandle_t *zh, int xid,
+        strings_stat_completion_t dc, const void *data,watcher_registration_t* wo)
+{
+    return add_completion(zh, xid, COMPLETION_STRINGLIST_STAT, dc, data, 0,wo);
+}
+
 static int add_acl_completion(zhandle_t *zh, int xid, acl_completion_t dc,
         const void *data)
 {
@@ -2390,20 +2424,69 @@
     return (rc < 0)?ZMARSHALLINGERROR:ZOK;
 }
 
+static int zoo_awget_children_(zhandle_t *zh, const char *path,
+         watcher_fn watcher, void* watcherCtx, 
+         strings_completion_t sc,
+         const void *data)
+{
+    struct oarchive *oa;
+    struct RequestHeader h = { .xid = get_xid(), .type = GETCHILDREN_OP};
+    char * server_path = prepend_string(zh, path);
+    struct GetChildrenRequest req = {(char*)server_path, watcher!=0 }; 
+    int rc;
+        
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
+        return ZBADARGUMENTS;
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
+        return ZINVALIDSTATE;
+    }
+    oa = create_buffer_oarchive();
+    rc = serialize_RequestHeader(oa, "header", &h);
+    rc = rc < 0 ? rc : serialize_GetChildrenRequest(oa, "req", &req);
+    enter_critical(zh);
+    rc = rc < 0 ? rc : add_strings_completion(zh, h.xid, sc, data,
+            create_watcher_registration(server_path,child_result_checker,watcher,watcherCtx));
+    rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
+            get_buffer_len(oa));
+    leave_critical(zh);
+    free_duplicate_path(server_path, path);
+    /* We queued the buffer, so don't free it */
+    close_buffer_oarchive(&oa, 0);
+
+    LOG_DEBUG(("Sending request xid=%#x for path [%s] to %s",h.xid,path,
+            format_current_endpoint_info(zh)));
+    /* make a best (non-blocking) effort to send the requests asap */
+    adaptor_send_queue(zh, 0);
+    return (rc < 0)?ZMARSHALLINGERROR:ZOK;
+}
+
 int zoo_aget_children(zhandle_t *zh, const char *path, int watch,
         strings_completion_t dc, const void *data)
 {
-    return zoo_awget_children(zh,path,watch?zh->watcher:0,zh->context,dc,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)
+         watcher_fn watcher, void* watcherCtx, 
+         strings_completion_t dc,
+         const void *data)
+{
+    return zoo_awget_children_(zh,path,watcher,watcherCtx,dc,data);
+}
+
+static int zoo_awget_children2_(zhandle_t *zh, const char *path,
+         watcher_fn watcher, void* watcherCtx, 
+         strings_stat_completion_t ssc,
+         const void *data)
 {
+    /* invariant: (sc == NULL) != (sc == NULL) */
     struct oarchive *oa;
-    struct RequestHeader h = { .xid = get_xid(), .type = GETCHILDREN_OP};
+    struct RequestHeader h = { .xid = get_xid(), .type = GETCHILDREN2_OP};
     char * server_path = prepend_string(zh, path);
-    struct GetChildrenRequest req = {(char*)server_path, watcher!=0 }; 
+    struct GetChildren2Request req2 = {(char*)server_path, watcher!=0 }; 
     int rc;
         
     if (zh==0 || !isValidPath(server_path, 0)) {
@@ -2416,9 +2499,9 @@
     }
     oa = create_buffer_oarchive();
     rc = serialize_RequestHeader(oa, "header", &h);
-    rc = rc < 0 ? rc : serialize_GetChildrenRequest(oa, "req", &req);
+    rc = rc < 0 ? rc : serialize_GetChildren2Request(oa, "req", &req2);
     enter_critical(zh);
-    rc = rc < 0 ? rc : add_strings_completion(zh, h.xid, dc, data,
+    rc = rc < 0 ? rc : add_strings_stat_completion(zh, h.xid, ssc, data,
             create_watcher_registration(server_path,child_result_checker,watcher,watcherCtx));
     rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
             get_buffer_len(oa));
@@ -2434,6 +2517,20 @@
     return (rc < 0)?ZMARSHALLINGERROR:ZOK;
 }
 
+int zoo_aget_children2(zhandle_t *zh, const char *path, int watch,
+        strings_stat_completion_t dc, const void *data)
+{
+    return zoo_awget_children2_(zh,path,watch?zh->watcher:0,zh->context,dc,data);
+}
+
+int zoo_awget_children2(zhandle_t *zh, const char *path,
+         watcher_fn watcher, void* watcherCtx, 
+         strings_stat_completion_t dc,
+         const void *data)
+{
+    return zoo_awget_children2_(zh,path,watcher,watcherCtx,dc,data);
+}
+
 int zoo_async(zhandle_t *zh, const char *path,
         string_completion_t completion, const void *data)
 {
@@ -2862,30 +2959,51 @@
     return rc;
 }
 
-int zoo_get_children(zhandle_t *zh, const char *path, int watch,
+static int zoo_wget_children_(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx,
         struct String_vector *strings)
 {
-    return zoo_wget_children(zh,path,watch?zh->watcher:0,zh->context,strings);
+    struct sync_completion *sc = alloc_sync_completion();
+    int rc;
+    if (!sc) {
+        return ZSYSTEMERROR;
+    }
+    rc= zoo_awget_children (zh, path, watcher, watcherCtx, SYNCHRONOUS_MARKER, sc);
+    if(rc==ZOK){
+        wait_sync_completion(sc);
+        rc = sc->rc;
+        if (rc == 0) {
+            if (strings) {
+                *strings = sc->u.strs2;
+            } else {
+                deallocate_String_vector(&sc->u.strs2);
+            }
+        }
+    }
+    free_sync_completion(sc);
+    return rc;
 }
 
-int zoo_wget_children(zhandle_t *zh, const char *path, 
+static int zoo_wget_children2_(zhandle_t *zh, const char *path, 
         watcher_fn watcher, void* watcherCtx,
-        struct String_vector *strings)
+        struct String_vector *strings, struct Stat *stat)
 {
     struct sync_completion *sc = alloc_sync_completion();
     int rc;
     if (!sc) {
         return ZSYSTEMERROR;
     }
-    rc=zoo_awget_children(zh, path, watcher, watcherCtx, SYNCHRONOUS_MARKER, sc);
+    rc= zoo_awget_children2(zh, path, watcher, watcherCtx, SYNCHRONOUS_MARKER, sc);
+
     if(rc==ZOK){
         wait_sync_completion(sc);
         rc = sc->rc;
         if (rc == 0) {
+            *stat = sc->u.strs_stat.stat2;
             if (strings) {
-                *strings = sc->u.strs;
+                *strings = sc->u.strs_stat.strs2;
             } else {
-                deallocate_String_vector(&sc->u.strs);
+                deallocate_String_vector(&sc->u.strs_stat.strs2);
             }
         }
     }
@@ -2893,6 +3011,32 @@
     return rc;
 }
 
+int zoo_get_children(zhandle_t *zh, const char *path, int watch,
+        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)
+{
+    return zoo_wget_children_(zh,path,watcher,watcherCtx,strings);
+}
+
+int zoo_get_children2(zhandle_t *zh, const char *path, int watch,
+        struct String_vector *strings, struct Stat *stat)
+{
+    return zoo_wget_children2_(zh,path,watch?zh->watcher:0,zh->context,strings,stat);
+}
+
+int zoo_wget_children2(zhandle_t *zh, const char *path, 
+        watcher_fn watcher, void* watcherCtx,
+        struct String_vector *strings, struct Stat *stat)
+{
+    return zoo_wget_children2_(zh,path,watcher,watcherCtx,strings,stat);
+}
+
 int zoo_get_acl(zhandle_t *zh, const char *path, struct ACL_vector *acl,
         struct Stat *stat)
 {

Modified: hadoop/zookeeper/trunk/src/c/tests/TestClient.cc
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/tests/TestClient.cc?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/tests/TestClient.cc (original)
+++ hadoop/zookeeper/trunk/src/c/tests/TestClient.cc Fri Oct 30 23:09:26 2009
@@ -39,6 +39,21 @@
 
 #include "Util.h"
 
+static int Stat_eq(struct Stat* a, struct Stat* b)
+{
+    if (a->czxid != b->czxid) return 0;
+    if (a->mzxid != b->mzxid) return 0;
+    if (a->ctime != b->ctime) return 0;
+    if (a->mtime != b->mtime) return 0;
+    if (a->version != b->version) return 0;
+    if (a->cversion != b->cversion) return 0;
+    if (a->aversion != b->aversion) return 0;
+    if (a->ephemeralOwner != b->ephemeralOwner) return 0;
+    if (a->dataLength != b->dataLength) return 0;
+    if (a->numChildren != b->numChildren) return 0;
+    if (a->pzxid != b->pzxid) return 0;
+    return 1;
+}
 #ifdef THREADED
     static void yield(zhandle_t *zh, int i)
     {
@@ -173,6 +188,7 @@
     CPPUNIT_TEST(testAuth);
     CPPUNIT_TEST(testWatcherAutoResetWithGlobal);
     CPPUNIT_TEST(testWatcherAutoResetWithLocal);
+    CPPUNIT_TEST(testGetChildren2);
 #endif
     CPPUNIT_TEST_SUITE_END();
 
@@ -521,7 +537,40 @@
         rc = zoo_set_acl(zk, "/", -1, &ZOO_OPEN_ACL_UNSAFE);
         CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
     }
-    
+
+    void testGetChildren2() {
+        int rc;
+        watchctx_t ctx;
+        zhandle_t *zk = createClient(&ctx);
+
+        rc = zoo_create(zk, "/parent", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        rc = zoo_create(zk, "/parent/child_a", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        rc = zoo_create(zk, "/parent/child_b", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        rc = zoo_create(zk, "/parent/child_c", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        rc = zoo_create(zk, "/parent/child_d", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        struct String_vector strings;
+        struct Stat stat_a, stat_b;
+
+        rc = zoo_get_children2(zk, "/parent", 0, &strings, &stat_a);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        rc = zoo_exists(zk, "/parent", 0, &stat_b);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+
+        CPPUNIT_ASSERT(Stat_eq(&stat_a, &stat_b));
+        CPPUNIT_ASSERT(stat_a.numChildren == 4);
+    }
+
     void testNullData() {
         watchctx_t ctx;
         zhandle_t *zk = createClient(&ctx);
@@ -717,7 +766,7 @@
         waitForCreateCompletion(3);
         CPPUNIT_ASSERT(count == 0);
     }
-        
+
     void testAsyncWatcherAutoReset()
     {
         watchctx_t ctx;

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java Fri Oct 30 23:09:26 2009
@@ -42,6 +42,11 @@
                 List<String> children);
     }
 
+    interface Children2Callback extends AsyncCallback {
+        public void processResult(int rc, String path, Object ctx,
+                List<String> children, Stat stat);
+    }
+
     interface StringCallback extends AsyncCallback {
         public void processResult(int rc, String path, Object ctx, String name);
     }

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Fri Oct 30 23:09:26 2009
@@ -40,6 +40,7 @@
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.AsyncCallback.ACLCallback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.AsyncCallback.StatCallback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
@@ -58,6 +59,7 @@
 import org.apache.zookeeper.proto.ExistsResponse;
 import org.apache.zookeeper.proto.GetACLResponse;
 import org.apache.zookeeper.proto.GetChildrenResponse;
+import org.apache.zookeeper.proto.GetChildren2Response;
 import org.apache.zookeeper.proto.GetDataResponse;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.RequestHeader;
@@ -486,6 +488,15 @@
                                 } else {
                                     cb.processResult(rc, clientPath, p.ctx, null);
                                 }
+                            } else if (p.response instanceof GetChildren2Response) {
+                                Children2Callback cb = (Children2Callback) p.cb;
+                                GetChildren2Response rsp = (GetChildren2Response) p.response;
+                                if (rc == 0) {
+                                    cb.processResult(rc, clientPath, p.ctx, rsp
+                                            .getChildren(), rsp.getStat());
+                                } else {
+                                    cb.processResult(rc, clientPath, p.ctx, null, null);
+                                }
                             } else if (p.response instanceof CreateResponse) {
                                 StringCallback cb = (StringCallback) p.cb;
                                 CreateResponse rsp = (CreateResponse) p.response;

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java Fri Oct 30 23:09:26 2009
@@ -48,6 +48,8 @@
 
         public final int ping = 11;
 
+        public final int getChildren2 = 12;
+
         public final int auth = 100;
         
         public final int setWatches = 101;
@@ -108,5 +110,5 @@
 
     final public static String[] opNames = { "notification", "create",
             "delete", "exists", "getData", "setData", "getACL", "setACL",
-            "getChildren", "getMaxChildren", "setMaxChildren", "ping" };
+            "getChildren", "getChildren2", "getMaxChildren", "setMaxChildren", "ping" };
 }

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Fri Oct 30 23:09:26 2009
@@ -28,6 +28,7 @@
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.AsyncCallback.ACLCallback;
+import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.AsyncCallback.StatCallback;
@@ -42,6 +43,8 @@
 import org.apache.zookeeper.proto.ExistsRequest;
 import org.apache.zookeeper.proto.GetACLRequest;
 import org.apache.zookeeper.proto.GetACLResponse;
+import org.apache.zookeeper.proto.GetChildren2Request;
+import org.apache.zookeeper.proto.GetChildren2Response;
 import org.apache.zookeeper.proto.GetChildrenRequest;
 import org.apache.zookeeper.proto.GetChildrenResponse;
 import org.apache.zookeeper.proto.GetDataRequest;
@@ -342,7 +345,7 @@
      * Added in 3.2.0: An optional "chroot" suffix may also be appended to the
      * connection string. This will run the client commands while interpreting
      * all paths relative to this root (similar to the unix chroot command).
-     * 
+     *
      * @param connectString
      *            comma separated host:port pairs, each corresponding to a zk
      *            server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If
@@ -357,7 +360,7 @@
      * @param watcher
      *            a watcher object which will be notified of state changes, may
      *            also be notified for node events
-     * 
+     *
      * @throws IOException
      *             in cases of network failure
      * @throws IllegalArgumentException
@@ -448,7 +451,7 @@
      * re-connect.
      *
      * This method is NOT thread safe
-     * 
+     *
      * @return current session id
      */
     public long getSessionId() {
@@ -461,7 +464,7 @@
      * change after a re-connect.
      *
      * This method is NOT thread safe
-     * 
+     *
      * @return current session password
      */
     public byte[] getSessionPasswd() {
@@ -470,9 +473,9 @@
 
     /**
      * Add the specified scheme:auth information to this connection.
-     * 
+     *
      * This method is NOT thread safe
-     * 
+     *
      * @param scheme
      * @param auth
      */
@@ -483,7 +486,7 @@
     /**
      * Specify the default watcher for the connection (overrides the one
      * specified during construction).
-     * 
+     *
      * @param watcher
      */
     public synchronized void register(Watcher watcher) {
@@ -528,7 +531,7 @@
             return clientPath;
         }
     }
-    
+
     /**
      * Create a node with the given path. The node data will be the given data,
      * and node acl will be the given acl.
@@ -675,7 +678,7 @@
     {
         final String clientPath = path;
         PathUtils.validatePath(clientPath);
-        
+
         final String serverPath;
 
         // maintain semantics even in chroot case
@@ -1279,7 +1282,132 @@
      * @see #getChildren(String, boolean)
      */
     public void getChildren(String path, boolean watch, ChildrenCallback cb,
-            Object ctx) {
+            Object ctx)
+    {
+        getChildren(path, watch ? watchManager.defaultWatcher : null, cb, ctx);
+    }
+
+    /**
+     * For the given znode path return the stat and children list.
+     * <p>
+     * If the watch is non-null and the call is successful (no exception is thrown),
+     * a watch will be left on the node with the given path. The watch willbe
+     * triggered by a successful operation that deletes the node of the given
+     * path or creates/delete a child under the node.
+     * <p>
+     * The list of children returned is not sorted and no guarantee is provided
+     * as to its natural or lexical order.
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if no node with the given path exists.
+     *
+     * @param path
+     * @param watcher explicit watcher
+     * @param stat stat of the znode designated by path
+     * @return an unordered array of children of the node with the given path
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero error code.
+     * @throws IllegalArgumentException if an invalid path is specified
+     */
+    public List<String> getChildren(final String path, Watcher watcher,
+            Stat stat)
+        throws KeeperException, InterruptedException
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        // the watch contains the un-chroot path
+        WatchRegistration wcb = null;
+        if (watcher != null) {
+            wcb = new ChildWatchRegistration(watcher, clientPath);
+        }
+
+        final String serverPath = prependChroot(clientPath);
+
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getChildren2);
+        GetChildren2Request request = new GetChildren2Request();
+        request.setPath(serverPath);
+        request.setWatch(watcher != null);
+        GetChildren2Response response = new GetChildren2Response();
+        ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()),
+                    clientPath);
+        }
+        if (stat != null) {
+            DataTree.copyStat(response.getStat(), stat);
+        }
+        return response.getChildren();
+    }
+
+    /**
+     * For the given znode path return the stat and children list.
+     * <p>
+     * If the watch is true and the call is successful (no exception is thrown),
+     * a watch will be left on the node with the given path. The watch willbe
+     * triggered by a successful operation that deletes the node of the given
+     * path or creates/delete a child under the node.
+     * <p>
+     * The list of children returned is not sorted and no guarantee is provided
+     * as to its natural or lexical order.
+     * <p>
+     * A KeeperException with error code KeeperException.NoNode will be thrown
+     * if no node with the given path exists.
+     *
+     * @param path
+     * @param watch
+     * @param stat stat of the znode designated by path
+     * @return an unordered array of children of the node with the given path
+     * @throws InterruptedException If the server transaction is interrupted.
+     * @throws KeeperException If the server signals an error with a non-zero
+     *  error code.
+     */
+    public List<String> getChildren(String path, boolean watch, Stat stat)
+            throws KeeperException, InterruptedException {
+        return getChildren(path, watch ? watchManager.defaultWatcher : null,
+                stat);
+    }
+
+    /**
+     * The Asynchronous version of getChildren. The request doesn't actually
+     * until the asynchronous callback is called.
+     *
+     * @see #getChildren(String, Watcher)
+     */
+    public void getChildren(final String path, Watcher watcher,
+            Children2Callback cb, Object ctx)
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        // the watch contains the un-chroot path
+        WatchRegistration wcb = null;
+        if (watcher != null) {
+            wcb = new ChildWatchRegistration(watcher, clientPath);
+        }
+
+        final String serverPath = prependChroot(clientPath);
+
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.getChildren2);
+        GetChildren2Request request = new GetChildren2Request();
+        request.setPath(serverPath);
+        request.setWatch(watcher != null);
+        GetChildren2Response response = new GetChildren2Response();
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, wcb);
+    }
+
+    /**
+     * The Asynchronous version of getChildren. The request doesn't actually
+     * until the asynchronous callback is called.
+     *
+     * @see #getChildren(String, boolean)
+     */
+    public void getChildren(String path, boolean watch, Children2Callback cb,
+            Object ctx)
+    {
         getChildren(path, watch ? watchManager.defaultWatcher : null, cb, ctx);
     }
 

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java Fri Oct 30 23:09:26 2009
@@ -68,6 +68,7 @@
         commandMap.put("set","path data [version]");
         commandMap.put("get","path [watch]");
         commandMap.put("ls","path [watch]");
+        commandMap.put("ls2","path [watch]");
         commandMap.put("getAcl","path");
         commandMap.put("setAcl","path acl");
         commandMap.put("stat","path [watch]");
@@ -697,6 +698,11 @@
             path = args[1];
             List<String> children = zk.getChildren(path, watch);
             System.out.println(children);
+        } else if (cmd.equals("ls2") && args.length >= 2) {
+            path = args[1];
+            List<String> children = zk.getChildren(path, watch, stat);
+            System.out.println(children);
+            printStat(stat);
         } else if (cmd.equals("getAcl") && args.length >= 2) {
             path = args[1];
             acl = zk.getACL(path, stat);

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java Fri Oct 30 23:09:26 2009
@@ -626,6 +626,9 @@
         }
         synchronized (n) {
             ArrayList<String> children = new ArrayList<String>();
+            if (stat != null) {
+                n.copyStat(stat);
+            }
             children.addAll(n.children);
             if (watcher != null) {
                 childWatches.addWatch(path, watcher);

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java Fri Oct 30 23:09:26 2009
@@ -37,6 +37,8 @@
 import org.apache.zookeeper.proto.GetACLResponse;
 import org.apache.zookeeper.proto.GetChildrenRequest;
 import org.apache.zookeeper.proto.GetChildrenResponse;
+import org.apache.zookeeper.proto.GetChildren2Request;
+import org.apache.zookeeper.proto.GetChildren2Response;
 import org.apache.zookeeper.proto.GetDataRequest;
 import org.apache.zookeeper.proto.GetDataResponse;
 import org.apache.zookeeper.proto.ReplyHeader;
@@ -147,38 +149,46 @@
                 LOG.debug(request);
             }
             switch (request.type) {
-            case OpCode.ping:
+            case OpCode.ping: {
                 request.cnxn.sendResponse(new ReplyHeader(-2,
                         zks.dataTree.lastProcessedZxid, 0), null, "response");
                 return;
-            case OpCode.createSession:
+            }
+            case OpCode.createSession: {
                 request.cnxn.finishSessionInit(true);
                 return;
-            case OpCode.create:
+            }
+            case OpCode.create: {
                 rsp = new CreateResponse(rc.path);
                 err = Code.get(rc.err);
                 break;
-            case OpCode.delete:
+            }
+            case OpCode.delete: {
                 err = Code.get(rc.err);
                 break;
-            case OpCode.setData:
+            }
+            case OpCode.setData: {
                 rsp = new SetDataResponse(rc.stat);
                 err = Code.get(rc.err);
                 break;
-            case OpCode.setACL:
+            }
+            case OpCode.setACL: {
                 rsp = new SetACLResponse(rc.stat);
                 err = Code.get(rc.err);
                 break;
-            case OpCode.closeSession:
+            }
+            case OpCode.closeSession: {
                 err = Code.get(rc.err);
                 break;
-            case OpCode.sync:
+            }
+            case OpCode.sync: {
                 SyncRequest syncRequest = new SyncRequest();
                 ZooKeeperServer.byteBuffer2Record(request.request,
                         syncRequest);
                 rsp = new SyncResponse(syncRequest.getPath());
                 break;
-            case OpCode.exists:
+            }
+            case OpCode.exists: {
                 // TODO we need to figure out the security requirement for this!
                 ExistsRequest existsRequest = new ExistsRequest();
                 ZooKeeperServer.byteBuffer2Record(request.request,
@@ -191,7 +201,8 @@
                         .getWatch() ? request.cnxn : null);
                 rsp = new ExistsResponse(stat);
                 break;
-            case OpCode.getData:
+            }
+            case OpCode.getData: {
                 GetDataRequest getDataRequest = new GetDataRequest();
                 ZooKeeperServer.byteBuffer2Record(request.request,
                         getDataRequest);
@@ -206,12 +217,13 @@
                 PrepRequestProcessor.checkACL(zks, zks.dataTree.convertLong(aclL),
                         ZooDefs.Perms.READ,
                         request.authInfo);
-                stat = new Stat();
+                Stat stat = new Stat();
                 byte b[] = zks.dataTree.getData(getDataRequest.getPath(), stat,
                         getDataRequest.getWatch() ? request.cnxn : null);
                 rsp = new GetDataResponse(b, stat);
                 break;
-            case OpCode.setWatches:
+            }
+            case OpCode.setWatches: {
                 SetWatches setWatches = new SetWatches();
                 // XXX We really should NOT need this!!!!
                 request.request.rewind();
@@ -222,21 +234,22 @@
                         setWatches.getExistWatches(),
                         setWatches.getChildWatches(), request.cnxn);
                 break;
-            case OpCode.getACL:
+            }
+            case OpCode.getACL: {
                 GetACLRequest getACLRequest = new GetACLRequest();
                 ZooKeeperServer.byteBuffer2Record(request.request,
                         getACLRequest);
-                stat = new Stat();
+                Stat stat = new Stat();
                 List<ACL> acl = 
                     zks.dataTree.getACL(getACLRequest.getPath(), stat);
                 rsp = new GetACLResponse(acl, stat);
                 break;
-            case OpCode.getChildren:
+            }
+            case OpCode.getChildren: {
                 GetChildrenRequest getChildrenRequest = new GetChildrenRequest();
                 ZooKeeperServer.byteBuffer2Record(request.request,
                         getChildrenRequest);
-                stat = new Stat();
-                n = zks.dataTree.getNode(getChildrenRequest.getPath());
+                DataNode n = zks.dataTree.getNode(getChildrenRequest.getPath());
                 if (n == null) {
                     throw new KeeperException.NoNodeException();
                 }
@@ -248,11 +261,34 @@
                         ZooDefs.Perms.READ,
                         request.authInfo);
                 List<String> children = zks.dataTree.getChildren(
-                        getChildrenRequest.getPath(), stat, getChildrenRequest
+                        getChildrenRequest.getPath(), null, getChildrenRequest
                                 .getWatch() ? request.cnxn : null);
                 rsp = new GetChildrenResponse(children);
                 break;
             }
+            case OpCode.getChildren2: {
+                GetChildren2Request getChildren2Request = new GetChildren2Request();
+                ZooKeeperServer.byteBuffer2Record(request.request,
+                        getChildren2Request);
+                Stat stat = new Stat();
+                DataNode n = zks.dataTree.getNode(getChildren2Request.getPath());
+                if (n == null) {
+                    throw new KeeperException.NoNodeException();
+                }
+                Long aclG;
+                synchronized(n) {
+                    aclG = n.acl;
+                }
+                PrepRequestProcessor.checkACL(zks, zks.dataTree.convertLong(aclG), 
+                        ZooDefs.Perms.READ,
+                        request.authInfo);
+                List<String> children = zks.dataTree.getChildren(
+                        getChildren2Request.getPath(), stat, getChildren2Request
+                                .getWatch() ? request.cnxn : null);
+                rsp = new GetChildren2Response(children, stat);
+                break;
+            }
+            }
         } catch (KeeperException e) {
             err = e.code();
         } catch (Exception e) {

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java Fri Oct 30 23:09:26 2009
@@ -389,6 +389,7 @@
             case OpCode.getData:
             case OpCode.getACL:
             case OpCode.getChildren:
+            case OpCode.getChildren2:
             case OpCode.ping:
             case OpCode.setWatches:
             	zks.sessionTracker.checkSession(request.sessionId, request.getOwner());

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java Fri Oct 30 23:09:26 2009
@@ -110,6 +110,7 @@
         case OpCode.getACL:
         case OpCode.setACL:
         case OpCode.getChildren:
+        case OpCode.getChildren2:
         case OpCode.ping:
         case OpCode.closeSession:
         case OpCode.setWatches:
@@ -124,6 +125,7 @@
         case OpCode.exists:
         case OpCode.getACL:
         case OpCode.getChildren:
+        case OpCode.getChildren2:
         case OpCode.getData:
             return false;
         case OpCode.error:
@@ -163,6 +165,8 @@
             return "setACL";
         case OpCode.getChildren:
             return "getChildren";
+        case OpCode.getChildren2:
+            return "getChildren2";
         case OpCode.ping:
             return "ping";
         case OpCode.createSession:

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/TraceFormatter.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/TraceFormatter.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/TraceFormatter.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/TraceFormatter.java Fri Oct 30 23:09:26 2009
@@ -49,6 +49,8 @@
             return "setACL";
         case OpCode.getChildren:
             return "getChildren";
+        case OpCode.getChildren2:
+            return "getChildren2";
         case OpCode.ping:
             return "ping";
         case OpCode.createSession:

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java Fri Oct 30 23:09:26 2009
@@ -32,6 +32,7 @@
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.ACLCallback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.AsyncCallback.StatCallback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
@@ -278,11 +279,95 @@
         
         public void verifyGetChildrenEmpty() {
             StringCB parent = createNode();
+            path = parent.path;
+            verify();
+        }
+
+        public void verifyGetChildrenSingle() {
+            StringCB parent = createNode();
+            StringCB child = createNode(parent);
 
             path = parent.path;
+            children.add(child.nodeName());
+            
+            verify();
+        }
+        
+        public void verifyGetChildrenTwo() {
+            StringCB parent = createNode();
+            StringCB child1 = createNode(parent, "child1");
+            StringCB child2 = createNode(parent, "child2");
+        
+            path = parent.path;
+            children.add(child1.nodeName());
+            children.add(child2.nodeName());
+            
             verify();
         }
         
+        public void verifyGetChildrenFailure_NoNode() {
+            rc = KeeperException.Code.NONODE;
+            verify();
+        }
+        
+        @Override
+        public void verify() {
+            zk.getChildren(path, false, this, toString());
+            super.verify();
+        }
+
+        @Override
+        public String toString() {
+            return super.toString() + children.toString();
+        }
+    }
+
+    public static class Children2CB extends AsyncCB implements Children2Callback {
+        List<String> children = new ArrayList<String>();
+
+        Children2CB(ZooKeeper zk) {
+            this(zk, new CountDownLatch(1));
+        }
+
+        Children2CB(ZooKeeper zk, CountDownLatch latch) {
+            super(zk, latch);
+        }
+
+        public void processResult(int rc, String path, Object ctx,
+                List<String> children, Stat stat)
+        {
+            this.children =
+                (children == null ? new ArrayList<String>() : children);
+            super.processResult(Code.get(rc), path, ctx);
+        }
+        
+        public StringCB createNode() {
+            StringCB parent = new StringCB(zk);
+            parent.verifyCreate();
+
+            return parent;
+        }
+        
+        public StringCB createNode(StringCB parent) {
+            String childName = "bar";
+
+            return createNode(parent, childName);
+        }
+
+        public StringCB createNode(StringCB parent, String childName) {
+            StringCB child = new StringCB(zk);
+            child.setPath(parent.path + "/" + childName);
+            child.verifyCreate();
+            
+            return child;
+        }
+        
+        public void verifyGetChildrenEmpty() {
+            StringCB parent = createNode();
+            path = parent.path;
+            verify();
+        }
+
         public void verifyGetChildrenSingle() {
             StringCB parent = createNode();
             StringCB child = createNode(parent);
@@ -297,7 +382,7 @@
             StringCB parent = createNode();
             StringCB child1 = createNode(parent, "child1");
             StringCB child2 = createNode(parent, "child2");
-
+        
             path = parent.path;
             children.add(child1.nodeName());
             children.add(child2.nodeName());
@@ -313,7 +398,6 @@
         @Override
         public void verify() {
             zk.getChildren(path, false, this, toString());
-
             super.verify();
         }
         

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java Fri Oct 30 23:09:26 2009
@@ -24,6 +24,7 @@
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.test.AsyncOps.ACLCB;
 import org.apache.zookeeper.test.AsyncOps.ChildrenCB;
+import org.apache.zookeeper.test.AsyncOps.Children2CB;
 import org.apache.zookeeper.test.AsyncOps.DataCB;
 import org.apache.zookeeper.test.AsyncOps.StatCB;
 import org.apache.zookeeper.test.AsyncOps.StringCB;
@@ -167,6 +168,26 @@
     }
 
     @Test
+    public void testAsyncGetChildren2Empty() {
+        new ChildrenCB(zk).verifyGetChildrenEmpty();
+    }
+
+    @Test
+    public void testAsyncGetChildren2Single() {
+        new Children2CB(zk).verifyGetChildrenSingle();
+    }
+
+    @Test
+    public void testAsyncGetChildren2Two() {
+        new Children2CB(zk).verifyGetChildrenTwo();
+    }
+
+    @Test
+    public void testAsyncGetChildren2Failure_NoNode() {
+        new Children2CB(zk).verifyGetChildrenFailure_NoNode();
+    }
+
+    @Test
     public void testAsyncGetData() {
         new DataCB(zk).verifyGetData();
     }

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java Fri Oct 30 23:09:26 2009
@@ -231,11 +231,15 @@
                 List<String> children =
                     zk.getChildren("/test-" + i, false);
                 assertEquals(childCount, children.size());
+                children = zk.getChildren("/test-" + i, false, null);
+                assertEquals(childCount, children.size());
             }
             for (int i = 0; i < threads.length; i++) {
                 List<String> children =
                     zk.getChildren("/test-" + i, false);
                 assertEquals(childCount, children.size());
+                children = zk.getChildren("/test-" + i, false, null);
+                assertEquals(childCount, children.size());
             }
         } finally {
             zk.close();

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java Fri Oct 30 23:09:26 2009
@@ -334,6 +334,8 @@
             List<String> children = zk.getChildren("/pat", false);
             assertEquals(1, children.size());
             assertEquals("ben", children.get(0));
+            List<String> children2 = zk.getChildren("/pat", false, null);
+            assertEquals(children, children2);
             String value = new String(zk.getData("/pat/ben", false, stat));
             assertEquals("Ben was here", value);
             // Test stat and watch of non existent node

Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GetChildren2Test.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GetChildren2Test.java?rev=831486&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GetChildren2Test.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GetChildren2Test.java Fri Oct 30 23:09:26 2009
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class GetChildren2Test extends ClientBase {
+    private ZooKeeper zk;
+    
+    protected void setUp() throws Exception {
+        super.setUp();
+        
+        zk = createClient();
+    }
+
+    protected void tearDown() throws Exception {
+        super.tearDown();
+        
+        zk.close();
+    }
+    
+    @Test
+    public void testChild()
+        throws IOException, KeeperException, InterruptedException
+    {
+        String name = "/foo";
+        zk.create(name, name.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+
+        String childname = name + "/bar";
+        zk.create(childname, childname.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                CreateMode.EPHEMERAL);
+
+        Stat stat = new Stat();
+        List<String> s = zk.getChildren(name, false, stat);
+
+        assertEquals(stat.getCzxid(), stat.getMzxid());
+        assertEquals(stat.getCzxid() + 1, stat.getPzxid());
+        assertEquals(stat.getCtime(), stat.getMtime());
+        assertEquals(1, stat.getCversion());
+        assertEquals(0, stat.getVersion());
+        assertEquals(0, stat.getAversion());
+        assertEquals(0, stat.getEphemeralOwner());
+        assertEquals(name.length(), stat.getDataLength());
+        assertEquals(1, stat.getNumChildren());
+        assertEquals(s.size(), stat.getNumChildren());
+
+        s = zk.getChildren(childname, false, stat);
+
+        assertEquals(stat.getCzxid(), stat.getMzxid());
+        assertEquals(stat.getCzxid(), stat.getPzxid());
+        assertEquals(stat.getCtime(), stat.getMtime());
+        assertEquals(0, stat.getCversion());
+        assertEquals(0, stat.getVersion());
+        assertEquals(0, stat.getAversion());
+        assertEquals(zk.getSessionId(), stat.getEphemeralOwner());
+        assertEquals(childname.length(), stat.getDataLength());
+        assertEquals(0, stat.getNumChildren());
+        assertEquals(s.size(), stat.getNumChildren());
+    }
+
+    @Test
+    public void testChildren()
+        throws IOException, KeeperException, InterruptedException
+    {
+        String name = "/foo";
+        zk.create(name, name.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+
+        List<String> children = new ArrayList<String>();
+        List<String> children_s = new ArrayList<String>();
+
+        for (int i = 0; i < 10; i++) {
+            String childname = name + "/bar" + i;
+            String childname_s = "bar" + i;
+            children.add(childname);
+            children_s.add(childname_s);
+        }
+
+        for(int i = 0; i < children.size(); i++) {
+            String childname = children.get(i);
+            zk.create(childname, childname.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.EPHEMERAL);
+
+            Stat stat = new Stat();
+            List<String> s = zk.getChildren(name, false, stat);
+
+            assertEquals(stat.getCzxid(), stat.getMzxid());
+            assertEquals(stat.getCzxid() + i + 1, stat.getPzxid());
+            assertEquals(stat.getCtime(), stat.getMtime());
+            assertEquals(i + 1, stat.getCversion());
+            assertEquals(0, stat.getVersion());
+            assertEquals(0, stat.getAversion());
+            assertEquals(0, stat.getEphemeralOwner());
+            assertEquals(name.length(), stat.getDataLength());
+            assertEquals(i + 1, stat.getNumChildren());
+            assertEquals(s.size(), stat.getNumChildren());
+        }
+        List<String> p = zk.getChildren(name, false, null);
+        List<String> c_a = children_s;
+        List<String> c_b = p;
+        Collections.sort(c_a);
+        Collections.sort(c_b);
+        assertEquals(c_a.size(), 10);
+        assertEquals(c_a, c_b);
+    }
+}

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java Fri Oct 30 23:09:26 2009
@@ -28,14 +28,16 @@
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
 import org.junit.Test;
 
 
 public class SyncCallTest extends ClientBase
-    implements ChildrenCallback, StringCallback, VoidCallback
+    implements ChildrenCallback, Children2Callback, StringCallback, VoidCallback
 {
     private CountDownLatch opsCount;
     
@@ -57,8 +59,11 @@
             for(int i = 0; i < 100; i++)
                 zk.delete("/test" + i, 0, this, results);
             for(int i = 0; i < 100; i++)
-                zk.getChildren("/", new NullWatcher(), this, results);
-
+                zk.getChildren("/", new NullWatcher(), (ChildrenCallback)this,
+                        results);
+            for(int i = 0; i < 100; i++)
+                zk.getChildren("/", new NullWatcher(), (Children2Callback)this,
+                        results);
             LOG.info("Submitted all operations:" + (new Date()).toString());
             
             if(!opsCount.await(10000, TimeUnit.MILLISECONDS))
@@ -81,6 +86,13 @@
     }
 
     @SuppressWarnings("unchecked")
+    public void processResult(int rc, String path, Object ctx,
+            List<String> children, Stat stat) { 
+        ((List<Integer>)ctx).add(rc);
+        opsCount.countDown();
+    }
+
+    @SuppressWarnings("unchecked")
     public void processResult(int rc, String path, Object ctx, String name){
         ((List<Integer>) ctx).add(rc);
         opsCount.countDown();

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java Fri Oct 30 23:09:26 2009
@@ -66,12 +66,22 @@
       return;
     }
 
-    List<String> children = zk.getChildren(nodeName, false);
-    if (children.size() == 0) {
+    List<String> children1 = zk.getChildren(nodeName, false);
+    List<String> c2 = zk.getChildren(nodeName, false, stat);
+
+    if (!children1.equals(c2)) {
+        fail("children lists from getChildren()/getChildren2() do not match");
+    }
+
+    if (!stat.equals(stat)) {
+        fail("stats from exists()/getChildren2() do not match");
+    }
+
+    if (children1.size() == 0) {
       zk.delete(nodeName, -1);
       return;
     }
-    for (String n : children) {
+    for (String n : children1) {
       deleteZKDir(zk, n);
     }
   }
@@ -162,8 +172,8 @@
     ZooKeeper zk = new ZooKeeper(hostPort, 10000, this);
     ZooKeeper zk_1 = new ZooKeeper(hostPort, 10000, this);
 
-    Stat stat = zk_1.exists(parentName, false);
-    if (stat == null) {
+    Stat stat_parent = zk_1.exists(parentName, false);
+    if (stat_parent == null) {
       try {
         zk.create(parentName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
       } catch (KeeperException ke) {
@@ -171,8 +181,8 @@
       }
     }
 
-    stat = zk_1.exists(nodeName, false);
-    if (stat != null) {
+    Stat stat_node = zk_1.exists(nodeName, false);
+    if (stat_node != null) {
 
       try {
         zk.delete(nodeName, -1);
@@ -186,7 +196,17 @@
       }
     }
 
-    List<String> firstGen = zk_1.getChildren(parentName, true);
+    List<String> firstGen1 = zk_1.getChildren(parentName, true);
+    Stat stat = new Stat();
+    List<String> firstGen2 = zk_1.getChildren(parentName, true, stat);
+
+    if (!firstGen1.equals(firstGen2)) {
+        fail("children lists from getChildren()/getChildren2() do not match");
+    }
+
+    if (!stat_parent.equals(stat)) {
+        fail("stat from exists()/getChildren() do not match");
+    }
 
     try {
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
@@ -208,8 +228,8 @@
       fail("Unexpected event was delivered: " + event.toString());
     }
 
-    stat = zk_1.exists(nodeName, false);
-    if (stat == null) {
+    stat_node = zk_1.exists(nodeName, false);
+    if (stat_node == null) {
       fail("node " + nodeName + " should exist");
     }
 
@@ -236,8 +256,14 @@
     }
 
     try {
-      List<String> children = zk.getChildren(nodeName, false);
-      if (children.size() > 0) {
+      List<String> children1 = zk.getChildren(nodeName, false);
+      List<String> children2 = zk.getChildren(nodeName, false, null);
+
+      if (!children1.equals(children2)) {
+          fail("children lists from getChildren()/getChildren2() does not match");
+      }
+
+      if (children1.size() > 0) {
         fail("ephemeral node " + nodeName + " should not have children");
       }
     } catch (KeeperException ke) {
@@ -247,16 +273,22 @@
         fail("Unexpected exception code for createin: " + code);
       }
     }
-    firstGen = zk_1.getChildren(parentName, true);
-    stat = zk_1.exists(nodeName, true);
-    if (stat == null) {
+    firstGen1 = zk_1.getChildren(parentName, true);
+    firstGen2 = zk_1.getChildren(parentName, true, null);
+
+    if (!firstGen1.equals(firstGen2)) {
+        fail("children list from getChildren()/getChildren2() does not match");
+    }
+
+    stat_node = zk_1.exists(nodeName, true);
+    if (stat_node == null) {
       fail("node " + nodeName + " should exist");
     }
     System.out.println("session id of zk: " + zk.getSessionId());
     System.out.println("session id of zk_1: " + zk_1.getSessionId());
     zk.close();
 
-    stat = zk_1.exists("nosuchnode", false);
+    Stat no_stat = zk_1.exists("nosuchnode", false);
 
     event = this.getEvent(10);
     if (event == null) {
@@ -285,12 +317,12 @@
       fail("Unexpected second event was delivered: " + event.toString());
     }
 
-    firstGen = zk_1.getChildren(parentName, false);
-    stat = zk_1.exists(nodeName, false);
-    if (stat != null) {
+    firstGen1 = zk_1.getChildren(parentName, false);
+    stat_node = zk_1.exists(nodeName, false);
+    if (stat_node != null) {
       fail("node " + nodeName + " should have been deleted");
     }
-    if (firstGen.contains(nodeName)) {
+    if (firstGen1.contains(nodeName)) {
       fail("node " + nodeName + " should not be a children");
     }
     deleteZKDir(zk_1, nodeName);

Modified: hadoop/zookeeper/trunk/src/zookeeper.jute
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/zookeeper.jute?rev=831486&r1=831485&r2=831486&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/zookeeper.jute (original)
+++ hadoop/zookeeper/trunk/src/zookeeper.jute Fri Oct 30 23:09:26 2009
@@ -131,6 +131,10 @@
         ustring path;
         boolean watch;
     }
+    class GetChildren2Request {
+        ustring path;
+        boolean watch;
+    }
     class GetMaxChildrenRequest {
         ustring path;
     }
@@ -179,7 +183,11 @@
         org.apache.zookeeper.data.Stat stat;
     }
     class GetChildrenResponse {
-        vector<ustring>children;
+        vector<ustring> children;
+    }
+    class GetChildren2Response {
+        vector<ustring> children;
+        org.apache.zookeeper.data.Stat stat;
     }
     class GetACLResponse {
         vector<org.apache.zookeeper.data.ACL> acl;