You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ph...@apache.org on 2012/12/19 19:16:51 UTC

svn commit: r1423996 - in /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/main/org/apache/zookeeper/server/util/ src/java/test/org/apache/zookeeper...

Author: phunt
Date: Wed Dec 19 18:16:50 2012
New Revision: 1423996

URL: http://svn.apache.org/viewvc?rev=1423996&view=rev
Log:
ZOOKEEPER-1297. Add stat information to create() call (Lenni Kuff via phunt)

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/CreateTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/c/include/proto.h
    zookeeper/trunk/src/c/include/zookeeper.h
    zookeeper/trunk/src/c/src/cli.c
    zookeeper/trunk/src/c/src/zookeeper.c
    zookeeper/trunk/src/c/tests/TestClient.cc
    zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiResponse.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/OpResult.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
    zookeeper/trunk/src/zookeeper.jute

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Wed Dec 19 18:16:50 2012
@@ -452,6 +452,10 @@ IMPROVEMENTS:
 
   ZOOKEEPER-1504. Multi-thread NIOServerCnxn (Jay Shrauner via phunt)
 
+  ZOOKEEPER-1297. Add stat information to create() call
+  (Lenni Kuff via phunt)
+
+
 Release 3.4.0 - 
 
 Non-backward compatible changes:

Modified: zookeeper/trunk/src/c/include/proto.h
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/include/proto.h?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/c/include/proto.h (original)
+++ zookeeper/trunk/src/c/include/proto.h Wed Dec 19 18:16:50 2012
@@ -36,6 +36,7 @@ extern "C" {
 #define ZOO_GETCHILDREN2_OP 12
 #define ZOO_CHECK_OP 13
 #define ZOO_MULTI_OP 14
+#define ZOO_CREATE2_OP 15
 #define ZOO_CLOSE_OP -11
 #define ZOO_SETAUTH_OP 100
 #define ZOO_SETWATCHES_OP 101

Modified: zookeeper/trunk/src/c/include/zookeeper.h
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/include/zookeeper.h?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ zookeeper/trunk/src/c/include/zookeeper.h Wed Dec 19 18:16:50 2012
@@ -710,6 +710,31 @@ typedef void (*strings_completion_t)(int
         const struct String_vector *strings, const void *data);
 
 /**
+ * \brief signature of a completion function that returns a string 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 value the value of the string returned.
+ * \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 (*string_stat_completion_t)(int rc,
+        const char *string, const struct Stat *stat, const void *data);
+
+/**
  * \brief signature of a completion function that returns a list of strings and stat.
  * .
  * 
@@ -829,6 +854,42 @@ ZOOAPI int zoo_acreate(zhandle_t *zh, co
         string_completion_t completion, const void *data);
 
 /**
+ * \brief create a node asynchronously and returns stat details.
+ *
+ * This method will create a node in ZooKeeper. A node can only be created if
+ * it does not already exists. The Create Flags affect the creation of nodes.
+ * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the
+ * client session goes away. If the ZOO_SEQUENCE flag is set, a unique
+ * monotonically increasing sequence number is appended to the path name. The
+ * sequence number is always fixed length of 10 digits, 0 padded.
+ *
+ * \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 value The data to be stored in the node.
+ * \param valuelen The number of bytes in data.
+ * \param acl The initial ACL of the node. The ACL must not be null or empty.
+ * \param flags this parameter can be set to 0 for normal create or an OR
+ *    of the Create Flags
+ * \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 successfully
+ * ZNONODE the parent node does not exist.
+ * ZNODEEXISTS the node already exists
+ * ZNOAUTH the client does not have permission.
+ * ZNOCHILDRENFOREPHEMERALS cannot create children of ephemeral nodes.
+ * \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_acreate2(zhandle_t *zh, const char *path, const char *value,
+        int valuelen, const struct ACL_vector *acl, int flags,
+        string_stat_completion_t completion, const void *data);
+
+/**
  * \brief delete a node in zookeeper.
  * 
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
@@ -1308,6 +1369,48 @@ ZOOAPI int zoo_create(zhandle_t *zh, con
         char *path_buffer, int path_buffer_len);
 
 /**
+ * \brief create a node synchronously and collect stat details.
+ *
+ * This method will create a node in ZooKeeper. A node can only be created if
+ * it does not already exists. The Create Flags affect the creation of nodes.
+ * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the
+ * client session goes away. If the ZOO_SEQUENCE flag is set, a unique
+ * monotonically increasing sequence number is appended to the path name.
+ *
+ * \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 value The data to be stored in the node.
+ * \param valuelen The number of bytes in data. To set the data to be NULL use
+ * value as NULL and valuelen as -1.
+ * \param acl The initial ACL of the node. The ACL must not be null or empty.
+ * \param flags this parameter can be set to 0 for normal create or an OR
+ *    of the Create Flags
+ * \param path_buffer Buffer which will be filled with the path of the
+ *    new node (this might be different than the supplied path
+ *    because of the ZOO_SEQUENCE flag).  The path string will always be
+ *    null-terminated. This parameter may be NULL if path_buffer_len = 0.
+ * \param path_buffer_len Size of path buffer; if the path of the new
+ *    node (including space for the null terminator) exceeds the buffer size,
+ *    the path string will be truncated to fit.  The actual path of the
+ *    new node in the server will not be affected by the truncation.
+ *    The path string will always be null-terminated.
+ * \param stat The Stat struct to store Stat info into.
+ * \return  one of the following codes are returned:
+ * ZOK operation completed successfully
+ * ZNONODE the parent node does not exist.
+ * ZNODEEXISTS the node already exists
+ * ZNOAUTH the client does not have permission.
+ * ZNOCHILDRENFOREPHEMERALS cannot create children of ephemeral nodes.
+ * 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_create2(zhandle_t *zh, const char *path, const char *value,
+        int valuelen, const struct ACL_vector *acl, int flags,
+        char *path_buffer, int path_buffer_len, struct Stat *stat);
+
+/**
  * \brief delete a node in zookeeper synchronously.
  * 
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init

Modified: zookeeper/trunk/src/c/src/cli.c
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/src/cli.c?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/c/src/cli.c (original)
+++ zookeeper/trunk/src/c/src/cli.c Wed Dec 19 18:16:50 2012
@@ -182,6 +182,12 @@ void my_string_completion(int rc, const 
       shutdownThisThing=1;
 }
 
+void my_string_stat_completion(int rc, const char *name, const struct Stat *stat,
+        const void *data)  {
+    my_string_completion(rc, name, data);
+    dumpStat(stat);
+}
+
 void my_data_completion(int rc, const char *value, int value_len,
         const struct Stat *stat, const void *data) {
     struct timeval tv;
@@ -308,6 +314,7 @@ void processline(char *line) {
     }
     if (startsWith(line, "help")) {
       fprintf(stderr, "    create [+[e|s]] <path>\n");
+      fprintf(stderr, "    create2 [+[e|s]] <path>\n");
       fprintf(stderr, "    delete <path>\n");
       fprintf(stderr, "    set <path> <data>\n");
       fprintf(stderr, "    get <path>\n");
@@ -391,9 +398,10 @@ void processline(char *line) {
         if (rc) {
             fprintf(stderr, "Error %d for %s\n", rc, line);
         }
-    } else if (startsWith(line, "create ")) {
+    } else if (startsWith(line, "create ") || startsWith(line, "create2 ")) {
         int flags = 0;
-        line += 7;
+        int is_create2 = startsWith(line, "create2 ");
+        line += is_create2 ? 8 : 7;
         if (line[0] == '+') {
             line++;
             if (line[0] == 'e') {
@@ -417,8 +425,13 @@ void processline(char *line) {
 //            rc = zoo_acreate(zh, line, "new", 3, &CREATE_ONLY_ACL, flags,
 //                    my_string_completion, strdup(line));
 //        }
-        rc = zoo_acreate(zh, line, "new", 3, &ZOO_OPEN_ACL_UNSAFE, flags,
-                my_string_completion, strdup(line));
+        if (is_create2) {
+          rc = zoo_acreate2(zh, line, "new", 3, &ZOO_OPEN_ACL_UNSAFE, flags,
+                my_string_stat_completion, strdup(line));
+        } else {
+          rc = zoo_acreate(zh, line, "new", 3, &ZOO_OPEN_ACL_UNSAFE, flags,
+                  my_string_completion, strdup(line));
+        }
         if (rc) {
             fprintf(stderr, "Error %d for %s\n", rc, line);
         }
@@ -524,7 +537,7 @@ int main(int argc, char **argv) {
 
     if (argc < 2) {
         fprintf(stderr,
-                "USAGE %s zookeeper_host_list [clientid_file|cmd:(ls|ls2|create|od|...)]\n", 
+                "USAGE %s zookeeper_host_list [clientid_file|cmd:(ls|ls2|create|create2|od|...)]\n", 
                 argv[0]);
         fprintf(stderr,
                 "Version: ZooKeeper cli (c client) version %d.%d.%d\n", 

Modified: zookeeper/trunk/src/c/src/zookeeper.c
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/src/zookeeper.c?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ zookeeper/trunk/src/c/src/zookeeper.c Wed Dec 19 18:16:50 2012
@@ -144,6 +144,7 @@ struct ACL_vector ZOO_CREATOR_ALL_ACL = 
 #define COMPLETION_ACLLIST 5
 #define COMPLETION_STRING 6
 #define COMPLETION_MULTI 7
+#define COMPLETION_STRING_STAT 8
 
 typedef struct _auth_completion_list {
     void_completion_t completion;
@@ -161,6 +162,7 @@ typedef struct completion {
         strings_stat_completion_t strings_stat_result;
         acl_completion_t acl_result;
         string_completion_t string_result;
+        string_stat_completion_t string_stat_result;
         struct watcher_object_list *watcher_result;
     };
     completion_head_t clist; /* For multi-op */
@@ -1837,6 +1839,9 @@ static struct timeval get_timeval(int in
      const void *data);
  static int add_string_completion(zhandle_t *zh, int xid,
      string_completion_t dc, const void *data);
+ static int add_string_stat_completion(zhandle_t *zh, int xid,
+     string_stat_completion_t dc, const void *data);
+
 
  int send_ping(zhandle_t* zh)
  {
@@ -2286,6 +2291,26 @@ static void process_sync_completion(
             deallocate_CreateResponse(&res);
         }
         break;
+    case COMPLETION_STRING_STAT:
+        if (sc->rc==0) {
+            struct Create2Response res;
+            int len;
+            const char * client_path;
+            deserialize_Create2Response(ia, "reply", &res);
+            client_path = sub_string(zh, res.path);
+            len = strlen(client_path) + 1;
+            if (len > sc->u.str.str_len) {
+                len = sc->u.str.str_len;
+            }
+            if (len > 0) {
+                memcpy(sc->u.str.str, client_path, len - 1);
+                sc->u.str.str[len - 1] = '\0';
+            }
+            free_duplicate_path(client_path, res.path);
+            sc->u.stat = res.stat;
+            deallocate_Create2Response(&res);
+        }
+        break;
     case COMPLETION_ACLLIST:
         if (sc->rc==0) {
             struct GetACLResponse res;
@@ -2398,6 +2423,18 @@ static void deserialize_response(int typ
             deallocate_CreateResponse(&res);
         }
         break;
+    case COMPLETION_STRING_STAT:
+        LOG_DEBUG(("Calling COMPLETION_STRING_STAT for xid=%#x failed=%d, rc=%d",
+                    cptr->xid, failed, rc));
+        if (failed) {
+            cptr->c.string_stat_result(rc, 0, 0, cptr->data);
+        } else {
+            struct Create2Response res;
+            deserialize_Create2Response(ia, "reply", &res);
+            cptr->c.string_stat_result(rc, res.path, &res.stat, cptr->data);
+            deallocate_Create2Response(&res);
+        }
+        break;
     case COMPLETION_ACLLIST:
         LOG_DEBUG(("Calling COMPLETION_ACLLIST for xid=%#x failed=%d rc=%d",
                     cptr->xid, failed, rc));
@@ -2694,6 +2731,8 @@ static completion_list_t* create_complet
     case COMPLETION_STRINGLIST_STAT:
         c->c.strings_stat_result = (strings_stat_completion_t)dc;
         break;
+    case COMPLETION_STRING_STAT:
+        c->c.string_stat_result = (string_stat_completion_t)dc;
     case COMPLETION_ACLLIST:
         c->c.acl_result = (acl_completion_t)dc;
         break;
@@ -2817,6 +2856,12 @@ static int add_string_completion(zhandle
     return add_completion(zh, xid, COMPLETION_STRING, dc, data, 0, 0, 0);
 }
 
+static int add_string_stat_completion(zhandle_t *zh, int xid,
+        string_stat_completion_t dc, const void *data)
+{
+    return add_completion(zh, xid, COMPLETION_STRING_STAT, dc, data, 0, 0, 0);
+}
+
 static int add_multi_completion(zhandle_t *zh, int xid, void_completion_t dc,
         const void *data, completion_head_t *clist)
 {
@@ -3077,6 +3122,30 @@ static int CreateRequest_init(zhandle_t 
     return ZOK;
 }
 
+static int Create2Request_init(zhandle_t *zh, struct Create2Request *req,
+        const char *path, const char *value,
+        int valuelen, const struct ACL_vector *acl_entries, int flags)
+{
+    int rc;
+    assert(req);
+    rc = Request_path_init(zh, flags, &req->path, path);
+    assert(req);
+    if (rc != ZOK) {
+        return rc;
+    }
+    req->flags = flags;
+    req->data.buff = (char*)value;
+    req->data.len = valuelen;
+    if (acl_entries == 0) {
+        req->acl.count = 0;
+        req->acl.data = 0;
+    } else {
+        req->acl = *acl_entries;
+    }
+
+    return ZOK;
+}
+
 int zoo_acreate(zhandle_t *zh, const char *path, const char *value,
         int valuelen, const struct ACL_vector *acl_entries, int flags,
         string_completion_t completion, const void *data)
@@ -3109,6 +3178,37 @@ int zoo_acreate(zhandle_t *zh, const cha
     return (rc < 0)?ZMARSHALLINGERROR:ZOK;
 }
 
+int zoo_acreate2(zhandle_t *zh, const char *path, const char *value,
+        int valuelen, const struct ACL_vector *acl_entries, int flags,
+        string_stat_completion_t completion, const void *data)
+{
+    struct oarchive *oa;
+    struct RequestHeader h = { get_xid(), ZOO_CREATE2_OP };
+    struct Create2Request req;
+
+    int rc = Create2Request_init(zh, &req, path, value, valuelen, acl_entries, flags);
+    if (rc != ZOK) {
+        return rc;
+    }
+    oa = create_buffer_oarchive();
+    rc = serialize_RequestHeader(oa, "header", &h);
+    rc = rc < 0 ? rc : serialize_Create2Request(oa, "req", &req);
+    enter_critical(zh);
+    rc = rc < 0 ? rc : add_string_stat_completion(zh, h.xid, completion, data);
+    rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa),
+            get_buffer_len(oa));
+    leave_critical(zh);
+    free_duplicate_path(req.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,
+            zoo_get_current_server(zh)));
+    /* make a best (non-blocking) effort to send the requests asap */
+    adaptor_send_queue(zh, 0);
+    return (rc < 0)?ZMARSHALLINGERROR:ZOK;
+}
+
 int DeleteRequest_init(zhandle_t *zh, struct DeleteRequest *req, 
         const char *path, int version)
 {
@@ -3549,6 +3649,21 @@ void zoo_create_op_init(zoo_op_t *op, co
     op->create_op.buflen = path_buffer_len;
 }
 
+void zoo_create2_op_init(zoo_op_t *op, const char *path, const char *value,
+        int valuelen,  const struct ACL_vector *acl, int flags, 
+        char *path_buffer, int path_buffer_len)
+{
+    assert(op);
+    op->type = ZOO_CREATE2_OP;
+    op->create_op.path = path;
+    op->create_op.data = value;
+    op->create_op.datalen = valuelen;
+    op->create_op.acl = acl;
+    op->create_op.flags = flags;
+    op->create_op.buf = path_buffer;
+    op->create_op.buflen = path_buffer_len;
+}
+
 void zoo_delete_op_init(zoo_op_t *op, const char *path, int version)
 {
     assert(op);
@@ -3827,6 +3942,30 @@ int zoo_create(zhandle_t *zh, const char
     return rc;
 }
 
+int zoo_create2(zhandle_t *zh, const char *path, const char *value,
+        int valuelen, const struct ACL_vector *acl, int flags,
+        char *path_buffer, int path_buffer_len, struct Stat *stat)
+{
+    struct sync_completion *sc = alloc_sync_completion();
+    int rc;
+    if (!sc) {
+        return ZSYSTEMERROR;
+    }
+
+    sc->u.str.str = path_buffer;
+    sc->u.str.str_len = path_buffer_len;
+    rc=zoo_acreate2(zh, path, value, valuelen, acl, flags, SYNCHRONOUS_MARKER, sc);
+    if(rc==ZOK){
+        wait_sync_completion(sc);
+        rc = sc->rc;
+        if (rc == 0 && stat) {
+            *stat = sc->u.stat;
+        }
+    }
+    free_sync_completion(sc);
+    return rc;
+}
+
 int zoo_delete(zhandle_t *zh, const char *path, int version)
 {
     struct sync_completion *sc = alloc_sync_completion();

Modified: zookeeper/trunk/src/c/tests/TestClient.cc
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/c/tests/TestClient.cc?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/c/tests/TestClient.cc (original)
+++ zookeeper/trunk/src/c/tests/TestClient.cc Wed Dec 19 18:16:50 2012
@@ -191,6 +191,7 @@ class Zookeeper_simpleSystem : public CP
 #ifdef ZOO_IPV6_ENABLED
     CPPUNIT_TEST(testIPV6);
 #endif
+    CPPUNIT_TEST(testCreate);
     CPPUNIT_TEST(testPath);
     CPPUNIT_TEST(testPathValidation);
     CPPUNIT_TEST(testPing);
@@ -377,6 +378,12 @@ public:
         }
     }
 
+    static void stringStatCompletion(int rc, const char *value, const struct Stat *stat,
+            const void *data) {
+        stringCompletion(rc, value, data);
+        CPPUNIT_ASSERT(stat != 0);
+    }
+
     static void create_completion_fn(int rc, const char* value, const void *data) {
         CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
         count++;
@@ -622,6 +629,33 @@ public:
         CPPUNIT_ASSERT(zookeeper_get_connected_host(zk, &addr, &addr_len) != NULL);
     }
 
+    void testCreate() {
+        watchctx_t ctx;
+        int rc = 0;
+        zhandle_t *zk = createClient(&ctx);
+        CPPUNIT_ASSERT(zk);
+        char pathbuf[80];
+
+        struct Stat stat_a = {0};
+        struct Stat stat_b = {0};
+        rc = zoo_create2(zk, "/testcreateA", "", 0,
+                        &ZOO_OPEN_ACL_UNSAFE, 0, pathbuf, sizeof(pathbuf), &stat_a);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        CPPUNIT_ASSERT(strcmp(pathbuf, "/testcreateA") == 0);
+        CPPUNIT_ASSERT(stat_a.czxid > 0);
+        CPPUNIT_ASSERT(stat_a.mtime > 0);
+
+        rc = zoo_create2(zk, "/testcreateB", "", 0,
+                        &ZOO_OPEN_ACL_UNSAFE, 0, pathbuf, sizeof(pathbuf), &stat_b);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        CPPUNIT_ASSERT(strcmp(pathbuf, "/testcreateB") == 0);
+        CPPUNIT_ASSERT(stat_b.czxid > 0);
+        CPPUNIT_ASSERT(stat_b.mtime > 0);
+
+        // Should get different Stats back from different creates
+        CPPUNIT_ASSERT(Stat_eq(&stat_a, &stat_b) != 1);
+    }
+
     void testGetChildren2() {
         int rc;
         watchctx_t ctx;
@@ -892,9 +926,17 @@ public:
 
         yield(zk, 0);
 
-        for(i = 0; i < COUNT/2; i++) {
+        for(i = 0; i < COUNT/4; i++) {
             sprintf(path, "/awar%d", i);
-            rc = zoo_acreate(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0, stringCompletion, strdup(path));
+            rc = zoo_acreate(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0,
+                stringCompletion, strdup(path));
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+        }
+
+        for(i = COUNT/4; i < COUNT/2; i++) {
+            sprintf(path, "/awar%d", i);
+            rc = zoo_acreate2(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0,
+                stringStatCompletion, strdup(path));
             CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/AsyncCallback.java Wed Dec 19 18:16:50 2012
@@ -46,11 +46,16 @@ public interface AsyncCallback {
         public void processResult(int rc, String path, Object ctx,
                 List<String> children, Stat stat);
     }
+    
+    interface Create2Callback extends AsyncCallback {
+        public void processResult(int rc, String path, Object ctx,
+        		String name, Stat stat);
+    }
 
     interface StringCallback extends AsyncCallback {
         public void processResult(int rc, String path, Object ctx, String name);
     }
-
+    
     interface VoidCallback extends AsyncCallback {
         public void processResult(int rc, String path, Object ctx);
     }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Wed Dec 19 18:16:50 2012
@@ -44,6 +44,7 @@ import org.apache.jute.Record;
 import org.apache.zookeeper.AsyncCallback.ACLCallback;
 import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.AsyncCallback.Create2Callback;
 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.client.HostP
 import org.apache.zookeeper.client.ZooKeeperSaslClient;
 import org.apache.zookeeper.proto.AuthPacket;
 import org.apache.zookeeper.proto.ConnectRequest;
+import org.apache.zookeeper.proto.Create2Response;
 import org.apache.zookeeper.proto.CreateResponse;
 import org.apache.zookeeper.proto.ExistsResponse;
 import org.apache.zookeeper.proto.GetACLResponse;
@@ -601,6 +603,18 @@ public class ClientCnxn {
                       } else {
                           cb.processResult(rc, clientPath, p.ctx, null);
                       }
+                  } else if (p.response instanceof Create2Response) {
+                	  Create2Callback cb = (Create2Callback) p.cb;
+                      Create2Response rsp = (Create2Response) p.response;
+                      if (rc == 0) {
+                          cb.processResult(rc, clientPath, p.ctx,
+                                  (chrootPath == null
+                                          ? rsp.getPath()
+                                          : rsp.getPath()
+                                    .substring(chrootPath.length())), rsp.getStat());
+                      } else {
+                          cb.processResult(rc, clientPath, p.ctx, null, null);
+                      }
                   } else if (p.cb instanceof VoidCallback) {
                       VoidCallback cb = (VoidCallback) p.cb;
                       cb.processResult(rc, clientPath, p.ctx);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiResponse.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiResponse.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiResponse.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiResponse.java Wed Dec 19 18:16:50 2012
@@ -20,6 +20,7 @@ package org.apache.zookeeper;
 import org.apache.jute.InputArchive;
 import org.apache.jute.OutputArchive;
 import org.apache.jute.Record;
+import org.apache.zookeeper.proto.Create2Response;
 import org.apache.zookeeper.proto.CreateResponse;
 import org.apache.zookeeper.proto.MultiHeader;
 import org.apache.zookeeper.proto.SetDataResponse;
@@ -66,6 +67,11 @@ public class MultiResponse implements Re
                 case ZooDefs.OpCode.create:
                     new CreateResponse(((OpResult.CreateResult) result).getPath()).serialize(archive, tag);
                     break;
+                case ZooDefs.OpCode.create2:
+                	OpResult.CreateResult createResult = (OpResult.CreateResult) result;
+                    new Create2Response(createResult.getPath(),
+                    		createResult.getStat()).serialize(archive, tag);
+                    break;
                 case ZooDefs.OpCode.delete:
                 case ZooDefs.OpCode.check:
                     break;
@@ -98,6 +104,12 @@ public class MultiResponse implements Re
                     results.add(new OpResult.CreateResult(cr.getPath()));
                     break;
 
+                case ZooDefs.OpCode.create2:
+                    Create2Response cr2 = new Create2Response();
+                    cr2.deserialize(archive, tag);
+                    results.add(new OpResult.CreateResult(cr2.getPath(), cr2.getStat()));
+                    break;
+
                 case ZooDefs.OpCode.delete:
                     results.add(new OpResult.DeleteResult());
                     break;

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/MultiTransactionRecord.java Wed Dec 19 18:16:50 2012
@@ -69,6 +69,9 @@ public class MultiTransactionRecord impl
                case ZooDefs.OpCode.create:
                     op.toRequestRecord().serialize(archive, tag);
                     break;
+               case ZooDefs.OpCode.create2:
+                   op.toRequestRecord().serialize(archive, tag);
+                   break;
                 case ZooDefs.OpCode.delete:
                     op.toRequestRecord().serialize(archive, tag);
                     break;
@@ -99,6 +102,11 @@ public class MultiTransactionRecord impl
                     cr.deserialize(archive, tag);
                     add(Op.create(cr.getPath(), cr.getData(), cr.getAcl(), cr.getFlags()));
                     break;
+               case ZooDefs.OpCode.create2:
+                   Create2Request cr2 = new Create2Request();
+                   cr2.deserialize(archive, tag);
+                   add(Op.create(cr2.getPath(), cr2.getData(), cr2.getAcl(), cr2.getFlags()));
+                   break;
                 case ZooDefs.OpCode.delete:
                     DeleteRequest dr = new DeleteRequest();
                     dr.deserialize(archive, tag);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/OpResult.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/OpResult.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/OpResult.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/OpResult.java Wed Dec 19 18:16:50 2012
@@ -48,28 +48,48 @@ public abstract class OpResult {
      */
     public static class CreateResult extends OpResult {
         private String path;
+        private Stat stat;
 
         public CreateResult(String path) {
-            super(ZooDefs.OpCode.create);
+        	this(ZooDefs.OpCode.create, path, null);
+        }
+
+        public CreateResult(String path, Stat stat) {
+            this(ZooDefs.OpCode.create2, path, stat);
+        }
+
+        private CreateResult(int opcode, String path, Stat stat) {
+        	super(opcode);
             this.path = path;
+            this.stat = stat;
         }
 
         public String getPath() {
             return path;
         }
 
+        public Stat getStat() {
+            return stat;
+        }
+
         @Override
         public boolean equals(Object o) {
             if (this == o) return true;
             if (!(o instanceof CreateResult)) return false;
 
             CreateResult other = (CreateResult) o;
-            return getType() == other.getType() && path.equals(other.getPath());
+
+            boolean statsAreEqual = (stat == null && other.stat == null ||
+                        						(stat != null && other.stat != null &&
+                        					   stat.getMzxid() == other.stat.getMzxid()));
+            return getType() == other.getType() &&
+                   path.equals(other.getPath()) && statsAreEqual;
         }
 
         @Override
         public int hashCode() {
-            return getType() * 35 + path.hashCode();
+            return (int) (getType() * 35 + path.hashCode() +
+                    (stat == null ? 0 : stat.getMzxid()));
         }
     }
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooDefs.java Wed Dec 19 18:16:50 2012
@@ -53,6 +53,8 @@ public class ZooDefs {
         public final int check = 13;
 
         public final int multi = 14;
+        
+        public final int create2 = 15;
 
         public final int auth = 100;
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Wed Dec 19 18:16:50 2012
@@ -483,7 +483,7 @@ public class ZooKeeper {
      *            servers but there's partitioned server it could reach, it
      *            connects to one in read-only mode, i.e. read requests are
      *            allowed while write requests are not. It continues seeking for
-     *            majority in the background.
+     *           majority in the background.
      *
      * @throws IOException
      *             in cases of network failure
@@ -851,13 +851,103 @@ public class ZooKeeper {
     }
 
     /**
+     * Create a node with the given path and returns the Stat of that node. The
+     * node data will be the given data and node acl will be the given acl.
+     * <p>
+     * The flags argument specifies whether the created node will be ephemeral
+     * or not.
+     * <p>
+     * An ephemeral node will be removed by the ZooKeeper automatically when the
+     * session associated with the creation of the node expires.
+     * <p>
+     * The flags argument can also specify to create a sequential node. The
+     * actual path name of a sequential node will be the given path plus a
+     * suffix "i" where i is the current sequential number of the node. The sequence
+     * number is always fixed length of 10 digits, 0 padded. Once
+     * such a node is created, the sequential number will be incremented by one.
+     * <p>
+     * If a node with the same actual path already exists in the ZooKeeper, a
+     * KeeperException with error code KeeperException.NodeExists will be
+     * thrown. Note that since a different actual path is used for each
+     * invocation of creating sequential node with the same path argument, the
+     * call will never throw "file exists" KeeperException.
+     * <p>
+     * If the parent node does not exist in the ZooKeeper, a KeeperException
+     * with error code KeeperException.NoNode will be thrown.
+     * <p>
+     * An ephemeral node cannot have children. If the parent node of the given
+     * path is ephemeral, a KeeperException with error code
+     * KeeperException.NoChildrenForEphemerals will be thrown.
+     * <p>
+     * This operation, if successful, will trigger all the watches left on the
+     * node of the given path by exists and getData API calls, and the watches
+     * left on the parent node by getChildren API calls.
+     * <p>
+     * If a node is created successfully, the ZooKeeper server will trigger the
+     * watches on the path left by exists calls, and the watches on the parent
+     * of the node by getChildren calls.
+     * <p>
+     * The maximum allowable size of the data array is 1 MB (1,048,576 bytes).
+     * Arrays larger than this will cause a KeeperExecption to be thrown.
+     *
+     * @param path
+     *                the path for the node
+     * @param data
+     *                the initial data for the node
+     * @param acl
+     *                the acl for the node
+     * @param createMode
+     *                specifying whether the node to be created is ephemeral
+     *                and/or sequential
+     * @param stat
+     *                The output Stat object.
+     * @return the actual path of the created node
+     * @throws KeeperException if the server returns a non-zero error code
+     * @throws KeeperException.InvalidACLException if the ACL is invalid, null, or empty
+     * @throws InterruptedException if the transaction is interrupted
+     * @throws IllegalArgumentException if an invalid path is specified
+     */
+    public String create(final String path, byte data[], List<ACL> acl,
+            CreateMode createMode, Stat stat)
+            throws KeeperException, InterruptedException {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath, createMode.isSequential());
+
+        final String serverPath = prependChroot(clientPath);
+
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.create2);
+        Create2Request request = new Create2Request();
+        Create2Response response = new Create2Response();
+        request.setData(data);
+        request.setFlags(createMode.toFlag());
+        request.setPath(serverPath);
+        if (acl != null && acl.size() == 0) {
+            throw new KeeperException.InvalidACLException();
+        }
+        request.setAcl(acl);
+        ReplyHeader r = cnxn.submitRequest(h, request, response, null);
+        if (r.getErr() != 0) {
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()),
+                    clientPath);
+        }
+        if (stat != null) {
+            DataTree.copyStat(response.getStat(), stat);
+        }
+        if (cnxn.chrootPath == null) {
+            return response.getPath();
+        } else {
+            return response.getPath().substring(cnxn.chrootPath.length());
+        }
+    }
+
+    /**
      * The asynchronous version of create.
      *
      * @see #create(String, byte[], List, CreateMode)
      */
-
     public void create(final String path, byte data[], List<ACL> acl,
-            CreateMode createMode,  StringCallback cb, Object ctx)
+            CreateMode createMode, StringCallback cb, Object ctx)
     {
         final String clientPath = path;
         PathUtils.validatePath(clientPath, createMode.isSequential());
@@ -878,6 +968,32 @@ public class ZooKeeper {
     }
 
     /**
+     * The asynchronous version of create.
+     *
+     * @see #create(String, byte[], List, CreateMode, Stat)
+     */
+    public void create(final String path, byte data[], List<ACL> acl,
+            CreateMode createMode, Create2Callback cb, Object ctx)
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath, createMode.isSequential());
+
+        final String serverPath = prependChroot(clientPath);
+
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.create2);
+        Create2Request request = new Create2Request();
+        Create2Response response = new Create2Response();
+        ReplyHeader r = new ReplyHeader();
+        request.setData(data);
+        request.setFlags(createMode.toFlag());
+        request.setPath(serverPath);
+        request.setAcl(acl);
+        cnxn.queuePacket(h, r, request, response, cb, clientPath,
+                serverPath, ctx, null);
+    }
+
+    /**
      * Delete the node with the given path. The call will succeed if such a node
      * exists, and the given version matches the node's version (if the given
      * version is -1, it matches any node's versions).

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java Wed Dec 19 18:16:50 2012
@@ -38,6 +38,7 @@ import org.apache.jute.Record;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.Quotas;
 import org.apache.zookeeper.StatsTrack;
 import org.apache.zookeeper.WatchedEvent;
@@ -396,18 +397,51 @@ public class DataTree {
     }
 
     /**
+     * Add a new node to the DataTree.
      * @param path
+     * 			  Path for the new node.
      * @param data
+     *            Data to store in the node.
      * @param acl
+     *            Node acls
      * @param ephemeralOwner
      *            the session id that owns this node. -1 indicates this is not
      *            an ephemeral node.
      * @param zxid
+     *            Transaction ID
      * @param time
+     * @throws NodeExistsException 
+     * @throws NoNodeException 
      * @throws KeeperException
      */
     public void createNode(final String path, byte data[], List<ACL> acl,
             long ephemeralOwner, int parentCVersion, long zxid, long time)
+    		throws NoNodeException, NodeExistsException {
+    	createNode(path, data, acl, ephemeralOwner, parentCVersion, zxid, time, null);
+    }
+    
+    /**
+     * Add a new node to the DataTree.
+     * @param path
+     * 			  Path for the new node.
+     * @param data
+     *            Data to store in the node.
+     * @param acl
+     *            Node acls
+     * @param ephemeralOwner
+     *            the session id that owns this node. -1 indicates this is not
+     *            an ephemeral node.
+     * @param zxid
+     *            Transaction ID
+     * @param time
+     * @param outputStat
+     * 			  A Stat object to store Stat output results into.
+     * @throws NodeExistsException 
+     * @throws NoNodeException 
+     * @throws KeeperException
+     */
+    public void createNode(final String path, byte data[], List<ACL> acl,
+            long ephemeralOwner, int parentCVersion, long zxid, long time, Stat outputStat)
             throws KeeperException.NoNodeException,
             KeeperException.NodeExistsException {
         int lastSlash = path.lastIndexOf('/');
@@ -452,6 +486,9 @@ public class DataTree {
                     list.add(path);
                 }
             }
+            if (outputStat != null) {
+            	child.copyStat(outputStat);
+            }
         }
         // now check if its one of the zookeeper node child
         if (parentName.startsWith(quotaZookeeper)) {
@@ -741,7 +778,20 @@ public class DataTree {
                             createTxn.getAcl(),
                             createTxn.getEphemeral() ? header.getClientId() : 0,
                             createTxn.getParentCVersion(),
-                            header.getZxid(), header.getTime());
+                            header.getZxid(), header.getTime(), null);
+                    break;
+                case OpCode.create2:
+                    CreateTxn create2Txn = (CreateTxn) txn;
+                    rc.path = create2Txn.getPath();
+                    Stat stat = new Stat();
+                    createNode(
+                            create2Txn.getPath(),
+                            create2Txn.getData(),
+                            create2Txn.getAcl(),
+                            create2Txn.getEphemeral() ? header.getClientId() : 0,
+                            create2Txn.getParentCVersion(),
+                            header.getZxid(), header.getTime(), stat);
+                    rc.stat = stat;
                     break;
                 case OpCode.delete:
                     DeleteTxn deleteTxn = (DeleteTxn) txn;

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java Wed Dec 19 18:16:50 2012
@@ -33,6 +33,7 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.Create2Response;
 import org.apache.zookeeper.proto.CreateResponse;
 import org.apache.zookeeper.proto.ExistsRequest;
 import org.apache.zookeeper.proto.ExistsResponse;
@@ -193,6 +194,9 @@ public class FinalRequestProcessor imple
                         case OpCode.create:
                             subResult = new CreateResult(subTxnResult.path);
                             break;
+                        case OpCode.create2:
+                            subResult = new CreateResult(subTxnResult.path, subTxnResult.stat);
+                            break;
                         case OpCode.delete:
                             subResult = new DeleteResult();
                             break;
@@ -217,6 +221,12 @@ public class FinalRequestProcessor imple
                 err = Code.get(rc.err);
                 break;
             }
+            case OpCode.create2: {
+                lastOp = "CREA";
+                rsp = new Create2Response(rc.path, rc.stat);
+                err = Code.get(rc.err);
+                break;
+            }
             case OpCode.delete: {
                 lastOp = "DELE";
                 err = Code.get(rc.err);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java Wed Dec 19 18:16:50 2012
@@ -39,6 +39,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.BadArgumentsException;
 import org.apache.zookeeper.MultiTransactionRecord;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.ZooDefs;
@@ -49,6 +50,7 @@ import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.StatPersisted;
 import org.apache.zookeeper.proto.CreateRequest;
+import org.apache.zookeeper.proto.Create2Request;
 import org.apache.zookeeper.proto.DeleteRequest;
 import org.apache.zookeeper.proto.SetACLRequest;
 import org.apache.zookeeper.proto.SetDataRequest;
@@ -303,6 +305,22 @@ public class PrepRequestProcessor extend
     }
 
     /**
+     * Performs basic validation of a path for a create request.
+     * Throws if the path is not valid and returns the parent path.
+     * @throws BadArgumentsException
+     */
+    private String validatePathForCreate(String path, long sessionId)
+            throws BadArgumentsException {
+        int lastSlash = path.lastIndexOf('/');
+        if (lastSlash == -1 || path.indexOf('\0') != -1 || failCreate) {
+            LOG.info("Invalid path " + path + " with session 0x" +
+                    Long.toHexString(sessionId));
+            throw new KeeperException.BadArgumentsException(path);
+        }
+        return path.substring(0, lastSlash);
+    }
+
+    /**
      * This method will be called inside the ProcessRequestThread, which is a
      * singleton, so there will be a single thread calling this code.
      *
@@ -318,24 +336,72 @@ public class PrepRequestProcessor extend
         request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, zks.getTime(), type));
 
         switch (type) {
-            case OpCode.create:
+            case OpCode.create: {
                 zks.sessionTracker.checkSession(request.sessionId, request.getOwner());
                 CreateRequest createRequest = (CreateRequest)record;
                 if (deserialize) {
                     ByteBufferInputStream.byteBuffer2Record(request.request, createRequest);
                 }
                 String path = createRequest.getPath();
-                int lastSlash = path.lastIndexOf('/');
-                if (lastSlash == -1 || path.indexOf('\0') != -1 || failCreate) {
+                String parentPath = validatePathForCreate(path, request.sessionId);
+
+                List<ACL> listACL = removeDuplicates(createRequest.getAcl());
+                if (!fixupACL(request.authInfo, listACL)) {
+                    throw new KeeperException.InvalidACLException(path);
+                }
+                ChangeRecord parentRecord = getRecordForPath(parentPath);
+
+                checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE, request.authInfo);
+                int parentCVersion = parentRecord.stat.getCversion();
+                CreateMode createMode = CreateMode.fromFlag(createRequest.getFlags());
+                if (createMode.isSequential()) {
+                    path = path + String.format(Locale.ENGLISH, "%010d", parentCVersion);
+                }
+                try {
+                    PathUtils.validatePath(path);
+                } catch(IllegalArgumentException ie) {
                     LOG.info("Invalid path " + path + " with session 0x" +
                             Long.toHexString(request.sessionId));
                     throw new KeeperException.BadArgumentsException(path);
                 }
+                try {
+                    if (getRecordForPath(path) != null) {
+                        throw new KeeperException.NodeExistsException(path);
+                    }
+                } catch (KeeperException.NoNodeException e) {
+                    // ignore this one
+                }
+                boolean ephemeralParent = parentRecord.stat.getEphemeralOwner() != 0;
+                if (ephemeralParent) {
+                    throw new KeeperException.NoChildrenForEphemeralsException(path);
+                }
+                int newCversion = parentRecord.stat.getCversion()+1;
+                request.setTxn(new CreateTxn(path, createRequest.getData(), listACL, createMode.isEphemeral(),
+                        newCversion));
+                StatPersisted s = new StatPersisted();
+                if (createMode.isEphemeral()) {
+                    s.setEphemeralOwner(request.sessionId);
+                }
+                parentRecord = parentRecord.duplicate(request.getHdr().getZxid());
+                parentRecord.childCount++;
+                parentRecord.stat.setCversion(newCversion);
+                addChangeRecord(parentRecord);
+                addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, s, 0, listACL));
+                break;
+            }
+            case OpCode.create2: {
+                zks.sessionTracker.checkSession(request.sessionId, request.getOwner());
+                Create2Request createRequest = (Create2Request)record;
+                if (deserialize) {
+                    ByteBufferInputStream.byteBuffer2Record(request.request, createRequest);
+                }
+                String path = createRequest.getPath();
+                String parentPath = validatePathForCreate(path, request.sessionId);
+
                 List<ACL> listACL = removeDuplicates(createRequest.getAcl());
                 if (!fixupACL(request.authInfo, listACL)) {
                     throw new KeeperException.InvalidACLException(path);
                 }
-                String parentPath = path.substring(0, lastSlash);
                 ChangeRecord parentRecord = getRecordForPath(parentPath);
 
                 checkACL(zks, parentRecord.acl, ZooDefs.Perms.CREATE, request.authInfo);
@@ -375,19 +441,20 @@ public class PrepRequestProcessor extend
                 addChangeRecord(parentRecord);
                 addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, s, 0, listACL));
                 break;
+            }
             case OpCode.delete:
                 zks.sessionTracker.checkSession(request.sessionId, request.getOwner());
                 DeleteRequest deleteRequest = (DeleteRequest)record;
                 if(deserialize)
                     ByteBufferInputStream.byteBuffer2Record(request.request, deleteRequest);
-                path = deleteRequest.getPath();
-                lastSlash = path.lastIndexOf('/');
+                String path = deleteRequest.getPath();
+                int lastSlash = path.lastIndexOf('/');
                 if (lastSlash == -1 || path.indexOf('\0') != -1
                         || zks.getZKDatabase().isSpecialPath(path)) {
                     throw new KeeperException.BadArgumentsException(path);
                 }
-                parentPath = path.substring(0, lastSlash);
-                parentRecord = getRecordForPath(parentPath);
+                String parentPath = path.substring(0, lastSlash);
+                ChangeRecord parentRecord = getRecordForPath(parentPath);
                 ChangeRecord nodeRecord = getRecordForPath(path);
                 checkACL(zks, parentRecord.acl, ZooDefs.Perms.DELETE, request.authInfo);
                 checkAndIncVersion(nodeRecord.stat.getVersion(), deleteRequest.getVersion(), path);
@@ -420,7 +487,7 @@ public class PrepRequestProcessor extend
                 if(deserialize)
                     ByteBufferInputStream.byteBuffer2Record(request.request, setAclRequest);
                 path = setAclRequest.getPath();
-                listACL = removeDuplicates(setAclRequest.getAcl());
+                List<ACL> listACL = removeDuplicates(setAclRequest.getAcl());
                 if (!fixupACL(request.authInfo, listACL)) {
                     throw new KeeperException.InvalidACLException(path);
                 }
@@ -502,10 +569,14 @@ public class PrepRequestProcessor extend
 
         try {
             switch (request.type) {
-                case OpCode.create:
+            case OpCode.create:
                 CreateRequest createRequest = new CreateRequest();
                 pRequest2Txn(request.type, zks.getNextZxid(), request, createRequest, true);
                 break;
+            case OpCode.create2:
+                Create2Request create2Request = new Create2Request();
+                pRequest2Txn(request.type, zks.getNextZxid(), request, create2Request, true);
+                break;
             case OpCode.delete:
                 DeleteRequest deleteRequest = new DeleteRequest();               
                 pRequest2Txn(request.type, zks.getNextZxid(), request, deleteRequest, true);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java Wed Dec 19 18:16:50 2012
@@ -117,6 +117,7 @@ public class Request {
         case OpCode.notification:
             return false;
         case OpCode.create:
+        case OpCode.create2:
         case OpCode.delete:
         case OpCode.createSession:
         case OpCode.exists:
@@ -149,6 +150,7 @@ public class Request {
         case OpCode.error:
         case OpCode.closeSession:
         case OpCode.create:
+        case OpCode.create2:
         case OpCode.createSession:
         case OpCode.delete:
         case OpCode.setACL:
@@ -167,6 +169,8 @@ public class Request {
             return "notification";
         case OpCode.create:
             return "create";
+        case OpCode.create2:
+            return "create2";
         case OpCode.setWatches:
             return "setWatches";
         case OpCode.delete:

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/util/SerializeUtils.java Wed Dec 19 18:16:50 2012
@@ -65,6 +65,7 @@ public class SerializeUtils {
         case OpCode.closeSession:
             return null;
         case OpCode.create:
+        case OpCode.create2:
             txn = new CreateTxn();
             break;
         case OpCode.delete:

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOps.java Wed Dec 19 18:16:50 2012
@@ -29,6 +29,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.ACLCallback;
 import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.AsyncCallback.Create2Callback;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.AsyncCallback.StatCallback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
@@ -389,6 +390,64 @@ public class AsyncOps {
         }
     }
 
+    public static class Create2CB extends AsyncCB implements Create2Callback {
+    	  byte[] data = new byte[10];
+        List<ACL> acl = Ids.CREATOR_ALL_ACL;
+        CreateMode flags = CreateMode.PERSISTENT;
+        String name = path;
+        Stat stat = new Stat();
+
+        Create2CB(ZooKeeper zk) {
+            this(zk, new CountDownLatch(1));
+        }
+
+        Create2CB(ZooKeeper zk, CountDownLatch latch) {
+            super(zk, latch);
+        }
+
+        public void setPath(String path) {
+            super.setPath(path);
+            this.name = path;
+        }
+
+        public String nodeName() {
+            return path.substring(path.lastIndexOf('/') + 1);
+        }
+
+        public void processResult(int rc, String path, Object ctx,
+                String name, Stat stat) {
+            this.name = name;
+            this.stat = stat;
+            super.processResult(Code.get(rc), path, ctx);
+        }
+
+        public AsyncCB create() {
+            zk.create(path, data, acl, flags, this, toString());
+            return this;
+        }
+
+        public void verifyCreate() {
+            create();
+            verify();
+        }
+
+        public void verifyCreateFailure_NodeExists() {
+            new Create2CB(zk).verifyCreate();
+            rc = Code.NODEEXISTS;
+            name = null;
+            zk.create(path, data, acl, flags, this, toString());
+            verify();
+        }
+
+        @Override
+        public String toString() {
+            return super.toString() + name + ":" +
+                (stat == null ? "null" : stat.getAversion() + ":" +
+            		 stat.getCversion() + ":" + stat.getEphemeralOwner() +
+                 ":" + stat.getVersion());
+        }
+    }
+
     public static class DataCB extends AsyncCB implements DataCallback {
         byte[] data = new byte[10];
         Stat stat = new Stat();

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncOpsTest.java Wed Dec 19 18:16:50 2012
@@ -26,6 +26,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.test.AsyncOps.ACLCB;
 import org.apache.zookeeper.test.AsyncOps.Children2CB;
 import org.apache.zookeeper.test.AsyncOps.ChildrenCB;
+import org.apache.zookeeper.test.AsyncOps.Create2CB;
 import org.apache.zookeeper.test.AsyncOps.DataCB;
 import org.apache.zookeeper.test.AsyncOps.StatCB;
 import org.apache.zookeeper.test.AsyncOps.StringCB;
@@ -66,6 +67,11 @@ public class AsyncOpsTest extends Client
     }
 
     @Test
+    public void testAsyncCreate2() {
+        new Create2CB(zk).verifyCreate();
+    }
+
+    @Test
     public void testAsyncCreateThree() {
         CountDownLatch latch = new CountDownLatch(3);
 

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/CreateTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/CreateTest.java?rev=1423996&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/CreateTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/CreateTest.java Wed Dec 19 18:16:50 2012
@@ -0,0 +1,120 @@
+/**
+ * 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.AsyncCallback.Create2Callback;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.junit.Assert;
+import org.junit.Test;
+/**
+ * Test suite for validating the Create API.
+ */
+public class CreateTest extends ClientBase {
+  private ZooKeeper zk;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    zk = createClient();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    zk.close();
+  }
+
+  @Test
+  public void testCreate()
+      throws IOException, KeeperException, InterruptedException {
+    createNoStatVerifyResult("/foo");
+    createNoStatVerifyResult("/foo/child");
+  }
+
+  @Test
+  public void testCreateWithStat()
+      throws IOException, KeeperException, InterruptedException {
+    String name = "/foo";
+    Stat stat = createWithStatVerifyResult("/foo");
+    Stat childStat = createWithStatVerifyResult("/foo/child");
+    // Don't expect to get the same stats for different creates.
+    Assert.assertFalse(stat.equals(childStat));
+  }
+
+  @Test
+  public void testCreateWithNullStat()
+      throws IOException, KeeperException, InterruptedException {
+    String name = "/foo";
+    Assert.assertNull(zk.exists(name, false));
+
+    Stat stat = null;
+    // If a null Stat object is passed the create should still
+    // succeed, but no Stat info will be returned.
+    String path = zk.create(name, name.getBytes(),
+        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, stat);
+    Assert.assertNull(stat);
+    Assert.assertNotNull(zk.exists(name, false));
+  }
+
+  private void createNoStatVerifyResult(String newName)
+      throws KeeperException, InterruptedException {
+    Assert.assertNull("Node existed before created", zk.exists(newName, false));
+    String path = zk.create(newName, newName.getBytes(),
+                            Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    Assert.assertEquals(path, newName);
+    Assert.assertNotNull("Node was not created as expected",
+                         zk.exists(newName, false));
+  }
+
+  private Stat createWithStatVerifyResult(String newName)
+        throws KeeperException, InterruptedException {
+    Assert.assertNull("Node existed before created", zk.exists(newName, false));
+    Stat stat = new Stat();
+    String path = zk.create(newName, newName.getBytes(),
+                            Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, stat);
+    Assert.assertEquals(path, newName);
+    validateCreateStat(stat, newName);
+
+    Stat referenceStat = zk.exists(newName, false);
+    Assert.assertNotNull("Node was not created as expected", referenceStat);
+    Assert.assertEquals(referenceStat, stat);
+
+    return stat;
+  }
+
+  private void validateCreateStat(Stat stat, String name) {
+    Assert.assertEquals(stat.getCzxid(), stat.getMzxid());
+    Assert.assertEquals(stat.getCzxid(), stat.getPzxid());
+    Assert.assertEquals(stat.getCtime(), stat.getMtime());
+    Assert.assertEquals(0, stat.getCversion());
+    Assert.assertEquals(0, stat.getVersion());
+    Assert.assertEquals(0, stat.getAversion());
+    Assert.assertEquals(0, stat.getEphemeralOwner());
+    Assert.assertEquals(name.length(), stat.getDataLength());
+    Assert.assertEquals(0, stat.getNumChildren());
+  }
+}

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java Wed Dec 19 18:16:50 2012
@@ -328,28 +328,40 @@ public class MultiTransactionTest extend
     @Test
     public void testOpResultEquals() {
         opEquals(new CreateResult("/foo"),
-                new CreateResult("/foo"),
-                new CreateResult("nope"));
+                 new CreateResult("/foo"),
+                 new CreateResult("nope"));
+
+        opEquals(new CreateResult("/foo"),
+                 new CreateResult("/foo"),
+                 new CreateResult("/foo", new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)));
+
+        opEquals(new CreateResult("/foo", new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
+                 new CreateResult("/foo", new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
+                 new CreateResult("nope", new Stat(11, 12, 13, 14, 15, 16, 17, 18, 19, 110, 111)));
+
+        opEquals(new CreateResult("/foo", new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
+                 new CreateResult("/foo", new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
+                 new CreateResult("/foo"));
 
         opEquals(new CheckResult(),
-                new CheckResult(),
-                null);
-        
+                 new CheckResult(),
+                 null);
+
         opEquals(new SetDataResult(new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
-                new SetDataResult(new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
-                new SetDataResult(new Stat(11, 12, 13, 14, 15, 16, 17, 18, 19, 110, 111)));
+                 new SetDataResult(new Stat(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)),
+                 new SetDataResult(new Stat(11, 12, 13, 14, 15, 16, 17, 18, 19, 110, 111)));
         
         opEquals(new ErrorResult(1),
-                new ErrorResult(1),
-                new ErrorResult(2));
+                 new ErrorResult(1),
+                 new ErrorResult(2));
         
         opEquals(new DeleteResult(),
-                new DeleteResult(),
-                null);
+                 new DeleteResult(),
+                 null);
 
         opEquals(new ErrorResult(1),
-                new ErrorResult(1),
-                new ErrorResult(2));
+                 new ErrorResult(1),
+                 new ErrorResult(2));
     }
 
     private void opEquals(OpResult expected, OpResult value, OpResult near) {

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java Wed Dec 19 18:16:50 2012
@@ -29,6 +29,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.Children2Callback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
+import org.apache.zookeeper.AsyncCallback.Create2Callback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
@@ -37,7 +38,8 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class SyncCallTest extends ClientBase
-    implements ChildrenCallback, Children2Callback, StringCallback, VoidCallback
+    implements ChildrenCallback, Children2Callback,
+               StringCallback, VoidCallback, Create2Callback
 {
     private CountDownLatch opsCount;
     
@@ -50,11 +52,16 @@ public class SyncCallTest extends Client
             LOG.info("Starting ZK:" + (new Date()).toString());
             opsCount = new CountDownLatch(limit);
             ZooKeeper zk = createClient();
-            
+
             LOG.info("Beginning test:" + (new Date()).toString());
-            for(int i = 0; i < 100; i++)
+            for(int i = 0; i < 50; i++)
                 zk.create("/test" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
-                        CreateMode.PERSISTENT, this, results);
+                          CreateMode.PERSISTENT, (StringCallback)this, results);
+
+            for(int i = 50; i < 100; i++) {
+              zk.create("/test" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT, (Create2Callback)this, results);
+            }
             zk.sync("/test", this, results);
             for(int i = 0; i < 100; i++)
                 zk.delete("/test" + i, 0, this, results);
@@ -105,4 +112,11 @@ public class SyncCallTest extends Client
         opsCount.countDown();
     
     }
+
+    @Override
+    public void processResult(int rc, String path, Object ctx, String name,
+        Stat stat) {
+      ((List<Integer>) ctx).add(rc);
+      opsCount.countDown();
+    }
 }

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java Wed Dec 19 18:16:50 2012
@@ -18,6 +18,7 @@
 
 package org.apache.zookeeper.test;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -33,6 +34,8 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 
 public class ZooKeeperTestClient extends ZKTestCase implements Watcher {
@@ -394,11 +397,53 @@ public class ZooKeeperTestClient extends
     zk.close();
   }
 
+  private void deleteNodeIfExists(ZooKeeper zk, String nodeName)
+      throws InterruptedException {
+    try {
+      zk.delete(nodeName, -1);
+    } catch (KeeperException ke) {
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NONODE ||
+                      code == KeeperException.Code.NOTEMPTY;
+      if (!valid) {
+        Assert.fail("Unexpected exception code for delete: " + ke.getMessage());
+      }
+    }
+  }
+
+  private void create_get_stat_test()
+      throws IOException, InterruptedException, KeeperException {
+    checkRoot();
+    ZooKeeper zk = new ZooKeeper(hostPort, 10000, this);
+    String parentName = testDirOnZK;
+    String nodeName = parentName + "/create_with_stat_tmp";
+    deleteNodeIfExists(zk, nodeName);
+    deleteNodeIfExists(zk, nodeName + "_2");
+    Stat stat = new Stat();
+    zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
+        stat);
+    Assert.assertNotNull(stat);
+    Assert.assertTrue(stat.getCzxid() > 0);
+    Assert.assertTrue(stat.getCtime() > 0);
+
+    Stat stat2 = new Stat();
+    zk.create(nodeName + "_2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
+        stat2);
+    Assert.assertNotNull(stat2);
+    Assert.assertTrue(stat2.getCzxid() > stat.getCzxid());
+    Assert.assertTrue(stat2.getCtime() > stat.getCtime());
+
+    deleteNodeIfExists(zk, nodeName);
+    deleteNodeIfExists(zk, nodeName + "_2");
+    zk.close();
+  }
+
   public void my_test_1() throws IOException,
           InterruptedException, KeeperException {
     enode_test_1();
     enode_test_2();
     delete_create_get_set_test_1();
+    create_get_stat_test();
   }
 
   synchronized public void process(WatchedEvent event) {
@@ -418,4 +463,4 @@ public class ZooKeeperTestClient extends
       e.printStackTrace();
     }
   }
-}
+}
\ No newline at end of file

Modified: zookeeper/trunk/src/zookeeper.jute
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/zookeeper.jute?rev=1423996&r1=1423995&r2=1423996&view=diff
==============================================================================
--- zookeeper/trunk/src/zookeeper.jute (original)
+++ zookeeper/trunk/src/zookeeper.jute Wed Dec 19 18:16:50 2012
@@ -119,6 +119,12 @@ module org.apache.zookeeper.proto {
         vector<org.apache.zookeeper.data.ACL> acl;
         int flags;
     }
+    class Create2Request {
+        ustring path;
+        buffer data;
+        vector<org.apache.zookeeper.data.ACL> acl;
+        int flags;
+    }
     class DeleteRequest {
         ustring path;
         int version;
@@ -173,6 +179,10 @@ module org.apache.zookeeper.proto {
     class CreateResponse {
         ustring path;
     }
+    class Create2Response {
+        ustring path;
+        org.apache.zookeeper.data.Stat stat;
+    }
     class ExistsRequest {
         ustring path;
         boolean watch;