Browse Source

ZOOKEEPER-237. Add a Chroot request (phunt and mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@788205 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 16 năm trước cách đây
mục cha
commit
e3650bf5dd

+ 2 - 0
CHANGES.txt

@@ -251,6 +251,8 @@ NEW FEATURES:
 
   ZOOKEEPER-395. Python bindings. (henry robinson via mahadev)
 
+  ZOOKEEPER-237. Add a Chroot request (phunt and mahadev)
+
 Release 3.1.0 - 2009-02-06
   
 Non-backward compatible changes:

+ 5 - 2
docs/zookeeperAdmin.html

@@ -1450,8 +1450,11 @@ imok
 <a name="N104A0"></a><a name="sc_bestPractices"></a>
 <h3 class="h4">Best Practices</h3>
 <p>For best results, take note of the following list of good
-      Zookeeper practices. <em>[tbd...]</em>
-</p>
+      Zookeeper practices:</p>
+<p>For multi-tennant installations see the <a href="zookeeperProgrammers.html#ch_zkSessions">section</a>
+      detailing ZooKeeper "chroot" support, this can be very useful
+      when deploying many applications/services interfacing to a
+      single ZooKeeper cluster.</p>
 </div>
 
 <p align="right">

Những thai đổi đã bị hủy bỏ vì nó quá lớn
+ 1 - 2
docs/zookeeperAdmin.pdf


+ 41 - 24
docs/zookeeperProgrammers.html

@@ -758,7 +758,7 @@ document.write("Last Published: " + document.lastModified);
 <h2 class="h3">ZooKeeper Sessions</h2>
 <div class="section">
 <p>To create a client session the application code must provide
-    a string containing a comma separated list of host:port pairs,
+    a connection string containing a comma separated list of host:port pairs,
     each corresponding to a ZooKeeper server (e.g. "127.0.0.1:4545" or
     "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"). The ZooKeeper
     client library will pick an arbitrary server and try to connect to
@@ -766,6 +766,23 @@ document.write("Last Published: " + document.lastModified);
     disconnected from the server for any reason, the client will
     automatically try the next server in the list, until a connection
     is (re-)established.</p>
+<p> 
+<strong>Added in 3.2.0</strong>: 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). If used the example would look like:
+    "127.0.0.1:4545/app/a" or
+    "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" where the
+    client would be rooted at "/app/a" and all paths would be relative
+    to this root - ie getting/setting/etc...  "/foo/bar" would result
+    in operations being run on "/app/a/foo/bar" (from the server
+    perspective). This feature is particularly useful in multi-tenant
+    environments where each user of a particular ZooKeeper service
+    could be rooted differently. This makes re-use much simpler as
+    each user can code his/her application as if it were rooted at
+    "/", while actual location (say /app/a) could be determined at
+    deployment time.</p>
 <p>When a client gets a handle to the ZooKeeper service,
     ZooKeeper creates a ZooKeeper session, represented as a 64-bit
     number, that it assigns to the client. If the client connects to a
@@ -825,7 +842,7 @@ document.write("Last Published: " + document.lastModified);
 </div>
 
   
-<a name="N101C9"></a><a name="ch_zkWatches"></a>
+<a name="N101CF"></a><a name="ch_zkWatches"></a>
 <h2 class="h3">ZooKeeper Watches</h2>
 <div class="section">
 <p>All of the read operations in ZooKeeper - <strong>getData()</strong>, <strong>getChildren()</strong>, and <strong>exists()</strong> - have the option of setting a watch as a
@@ -908,7 +925,7 @@ document.write("Last Published: " + document.lastModified);
     general this all occurs transparently. There is one case where a watch
     may be missed: a watch for the existance of a znode not yet created will
     be missed if the znode is created and deleted while disconnected.</p>
-<a name="N101FF"></a><a name="sc_WatchGuarantees"></a>
+<a name="N10205"></a><a name="sc_WatchGuarantees"></a>
 <h3 class="h4">What ZooKeeper Guarantees about Watches</h3>
 <p>With regard to watches, ZooKeeper maintains these
       guarantees:</p>
@@ -943,7 +960,7 @@ document.write("Last Published: " + document.lastModified);
 </li>
       
 </ul>
-<a name="N10224"></a><a name="sc_WatchRememberThese"></a>
+<a name="N1022A"></a><a name="sc_WatchRememberThese"></a>
 <h3 class="h4">Things to Remember about Watches</h3>
 <ul>
         
@@ -1002,7 +1019,7 @@ document.write("Last Published: " + document.lastModified);
 </div>
 
   
-<a name="N10250"></a><a name="sc_ZooKeeperAccessControl"></a>
+<a name="N10256"></a><a name="sc_ZooKeeperAccessControl"></a>
 <h2 class="h3">ZooKeeper access control using ACLs</h2>
 <div class="section">
 <p>ZooKeeper uses ACLs to control access to its znodes (the
@@ -1037,7 +1054,7 @@ document.write("Last Published: " + document.lastModified);
     example, the pair <em>(ip:19.22.0.0/16, READ)</em>
     gives the <em>READ</em> permission to any clients with
     an IP address that starts with 19.22.</p>
-<a name="N10283"></a><a name="sc_ACLPermissions"></a>
+<a name="N10289"></a><a name="sc_ACLPermissions"></a>
 <h3 class="h4">ACL Permissions</h3>
 <p>ZooKeeper supports the following permissions:</p>
 <ul>
@@ -1093,7 +1110,7 @@ document.write("Last Published: " + document.lastModified);
       node, but nothing more. (The problem is, if you want to call
       zoo_exists() on a node that doesn't exist, there is no
       permission to check.)</p>
-<a name="N102D9"></a><a name="sc_BuiltinACLSchemes"></a>
+<a name="N102DF"></a><a name="sc_BuiltinACLSchemes"></a>
 <h4>Builtin ACL Schemes</h4>
 <p>ZooKeeeper has the following built in schemes:</p>
 <ul>
@@ -1142,7 +1159,7 @@ document.write("Last Published: " + document.lastModified);
 
       
 </ul>
-<a name="N1031D"></a><a name="ZooKeeper+C+client+API"></a>
+<a name="N10323"></a><a name="ZooKeeper+C+client+API"></a>
 <h4>ZooKeeper C client API</h4>
 <p>The following constants are provided by the ZooKeeper C
       library:</p>
@@ -1364,7 +1381,7 @@ int main(int argc, char argv) {
 </div>
 
   
-<a name="N10434"></a><a name="sc_ZooKeeperPluggableAuthentication"></a>
+<a name="N1043A"></a><a name="sc_ZooKeeperPluggableAuthentication"></a>
 <h2 class="h3">Pluggable ZooKeeper authentication</h2>
 <div class="section">
 <p>ZooKeeper runs in a variety of different environments with
@@ -1450,7 +1467,7 @@ authProvider.2=com.f.MyAuth2
 </div>
       
   
-<a name="N104A0"></a><a name="ch_zkGuarantees"></a>
+<a name="N104A6"></a><a name="ch_zkGuarantees"></a>
 <h2 class="h3">Consistency Guarantees</h2>
 <div class="section">
 <p>ZooKeeper is a high performance, scalable service. Both reads and
@@ -1576,12 +1593,12 @@ authProvider.2=com.f.MyAuth2
 </div>
 
   
-<a name="N10507"></a><a name="ch_bindings"></a>
+<a name="N1050D"></a><a name="ch_bindings"></a>
 <h2 class="h3">Bindings</h2>
 <div class="section">
 <p>The ZooKeeper client libraries come in two languages: Java and C.
     The following sections describe these.</p>
-<a name="N10510"></a><a name="Java+Binding"></a>
+<a name="N10516"></a><a name="Java+Binding"></a>
 <h3 class="h4">Java Binding</h3>
 <p>There are two packages that make up the ZooKeeper Java binding:
       <strong>org.apache.zookeeper</strong> and <strong>org.apache.zookeeper.data</strong>. The rest of the
@@ -1648,7 +1665,7 @@ authProvider.2=com.f.MyAuth2
       (SESSION_EXPIRED and AUTH_FAILED), the ZooKeeper object becomes invalid.
       On a close, the two threads shut down and any further access on zookeeper
       handle is undefined behavior and should be avoided. </p>
-<a name="N10559"></a><a name="C+Binding"></a>
+<a name="N1055F"></a><a name="C+Binding"></a>
 <h3 class="h4">C Binding</h3>
 <p>The C binding has a single-threaded and multi-threaded library.
       The multi-threaded library is easiest to use and is most similar to the
@@ -1665,7 +1682,7 @@ authProvider.2=com.f.MyAuth2
       (i.e. FreeBSD 4.x). In all other cases, application developers should
       link with zookeeper_mt, as it includes support for both Sync and Async
       API.</p>
-<a name="N10568"></a><a name="Installation"></a>
+<a name="N1056E"></a><a name="Installation"></a>
 <h4>Installation</h4>
 <p>If you're building the client from a check-out from the Apache
         repository, follow the steps outlined below. If you're building from a
@@ -1796,7 +1813,7 @@ authProvider.2=com.f.MyAuth2
 </li>
         
 </ol>
-<a name="N10611"></a><a name="Using+the+C+Client"></a>
+<a name="N10617"></a><a name="Using+the+C+Client"></a>
 <h4>Using the C Client</h4>
 <p>You can test your client by running a ZooKeeper server (see
         instructions on the project wiki page on how to run it) and connecting
@@ -1854,7 +1871,7 @@ authProvider.2=com.f.MyAuth2
 </div>
 
    
-<a name="N10657"></a><a name="ch_guideToZkOperations"></a>
+<a name="N1065D"></a><a name="ch_guideToZkOperations"></a>
 <h2 class="h3">Building Blocks: A Guide to ZooKeeper Operations</h2>
 <div class="section">
 <p>This section surveys all the operations a developer can perform
@@ -1872,28 +1889,28 @@ authProvider.2=com.f.MyAuth2
 </li>
     
 </ul>
-<a name="N1066B"></a><a name="sc_errorsZk"></a>
+<a name="N10671"></a><a name="sc_errorsZk"></a>
 <h3 class="h4">Handling Errors</h3>
 <p>Both the Java and C client bindings may report errors. The Java client binding does so by throwing KeeperException, calling code() on the exception will return the specific error code. The C client binding returns an error code as defined in the enum ZOO_ERRORS. API callbacks indicate result code for both language bindings. See the API documentation (javadoc for Java, doxygen for C) for full details on the possible errors and their meaning.</p>
-<a name="N10675"></a><a name="sc_connectingToZk"></a>
+<a name="N1067B"></a><a name="sc_connectingToZk"></a>
 <h3 class="h4">Connecting to ZooKeeper</h3>
 <p></p>
-<a name="N1067E"></a><a name="sc_readOps"></a>
+<a name="N10684"></a><a name="sc_readOps"></a>
 <h3 class="h4">Read Operations</h3>
 <p></p>
-<a name="N10687"></a><a name="sc_writeOps"></a>
+<a name="N1068D"></a><a name="sc_writeOps"></a>
 <h3 class="h4">Write Operations</h3>
 <p></p>
-<a name="N10690"></a><a name="sc_handlingWatches"></a>
+<a name="N10696"></a><a name="sc_handlingWatches"></a>
 <h3 class="h4">Handling Watches</h3>
 <p></p>
-<a name="N10699"></a><a name="sc_miscOps"></a>
+<a name="N1069F"></a><a name="sc_miscOps"></a>
 <h3 class="h4">Miscelleaneous ZooKeeper Operations</h3>
 <p></p>
 </div>
 
   
-<a name="N106A3"></a><a name="ch_programStructureWithExample"></a>
+<a name="N106A9"></a><a name="ch_programStructureWithExample"></a>
 <h2 class="h3">Program Structure, with Simple Example</h2>
 <div class="section">
 <p>
@@ -1902,7 +1919,7 @@ authProvider.2=com.f.MyAuth2
 </div>
 
   
-<a name="N106AE"></a><a name="ch_gotchas"></a>
+<a name="N106B4"></a><a name="ch_gotchas"></a>
 <h2 class="h3">Gotchas: Common Problems and Troubleshooting</h2>
 <div class="section">
 <p>So now you know ZooKeeper. It's fast, simple, your application

Những thai đổi đã bị hủy bỏ vì nó quá lớn
+ 5 - 5
docs/zookeeperProgrammers.pdf


+ 1 - 1
docs/zookeeperStarted.html

@@ -518,7 +518,7 @@ numChildren = 0
 <p>ZooKeeper has a Java bindings and C bindings. They are
       functionally equivalent. The C bindings exist in two variants: single
       threaded and multi-threaded. These differ only in how the messaging loop
-      is done. For more information, see the <a href="zookeeperProgrammers.html#ch_programStructureWithExample.html">Programming
+      is done. For more information, see the <a href="zookeeperProgrammers.html#ch_programStructureWithExample">Programming
       Examples in the ZooKeeper Programmer's Guide</a> for
       sample code using of the different APIs.</p>
 <a name="N1013E"></a><a name="sc_RunningReplicatedZooKeeper"></a>

+ 35 - 35
docs/zookeeperStarted.pdf

@@ -315,7 +315,7 @@ endobj
 /Rect [ 371.976 468.387 518.304 456.387 ]
 /C [ 0 0 0 ]
 /Border [ 0 0 0 ]
-/A << /URI (zookeeperProgrammers.html#ch_programStructureWithExample.html)
+/A << /URI (zookeeperProgrammers.html#ch_programStructureWithExample)
 /S /URI >>
 /H /I
 >>
@@ -326,7 +326,7 @@ endobj
 /Rect [ 90.0 455.187 246.12 443.187 ]
 /C [ 0 0 0 ]
 /Border [ 0 0 0 ]
-/A << /URI (zookeeperProgrammers.html#ch_programStructureWithExample.html)
+/A << /URI (zookeeperProgrammers.html#ch_programStructureWithExample)
 /S /URI >>
 /H /I
 >>
@@ -545,31 +545,31 @@ endobj
 xref
 0 68
 0000000000 65535 f 
-0000020805 00000 n 
-0000020905 00000 n 
-0000020997 00000 n 
+0000020795 00000 n 
+0000020895 00000 n 
+0000020987 00000 n 
 0000000015 00000 n 
 0000000071 00000 n 
 0000000933 00000 n 
 0000001053 00000 n 
 0000001134 00000 n 
-0000021142 00000 n 
+0000021132 00000 n 
 0000001269 00000 n 
-0000021205 00000 n 
+0000021195 00000 n 
 0000001406 00000 n 
-0000021271 00000 n 
+0000021261 00000 n 
 0000001543 00000 n 
-0000021337 00000 n 
+0000021327 00000 n 
 0000001680 00000 n 
-0000021402 00000 n 
+0000021392 00000 n 
 0000001817 00000 n 
-0000021467 00000 n 
+0000021457 00000 n 
 0000001954 00000 n 
-0000021533 00000 n 
+0000021523 00000 n 
 0000002091 00000 n 
-0000021598 00000 n 
+0000021588 00000 n 
 0000002227 00000 n 
-0000021664 00000 n 
+0000021654 00000 n 
 0000002364 00000 n 
 0000004805 00000 n 
 0000004928 00000 n 
@@ -592,26 +592,26 @@ xref
 0000014833 00000 n 
 0000014874 00000 n 
 0000015048 00000 n 
-0000015263 00000 n 
-0000015474 00000 n 
-0000017539 00000 n 
-0000021729 00000 n 
-0000017647 00000 n 
-0000018171 00000 n 
-0000018356 00000 n 
-0000018520 00000 n 
-0000018755 00000 n 
-0000019025 00000 n 
-0000019277 00000 n 
-0000019535 00000 n 
-0000019817 00000 n 
-0000020032 00000 n 
-0000020145 00000 n 
-0000020255 00000 n 
-0000020366 00000 n 
-0000020474 00000 n 
-0000020580 00000 n 
-0000020696 00000 n 
+0000015258 00000 n 
+0000015464 00000 n 
+0000017529 00000 n 
+0000021719 00000 n 
+0000017637 00000 n 
+0000018161 00000 n 
+0000018346 00000 n 
+0000018510 00000 n 
+0000018745 00000 n 
+0000019015 00000 n 
+0000019267 00000 n 
+0000019525 00000 n 
+0000019807 00000 n 
+0000020022 00000 n 
+0000020135 00000 n 
+0000020245 00000 n 
+0000020356 00000 n 
+0000020464 00000 n 
+0000020570 00000 n 
+0000020686 00000 n 
 trailer
 <<
 /Size 68
@@ -619,5 +619,5 @@ trailer
 /Info 4 0 R
 >>
 startxref
-21780
+21770
 %%EOF

+ 4 - 1
src/c/src/zk_adaptor.h

@@ -208,6 +208,8 @@ struct _zhandle {
     zk_hashtable* active_node_watchers;   
     zk_hashtable* active_exist_watchers;
     zk_hashtable* active_child_watchers;
+    /** used for chroot path at the client side **/
+    char *chroot;
 };
 
 
@@ -222,7 +224,8 @@ int adaptor_send_queue(zhandle_t *zh, int timeout);
 int process_async(int outstanding_sync);
 void process_completions(zhandle_t *zh);
 int flush_send_queue(zhandle_t*zh, int timeout);
-
+char* sub_string(zhandle_t *zh, const char* server_path);
+void free_duplicate_path(char* free_path, const char* path);
 void zoo_lock_auth(zhandle_t *zh);
 void zoo_unlock_auth(zhandle_t *zh);
 

+ 3 - 1
src/c/src/zk_hashtable.c

@@ -266,10 +266,12 @@ static void add_for_event(zk_hashtable *ht, char *path, watcher_object_list_t **
 static void do_foreach_watcher(watcher_object_t* wo,zhandle_t* zh,
         const char* path,int type,int state)
 {
+    char *client_path = sub_string(zh, path);
     while(wo!=0){
-        wo->watcher(zh,type,state,path,wo->context);
+        wo->watcher(zh,type,state,client_path,wo->context);
         wo=wo->next;
     }    
+    free_duplicate_path(client_path, path);
 }
 
 watcher_object_list_t *collectWatchers(zhandle_t *zh,int type, char *path)

+ 174 - 40
src/c/src/zookeeper.c

@@ -186,6 +186,7 @@ static int disable_conn_permute=0; // permute enabled by default
 static __attribute__((unused)) void print_completion_queue(zhandle_t *zh);
 
 static void *SYNCHRONOUS_MARKER = (void*)&SYNCHRONOUS_MARKER;
+static int isValidPath(const char* path, const int flags);
 
 const void *zoo_get_context(zhandle_t *zh) 
 {
@@ -372,6 +373,12 @@ static void destroy(zhandle_t *zh)
         free(zh->addrs);
         zh->addrs = NULL;
     }
+
+    if (zh->chroot != 0) {
+        free(zh->chroot);
+        zh->chroot = NULL;
+    }
+    
     free_auth_info(&zh->auth_h);
     destroy_zk_hashtable(zh->active_node_watchers);
     destroy_zk_hashtable(zh->active_exist_watchers);
@@ -413,7 +420,7 @@ int getaddrs(zhandle_t *zh)
         zh->addrs = 0;
     }
     if (!hosts) {
-        LOG_ERROR(("out of memory"));
+         LOG_ERROR(("out of memory"));
         errno=ENOMEM;
         return ZSYSTEMERROR;
     }
@@ -607,6 +614,7 @@ zhandle_t *zookeeper_init(const char *host, watcher_fn watcher,
 
     int errnosave;
     zhandle_t *zh = calloc(1, sizeof(*zh));
+    char *index_chroot;
     if (!zh) {
         return 0;
     }
@@ -624,7 +632,28 @@ zhandle_t *zookeeper_init(const char *host, watcher_fn watcher,
         errno=EINVAL;
         goto abort;
     }
-    zh->hostname = strdup(host);
+    //parse the host to get the chroot if 
+    //available
+    index_chroot = strchr(host, '/');
+    if (index_chroot) {
+        zh->chroot = strdup(index_chroot);
+        // if chroot is just / set it to null
+        if (strlen(zh->chroot) == 1) {
+            zh->chroot = NULL;
+        }
+        // cannot use strndup so allocate and strcpy
+        zh->hostname = (char *) malloc(index_chroot - host + 1);
+        zh->hostname = strncpy(zh->hostname, host, (index_chroot - host));
+        //strncpy does not null terminate
+        *(zh->hostname + (index_chroot - host) +1) = '\0';
+        
+    } else {
+        zh->chroot = NULL;
+        zh->hostname = strdup(host);
+    }
+    if (zh->chroot && !isValidPath(zh->chroot, 0)) { 
+        goto abort;
+    }
     if (zh->hostname == 0) {
         goto abort;
     }
@@ -661,6 +690,54 @@ abort:
     return 0;
 }
 
+/**
+ * deallocated the free_path only its beeen allocated
+ * and not equal to path
+ */
+void free_duplicate_path(char *free_path, const char* path) {
+    if (free_path != path) {
+        free(free_path);
+    }
+}
+
+/**
+  prepend the chroot path if available else return the path
+*/
+static char* prepend_string(zhandle_t *zh, const char* client_path) {
+    char *ret_str;
+    if (zh->chroot == NULL) 
+        return (char *) client_path;
+    // handle the chroot itself, client_path = "/"
+    if (strlen(client_path) == 1) { 
+        return strdup(zh->chroot);
+    } 
+    ret_str = (char *) malloc(strlen(zh->chroot) + strlen(client_path) + 1);
+    strcpy(ret_str, zh->chroot);
+    return strcat(ret_str, client_path);
+}
+    
+/**
+   strip off the chroot string from the server path
+   if there is one else return the exact path
+ */
+char* sub_string(zhandle_t *zh, const char* server_path) {
+    char *ret_str;
+    if (zh->chroot == NULL)
+        return (char *) server_path;
+    if (strncmp(server_path, zh->chroot, strlen(zh->chroot) != 0)) {
+        LOG_ERROR(("server path %s does not include chroot path %s", 
+                   server_path, zh->chroot));
+        return NULL;
+    }
+    if (strlen(server_path) == strlen(zh->chroot)) {
+        //return "/"
+        ret_str = strdup("/");
+        return ret_str;
+    }
+    ret_str = strdup(server_path + strlen(zh->chroot));
+    return ret_str;
+} 
+
 static buffer_list_t *allocate_buffer(char *buff, int len)
 {
     buffer_list_t *buffer = calloc(1, sizeof(*buffer));
@@ -1679,7 +1756,6 @@ int zookeeper_process(zhandle_t *zh, int events)
             deserialize_WatcherEvent(ia, "event", &evt);
             type = evt.type;
             path = evt.path;
-
             /* We are doing a notification, so there is no pending request */
             completion_list_t *c = 
                 create_completion_entry(WATCHER_EVENT_XID,-1,0,0,0);
@@ -1847,8 +1923,7 @@ int zookeeper_process(zhandle_t *zh, int events)
     if (process_async(zh->outstanding_sync)) {
         process_completions(zh);
     }
-    return api_epilog(zh,ZOK);
-}
+    return api_epilog(zh,ZOK);}
 
 int zoo_state(zhandle_t *zh)
 {
@@ -2093,24 +2168,29 @@ int zoo_awget(zhandle_t *zh, const char *path,
         data_completion_t dc, const void *data)
 {
     struct oarchive *oa; 
+    char *server_path = prepend_string(zh, path);
     struct RequestHeader h = { .xid = get_xid(), .type = GETDATA_OP};
-    struct GetDataRequest req = { (char*)path, watcher!=0 };
+    struct GetDataRequest req =  { (char*)server_path, watcher!=0 };
     int rc;
-    
-    if (zh==0 || !isValidPath(path, 0))
+            
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    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_GetDataRequest(oa, "req", &req);
     enter_critical(zh);
     rc = rc < 0 ? rc : add_data_completion(zh, h.xid, dc, data,
-        create_watcher_registration(path,data_result_checker,watcher,watcherCtx));
+        create_watcher_registration(server_path,data_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);
     
@@ -2128,13 +2208,19 @@ int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buflen,
     struct RequestHeader h = { .xid = get_xid(), .type = SETDATA_OP};
     struct SetDataRequest req;
     int rc;
+    char *server_path;
+    server_path = prepend_string(zh, path);
     
-    if (zh==0 || !isValidPath(path, 0))
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
         return ZINVALIDSTATE;
+    }
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
+    req.path = (char*)server_path;
     req.data.buff = (char*)buffer;
     req.data.len = buflen;
     req.version = version;
@@ -2145,6 +2231,7 @@ int zoo_aset(zhandle_t *zh, const char *path, const char *buffer, int buflen,
     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);
 
@@ -2163,13 +2250,19 @@ int zoo_acreate(zhandle_t *zh, const char *path, const char *value,
     struct RequestHeader h = { .xid = get_xid(), .type = CREATE_OP };
     struct CreateRequest req;
     int rc;
+    char *server_path;
     
-    if (zh==0 || !isValidPath(path, flags))
+    server_path = prepend_string(zh, path);
+    if (zh==0 || !isValidPath(server_path, flags)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
         return ZINVALIDSTATE;
+    }
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
+    req.path = (char*)server_path;
     req.flags = flags;
     req.data.buff = (char*)value;
     req.data.len = valuelen;
@@ -2186,6 +2279,7 @@ int zoo_acreate(zhandle_t *zh, const char *path, const char *value,
     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);
     
@@ -2203,13 +2297,19 @@ int zoo_adelete(zhandle_t *zh, const char *path, int version,
     struct RequestHeader h = { .xid = get_xid(), .type = DELETE_OP};
     struct DeleteRequest req;
     int rc;
+    char *server_path;
     
-    if (zh==0 || !isValidPath(path, 0))
+    server_path = prepend_string(zh, path);
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
         return ZINVALIDSTATE;
+    }
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
+    req.path = (char*)server_path;
     req.version = version;
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_DeleteRequest(oa, "req", &req);
@@ -2218,6 +2318,7 @@ int zoo_adelete(zhandle_t *zh, const char *path, int version,
     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);
 
@@ -2240,23 +2341,29 @@ int zoo_awexists(zhandle_t *zh, const char *path,
 {
     struct oarchive *oa;
     struct RequestHeader h = { .xid = get_xid(), .type = EXISTS_OP };
-    struct ExistsRequest req = {(char*)path, watcher!=0 };
+    char *server_path = prepend_string(zh, path);
+    struct ExistsRequest req  = {(char*)server_path, watcher!=0 }; 
     int rc;
     
-    if (zh==0 || !isValidPath(path, 0))
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    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_ExistsRequest(oa, "req", &req);
     enter_critical(zh);
     rc = rc < 0 ? rc : add_stat_completion(zh, h.xid, completion, data,
-        create_watcher_registration(path,exists_result_checker,
+        create_watcher_registration(server_path,exists_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);
 
@@ -2279,22 +2386,28 @@ int zoo_awget_children(zhandle_t *zh, const char *path,
 {
     struct oarchive *oa;
     struct RequestHeader h = { .xid = get_xid(), .type = GETCHILDREN_OP};
-    struct GetChildrenRequest req={(char*)path, watcher!=0 };
+    char * server_path = prepend_string(zh, path);
+    struct GetChildrenRequest req = {(char*)server_path, watcher!=0 }; 
     int rc;
-    
-    if (zh==0 || !isValidPath(path, 0))
+        
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    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, dc, data,
-            create_watcher_registration(path,child_result_checker,watcher,watcherCtx));
+            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);
 
@@ -2312,13 +2425,19 @@ int zoo_async(zhandle_t *zh, const char *path,
     struct RequestHeader h = { .xid = get_xid(), .type = SYNC_OP};
     struct SyncRequest req;
     int rc;
-
-    if (zh==0 || !isValidPath(path, 0))
+    char *server_path;
+    
+    server_path = prepend_string(zh, path);
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
         return ZINVALIDSTATE;
+    }
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
+    req.path = (char*)server_path;
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_SyncRequest(oa, "req", &req);
     enter_critical(zh);
@@ -2326,6 +2445,7 @@ int zoo_async(zhandle_t *zh, const char *path,
     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);
 
@@ -2344,13 +2464,19 @@ int zoo_aget_acl(zhandle_t *zh, const char *path, acl_completion_t completion,
     struct RequestHeader h = { .xid = get_xid(), .type = GETACL_OP};
     struct GetACLRequest req;
     int rc;
+    char *server_path;
     
-    if (zh==0 || !isValidPath(path, 0))
+    server_path = prepend_string(zh, path);
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
         return ZINVALIDSTATE;
+    }
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
+    req.path = (char*)server_path;
     rc = serialize_RequestHeader(oa, "header", &h);
     rc = rc < 0 ? rc : serialize_GetACLRequest(oa, "req", &req);
     enter_critical(zh);
@@ -2358,6 +2484,7 @@ int zoo_aget_acl(zhandle_t *zh, const char *path, acl_completion_t completion,
     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);
 
@@ -2375,13 +2502,19 @@ int zoo_aset_acl(zhandle_t *zh, const char *path, int version,
     struct RequestHeader h = { .xid = get_xid(), .type = SETACL_OP};
     struct SetACLRequest req;
     int rc;
-    
-    if (zh==0 || !isValidPath(path, 0))
+    char *server_path;
+
+    server_path = prepend_string(zh, path);
+    if (zh==0 || !isValidPath(server_path, 0)) {
+        free_duplicate_path(server_path, path);
         return ZBADARGUMENTS;
-    if (is_unrecoverable(zh))
+    }
+    if (is_unrecoverable(zh)) {
+        free_duplicate_path(server_path, path);
         return ZINVALIDSTATE;
+    }
     oa = create_buffer_oarchive();
-    req.path = (char*)path;
+    req.path = (char*)server_path;
     req.acl = *acl;
     req.version = version;
     rc = serialize_RequestHeader(oa, "header", &h);
@@ -2391,6 +2524,7 @@ int zoo_aset_acl(zhandle_t *zh, const char *path, int version,
     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);
 

+ 128 - 2
src/c/tests/TestClient.cc

@@ -164,6 +164,7 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture
     CPPUNIT_TEST(testPathValidation);
     CPPUNIT_TEST(testPing);
     CPPUNIT_TEST(testAcl);
+    CPPUNIT_TEST(testChroot);
     CPPUNIT_TEST(testAuth);
     CPPUNIT_TEST(testWatcherAutoResetWithGlobal);
     CPPUNIT_TEST(testWatcherAutoResetWithLocal);
@@ -191,7 +192,7 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture
     const char *getHostPorts() {
         return hostPorts;
     }
-
+    
     zhandle_t *createClient(watchctx_t *ctx) {
         zhandle_t *zk = zookeeper_init(hostPorts, watcher, 10000, 0,
                                        ctx, 0);
@@ -200,6 +201,14 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture
         return zk;
     }
     
+    zhandle_t *createchClient(watchctx_t *ctx) {
+        zhandle_t *zk = zookeeper_init(hp_chroot, watcher, 10000, 0,
+                                       ctx, 0);
+        ctx->zh = zk;
+        sleep(1);
+        return zk;
+    }
+        
 public:
 
 
@@ -273,6 +282,7 @@ public:
     
     static zhandle_t *async_zk;
     static volatile int count;
+    static char* hp_chroot;
     
     static void statCompletion(int rc, const struct Stat *stat, const void *data) {
         int tmp = (int) (long) data;
@@ -293,6 +303,35 @@ public:
         }
     }
     
+    static void create_completion_fn(int rc, const char* value, const void *data) {
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        count++;
+    }
+    
+    static void waitForCreateCompletion(int seconds) {
+        time_t expires = time(0) + seconds;
+        while(count == 0 && time(0) < expires) {
+            sleep(1);
+        }
+        count--;
+    }
+
+    static void watcher_chroot_fn(zhandle_t *zh, int type,
+                                    int state, const char *path,void *watcherCtx) {
+        // check for path
+        char *client_path = (char *) watcherCtx;
+        CPPUNIT_ASSERT(strcmp(client_path, path) == 0);
+        count ++;
+    }
+    
+    static void waitForChrootWatch(int seconds) {
+        time_t expires = time(0) + seconds;
+        while (count == 0 && time(0) < expires) {
+            sleep(1);
+        }
+        count--;
+    }
+
     static void waitForVoidCompletion(int seconds) {
         time_t expires = time(0) + seconds;
         while(count == 0 && time(0) < expires) {
@@ -522,7 +561,93 @@ public:
         verifyCreateOk("/f/.f/f", zk);
         verifyCreateOk("/f/f./f", zk);
     }
-
+    
+    void testChroot() {
+        // the c client async callbacks do 
+        // not callback with the path, so 
+        // we dont need to test taht for now
+        // we should fix that though soon!
+        watchctx_t ctx, ctx_ch;
+        zhandle_t *zk, *zk_ch;
+        char buf[60];
+        int rc, len;
+        struct Stat stat;
+        const char* data = "garbage";
+        const char* retStr = "/chroot"; 
+        const char* root= "/";
+        hp_chroot = "127.0.0.1:22181/test/mahadev";
+        zk_ch = createchClient(&ctx_ch);
+        CPPUNIT_ASSERT(zk_ch != NULL);
+        zk = createClient(&ctx);
+        rc = zoo_create(zk, "/test", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        rc = zoo_create(zk, "/test/mahadev", data, 7, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        // try an exists with /
+        len = 60;
+        rc = zoo_get(zk_ch, "/", 0, buf, &len, &stat);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+        //check if the data is the same
+        CPPUNIT_ASSERT(strncmp(buf, data, 7) == 0);
+        //check for watches 
+        rc = zoo_wexists(zk_ch, "/chroot", watcher_chroot_fn, (void *) retStr, &stat);
+        //now check if we can do create/delete/get/sets/acls/getChildren and others 
+        //check create
+        rc = zoo_create(zk_ch, "/chroot", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0,0);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        waitForChrootWatch(3);
+        CPPUNIT_ASSERT(count == 0);
+        rc = zoo_create(zk_ch, "/chroot/child", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        rc = zoo_exists(zk, "/test/mahadev/chroot/child", 0, &stat);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        
+        rc = zoo_delete(zk_ch, "/chroot/child", -1);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        rc = zoo_exists(zk, "/test/mahadev/chroot/child", 0, &stat);
+        CPPUNIT_ASSERT_EQUAL((int) ZNONODE, rc);
+        rc = zoo_wget(zk_ch, "/chroot", watcher_chroot_fn, (char*) retStr,
+                      buf, &len, &stat);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        rc = zoo_set(zk_ch, "/chroot",buf, 3, -1);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        waitForChrootWatch(3);
+        CPPUNIT_ASSERT(count == 0);
+        // check for getchildren
+        struct String_vector children;
+        rc = zoo_get_children(zk_ch, "/", 0, &children);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)1, children.count);
+        //check if te child if chroot
+        CPPUNIT_ASSERT(strcmp((retStr+1), children.data[0]) == 0);
+        // check for get/set acl
+        struct ACL_vector acl;
+        rc = zoo_get_acl(zk_ch, "/", &acl, &stat);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)1, acl.count);
+        CPPUNIT_ASSERT_EQUAL(ZOO_PERM_ALL, acl.data->perms);
+        // set acl
+        rc = zoo_set_acl(zk_ch, "/chroot", -1,  &ZOO_READ_ACL_UNSAFE);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        // see if you add children
+        rc = zoo_create(zk_ch, "/chroot/child1", "",0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOAUTH, rc);
+        //add wget children test
+        rc = zoo_wget_children(zk_ch, "/", watcher_chroot_fn, (char*) root, &children);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
+        
+        //now create a node
+        rc = zoo_create(zk_ch, "/child2", "",0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
+        CPPUNIT_ASSERT_EQUAL((int) ZOK, rc);
+        waitForChrootWatch(3);
+        CPPUNIT_ASSERT(count == 0);
+        //check for one async call just to make sure
+        rc = zoo_acreate(zk_ch, "/child3", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 
+                         create_completion_fn, 0);
+        waitForCreateCompletion(3);
+        CPPUNIT_ASSERT(count == 0);
+    }
+        
     void testAsyncWatcherAutoReset()
     {
         watchctx_t ctx;
@@ -747,4 +872,5 @@ public:
 volatile int Zookeeper_simpleSystem::count;
 zhandle_t *Zookeeper_simpleSystem::async_zk;
 const char Zookeeper_simpleSystem::hostPorts[] = "127.0.0.1:22181";
+char* Zookeeper_simpleSystem::hp_chroot;
 CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_simpleSystem);

+ 9 - 1
src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml

@@ -1139,7 +1139,15 @@ imok
       <title>Best Practices</title>
 
       <para>For best results, take note of the following list of good
-      Zookeeper practices. <emphasis>[tbd...]</emphasis></para>
+      Zookeeper practices:</para>
+
+
+      <para>For multi-tennant installations see the <ulink
+      url="zookeeperProgrammers.html#ch_zkSessions">section</ulink>
+      detailing ZooKeeper "chroot" support, this can be very useful
+      when deploying many applications/services interfacing to a
+      single ZooKeeper cluster.</para>
+
     </section>
   </section>
 </article>

+ 18 - 1
src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml

@@ -359,7 +359,7 @@
     <title>ZooKeeper Sessions</title>
 
     <para>To create a client session the application code must provide
-    a string containing a comma separated list of host:port pairs,
+    a connection string containing a comma separated list of host:port pairs,
     each corresponding to a ZooKeeper server (e.g. "127.0.0.1:4545" or
     "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002"). The ZooKeeper
     client library will pick an arbitrary server and try to connect to
@@ -368,6 +368,23 @@
     automatically try the next server in the list, until a connection
     is (re-)established.</para>
 
+    <para> <emphasis role="bold">Added in 3.2.0</emphasis>: 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). If used the example would look like:
+    "127.0.0.1:4545/app/a" or
+    "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" where the
+    client would be rooted at "/app/a" and all paths would be relative
+    to this root - ie getting/setting/etc...  "/foo/bar" would result
+    in operations being run on "/app/a/foo/bar" (from the server
+    perspective). This feature is particularly useful in multi-tenant
+    environments where each user of a particular ZooKeeper service
+    could be rooted differently. This makes re-use much simpler as
+    each user can code his/her application as if it were rooted at
+    "/", while actual location (say /app/a) could be determined at
+    deployment time.</para>
+
     <para>When a client gets a handle to the ZooKeeper service,
     ZooKeeper creates a ZooKeeper session, represented as a 64-bit
     number, that it assigns to the client. If the client connects to a

+ 1 - 1
src/docs/src/documentation/content/xdocs/zookeeperStarted.xml

@@ -328,7 +328,7 @@ numChildren = 0
       functionally equivalent. The C bindings exist in two variants: single
       threaded and multi-threaded. These differ only in how the messaging loop
       is done. For more information, see the <ulink
-      url="zookeeperProgrammers.html#ch_programStructureWithExample.html">Programming
+      url="zookeeperProgrammers.html#ch_programStructureWithExample">Programming
       Examples in the ZooKeeper Programmer's Guide</ulink> for
       sample code using of the different APIs.</para>
     </section>

+ 59 - 23
src/java/main/org/apache/zookeeper/ClientCnxn.java

@@ -50,6 +50,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.ZooKeeper.States;
 import org.apache.zookeeper.ZooKeeper.WatchRegistration;
+import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.proto.AuthPacket;
 import org.apache.zookeeper.proto.ConnectRequest;
 import org.apache.zookeeper.proto.ConnectResponse;
@@ -133,6 +134,8 @@ public class ClientCnxn {
 
     private byte sessionPasswd[] = new byte[16];
 
+    final String chrootPath;
+
     final SendThread sendThread;
 
     final EventThread eventThread;
@@ -176,7 +179,10 @@ public class ClientCnxn {
 
         ByteBuffer bb;
 
-        String path;
+        /** Client's view of the path (may differ due to chroot) **/
+        String clientPath;
+        /** Servers's view of the path (may differ due to chroot) **/
+        String serverPath;
 
         ReplyHeader replyHeader;
 
@@ -226,7 +232,8 @@ public class ClientCnxn {
         public String toString() {
             StringBuffer sb = new StringBuffer();
 
-            sb.append("path:" + path);
+            sb.append("clientPath:" + clientPath);
+            sb.append(" serverPath:" + serverPath);
             sb.append(" finished:" + finished);
 
             sb.append(" header:: " + header);
@@ -285,6 +292,23 @@ public class ClientCnxn {
         this.watcher = watcher;
         this.sessionId = sessionId;
         this.sessionPasswd = sessionPasswd;
+
+        // parse out chroot, if any
+        int off = hosts.indexOf('/');
+        if (off >= 0) {
+            String chrootPath = hosts.substring(off);
+            // ignore "/" chroot spec, same as null
+            if (chrootPath.length() == 1) {
+                this.chrootPath = null;
+            } else {
+                PathUtils.validatePath(chrootPath);
+                this.chrootPath = chrootPath;
+            }
+            hosts = hosts.substring(0,  off);
+        } else {
+            this.chrootPath = null;
+        }
+
         String hostsList[] = hosts.split(",");
         for (String host : hostsList) {
             int port = 2181;
@@ -406,7 +430,7 @@ public class ClientCnxn {
                         } else {
                             Packet p = (Packet) event;
                             int rc = 0;
-                            String path = p.path;
+                            String clientPath = p.clientPath;
                             if (p.replyHeader.getErr() != 0) {
                                 rc = p.replyHeader.getErr();
                             }
@@ -418,62 +442,65 @@ public class ClientCnxn {
                                 StatCallback cb = (StatCallback) p.cb;
                                 if (rc == 0) {
                                     if (p.response instanceof ExistsResponse) {
-                                        cb.processResult(rc, path, p.ctx,
+                                        cb.processResult(rc, clientPath, p.ctx,
                                                 ((ExistsResponse) p.response)
                                                         .getStat());
                                     } else if (p.response instanceof SetDataResponse) {
-                                        cb.processResult(rc, path, p.ctx,
+                                        cb.processResult(rc, clientPath, p.ctx,
                                                 ((SetDataResponse) p.response)
                                                         .getStat());
                                     } else if (p.response instanceof SetACLResponse) {
-                                        cb.processResult(rc, path, p.ctx,
+                                        cb.processResult(rc, clientPath, p.ctx,
                                                 ((SetACLResponse) p.response)
                                                         .getStat());
                                     }
                                 } else {
-                                    cb.processResult(rc, path, p.ctx, null);
+                                    cb.processResult(rc, clientPath, p.ctx, null);
                                 }
                             } else if (p.response instanceof GetDataResponse) {
                                 DataCallback cb = (DataCallback) p.cb;
                                 GetDataResponse rsp = (GetDataResponse) p.response;
                                 if (rc == 0) {
-                                    cb.processResult(rc, path, p.ctx, rsp
+                                    cb.processResult(rc, clientPath, p.ctx, rsp
                                             .getData(), rsp.getStat());
                                 } else {
-                                    cb.processResult(rc, path, p.ctx, null,
+                                    cb.processResult(rc, clientPath, p.ctx, null,
                                             null);
                                 }
                             } else if (p.response instanceof GetACLResponse) {
                                 ACLCallback cb = (ACLCallback) p.cb;
                                 GetACLResponse rsp = (GetACLResponse) p.response;
                                 if (rc == 0) {
-                                    cb.processResult(rc, path, p.ctx, rsp
+                                    cb.processResult(rc, clientPath, p.ctx, rsp
                                             .getAcl(), rsp.getStat());
                                 } else {
-                                    cb.processResult(rc, path, p.ctx, null,
+                                    cb.processResult(rc, clientPath, p.ctx, null,
                                             null);
                                 }
                             } else if (p.response instanceof GetChildrenResponse) {
                                 ChildrenCallback cb = (ChildrenCallback) p.cb;
                                 GetChildrenResponse rsp = (GetChildrenResponse) p.response;
                                 if (rc == 0) {
-                                    cb.processResult(rc, path, p.ctx, rsp
+                                    cb.processResult(rc, clientPath, p.ctx, rsp
                                             .getChildren());
                                 } else {
-                                    cb.processResult(rc, path, p.ctx, null);
+                                    cb.processResult(rc, clientPath, p.ctx, null);
                                 }
                             } else if (p.response instanceof CreateResponse) {
                                 StringCallback cb = (StringCallback) p.cb;
                                 CreateResponse rsp = (CreateResponse) p.response;
                                 if (rc == 0) {
-                                    cb.processResult(rc, path, p.ctx, rsp
-                                            .getPath());
+                                    cb.processResult(rc, clientPath, p.ctx,
+                                            (chrootPath == null
+                                                    ? rsp.getPath()
+                                                    : rsp.getPath()
+                                              .substring(chrootPath.length())));
                                 } else {
-                                    cb.processResult(rc, path, p.ctx, null);
+                                    cb.processResult(rc, clientPath, p.ctx, null);
                                 }
                             } else if (p.cb instanceof VoidCallback) {
                                 VoidCallback cb = (VoidCallback) p.cb;
-                                cb.processResult(rc, path, p.ctx);
+                                cb.processResult(rc, clientPath, p.ctx);
                             }
                         }
                     } catch (Throwable t) {
@@ -601,6 +628,13 @@ public class ClientCnxn {
                     + Long.toHexString(sessionId));
                 WatcherEvent event = new WatcherEvent();
                 event.deserialize(bbia, "response");
+
+                // convert from a server path to a client path
+                if (chrootPath != null) {
+                    String serverPath = event.getPath();
+                    event.setPath(serverPath.substring(chrootPath.length()));
+                }
+
                 WatchedEvent we = new WatchedEvent(event);
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Got " + we + " for sessionid 0x"
@@ -791,7 +825,7 @@ public class ClientCnxn {
         private void sendPing() {
             lastPingSentNs = System.nanoTime();
             RequestHeader h = new RequestHeader(-2, OpCode.ping);
-            queuePacket(h, null, null, null, null, null, null, null);
+            queuePacket(h, null, null, null, null, null, null, null, null);
         }
 
         int lastConnectIndex = -1;
@@ -1059,7 +1093,7 @@ public class ClientCnxn {
             throws InterruptedException {
         ReplyHeader r = new ReplyHeader();
         Packet packet = queuePacket(h, r, request, response, null, null, null,
-                    watchRegistration);
+                    null, watchRegistration);
         synchronized (packet) {
             while (!packet.finished) {
                 packet.wait();
@@ -1069,8 +1103,9 @@ public class ClientCnxn {
     }
 
     Packet queuePacket(RequestHeader h, ReplyHeader r, Record request,
-            Record response, AsyncCallback cb, String path, Object ctx,
-            WatchRegistration watchRegistration) {
+            Record response, AsyncCallback cb, String clientPath,
+            String serverPath, Object ctx, WatchRegistration watchRegistration)
+    {
         Packet packet = null;
         synchronized (outgoingQueue) {
             if (h.getType() != OpCode.ping && h.getType() != OpCode.auth) {
@@ -1080,7 +1115,8 @@ public class ClientCnxn {
                     watchRegistration);
             packet.cb = cb;
             packet.ctx = ctx;
-            packet.path = path;
+            packet.clientPath = clientPath;
+            packet.serverPath = serverPath;
             if (!zooKeeper.state.isAlive()) {
                 conLossPacket(packet);
             } else {
@@ -1098,7 +1134,7 @@ public class ClientCnxn {
         if (zooKeeper.state == States.CONNECTED) {
             queuePacket(new RequestHeader(-4, OpCode.auth), null,
                     new AuthPacket(0, scheme, auth), null, null, null, null,
-                    null);
+                    null, null);
         }
     }
 }

+ 302 - 157
src/java/main/org/apache/zookeeper/ZooKeeper.java

@@ -148,7 +148,9 @@ public class ZooKeeper {
          * @see org.apache.zookeeper.ClientWatchManager#materialize(Event.KeeperState, Event.EventType, java.lang.String)
          */
         public Set<Watcher> materialize(Watcher.Event.KeeperState state,
-                                        Watcher.Event.EventType type, String path) {
+                                        Watcher.Event.EventType type,
+                                        String clientPath)
+        {
             Set<Watcher> result = new HashSet<Watcher>();
 
             switch (type) {
@@ -183,36 +185,36 @@ public class ZooKeeper {
             case NodeDataChanged:
             case NodeCreated:
                 synchronized (dataWatches) {
-                    addTo(dataWatches.remove(path), result);
+                    addTo(dataWatches.remove(clientPath), result);
                 }
                 synchronized (existWatches) {
-                    addTo(existWatches.remove(path), result);
+                    addTo(existWatches.remove(clientPath), result);
                 }
                 break;
             case NodeChildrenChanged:
                 synchronized (childWatches) {
-                    addTo(childWatches.remove(path), result);
+                    addTo(childWatches.remove(clientPath), result);
                 }
                 break;
             case NodeDeleted:
                 synchronized (dataWatches) {
-                    addTo(dataWatches.remove(path), result);
+                    addTo(dataWatches.remove(clientPath), result);
                 }
                 // XXX This shouldn't be needed, but just in case
                 synchronized (existWatches) {
-                    Set<Watcher> list = existWatches.remove(path);
+                    Set<Watcher> list = existWatches.remove(clientPath);
                     if (list != null) {
-                        addTo(existWatches.remove(path), result);
+                        addTo(existWatches.remove(clientPath), result);
                         LOG.warn("We are triggering an exists watch for delete! Shouldn't happen!");
                     }
                 }
                 synchronized (childWatches) {
-                    addTo(childWatches.remove(path), result);
+                    addTo(childWatches.remove(clientPath), result);
                 }
                 break;
             default:
                 String msg = "Unhandled watch event type " + type
-                    + " with state " + state + " on path " + path;
+                    + " with state " + state + " on path " + clientPath;
                 LOG.error(msg);
                 throw new RuntimeException(msg);
             }
@@ -226,11 +228,11 @@ public class ZooKeeper {
      */
     abstract class WatchRegistration {
         private Watcher watcher;
-        private String path;
-        public WatchRegistration(Watcher watcher, String path)
+        private String clientPath;
+        public WatchRegistration(Watcher watcher, String clientPath)
         {
             this.watcher = watcher;
-            this.path = path;
+            this.clientPath = clientPath;
         }
 
         abstract protected Map<String, Set<Watcher>> getWatches(int rc);
@@ -244,10 +246,10 @@ public class ZooKeeper {
             if (shouldAddWatch(rc)) {
                 Map<String, Set<Watcher>> watches = getWatches(rc);
                 synchronized(watches) {
-                    Set<Watcher> watchers = watches.get(path);
+                    Set<Watcher> watchers = watches.get(clientPath);
                     if (watchers == null) {
                         watchers = new HashSet<Watcher>();
-                        watches.put(path, watchers);
+                        watches.put(clientPath, watchers);
                     }
                     watchers.add(watcher);
                 }
@@ -268,8 +270,8 @@ public class ZooKeeper {
      * even in the case where NONODE result code is returned.
      */
     class ExistsWatchRegistration extends WatchRegistration {
-        public ExistsWatchRegistration(Watcher watcher, String path) {
-            super(watcher, path);
+        public ExistsWatchRegistration(Watcher watcher, String clientPath) {
+            super(watcher, clientPath);
         }
 
         @Override
@@ -284,8 +286,8 @@ public class ZooKeeper {
     }
 
     class DataWatchRegistration extends WatchRegistration {
-        public DataWatchRegistration(Watcher watcher, String path) {
-            super(watcher, path);
+        public DataWatchRegistration(Watcher watcher, String clientPath) {
+            super(watcher, clientPath);
         }
 
         @Override
@@ -295,8 +297,8 @@ public class ZooKeeper {
     }
 
     class ChildWatchRegistration extends WatchRegistration {
-        public ChildWatchRegistration(Watcher watcher, String path) {
-            super(watcher, path);
+        public ChildWatchRegistration(Watcher watcher, String clientPath) {
+            super(watcher, clientPath);
         }
 
         @Override
@@ -318,17 +320,27 @@ public class ZooKeeper {
     protected final ClientCnxn cnxn;
 
     /**
-     * To create a client(ZooKeeper) object, the application needs to pass a
-     * string containing a comma separated list of host:port pairs, each 
-     * corresponding to a ZooKeeper server.
+     * To create a ZooKeeper client object, the application needs to pass a
+     * connection string containing a comma separated list of host:port pairs,
+     * each corresponding to a ZooKeeper server.
      * <p>
      * The client object will pick an arbitrary server and try to connect to it.
      * If failed, it will try the next one in the list, until a connection is
      * established, or all the servers have been tried.
+     * <p>
+     * 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 host
+     * @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 the optional chroot suffix is used the example would look
+     *            like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a"
+     *            where the client would be rooted at "/app/a" and all paths
+     *            would be relative to this root - ie getting/setting/etc...
+     *            "/foo/bar" would result in operations being run on
+     *            "/app/a/foo/bar" (from the server perspective).
      * @param sessionTimeout
      *            session timeout in milliseconds
      * @param watcher
@@ -336,35 +348,47 @@ public class ZooKeeper {
      *            also be notified for node events
      *
      * @throws IOException in cases of network failure
+     * @throws IllegalArgumentException if an invalid chroot path is specified
      */
-    public ZooKeeper(String host, int sessionTimeout, Watcher watcher)
-            throws IOException {
-        LOG.info("Initiating client connection, host=" + host
+    public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher)
+        throws IOException
+    {
+        LOG.info("Initiating client connection, connectString=" + connectString
                 + " sessionTimeout=" + sessionTimeout + " watcher=" + watcher);
 
         watchManager.defaultWatcher = watcher;
-        cnxn = new ClientCnxn(host, sessionTimeout, this, watchManager);
+        cnxn = new ClientCnxn(connectString, sessionTimeout, this, watchManager);
         cnxn.start();
     }
 
     /**
-     * To create a client(ZooKeeper) object, the application needs to pass a
-     * string containing a comma separated list of host:port pairs, each
-     * corresponding to a ZooKeeper server.
+     * To create a ZooKeeper client object, the application needs to pass a
+     * connection string containing a comma separated list of host:port pairs,
+     * each corresponding to a ZooKeeper server.
      * <p>
      * The client object will pick an arbitrary server and try to connect to it.
      * If failed, it will try the next one in the list, until a connection is
      * established, or all the servers have been tried.
      * <p>
+     * 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).
+     * <p>
      * Use {@link #getSessionId} and {@link #getSessionPasswd} on an established
      * client connection, these values must be passed as sessionId and
      * sessionPasswd respectively if reconnecting. Otherwise, if not
      * reconnecting, use the other constructor which does not require these
      * parameters.
      *
-     * @param host
+     * @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 the optional chroot suffix is used the example would look
+     *            like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a"
+     *            where the client would be rooted at "/app/a" and all paths
+     *            would be relative to this root - ie getting/setting/etc...
+     *            "/foo/bar" would result in operations being run on
+     *            "/app/a/foo/bar" (from the server perspective).
      * @param sessionTimeout
      *            session timeout in milliseconds
      * @param watcher
@@ -376,10 +400,13 @@ public class ZooKeeper {
      *            password for this session
      *
      * @throws IOException in cases of network failure
+     * @throws IllegalArgumentException if an invalid chroot path is specified
      */
-    public ZooKeeper(String host, int sessionTimeout, Watcher watcher,
-            long sessionId, byte[] sessionPasswd) throws IOException {
-        LOG.info("Initiating client connection, host=" + host
+    public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher,
+            long sessionId, byte[] sessionPasswd)
+        throws IOException
+    {
+        LOG.info("Initiating client connection, connectString=" + connectString
                 + " sessionTimeout=" + sessionTimeout
                 + " watcher=" + watcher
                 + " sessionId=" + sessionId
@@ -387,7 +414,7 @@ public class ZooKeeper {
                 + (sessionPasswd == null ? "<null>" : "<hidden>"));
 
         watchManager.defaultWatcher = watcher;
-        cnxn = new ClientCnxn(host, sessionTimeout, this, watchManager,
+        cnxn = new ClientCnxn(connectString, sessionTimeout, this, watchManager,
                 sessionId, sessionPasswd);
         cnxn.start();
     }
@@ -429,9 +456,6 @@ public class ZooKeeper {
      * their parents) will be triggered.
      *
      * @throws InterruptedException
-     *
-     * @throws IOException
-     * @throws InterruptedException
      */
     public synchronized void close() throws InterruptedException {
         LOG.info("Closing session: 0x" + Long.toHexString(getSessionId()));
@@ -445,6 +469,25 @@ public class ZooKeeper {
         LOG.info("Session: 0x" + Long.toHexString(getSessionId()) + " closed");
     }
 
+    /**
+     * Prepend the chroot to the client path (if present). The expectation of
+     * this function is that the client path has been validated before this
+     * function is called
+     * @param clientPath path to the node
+     * @return server view of the path (chroot prepended to client path)
+     */
+    private String prependChroot(String clientPath) {
+        if (cnxn.chrootPath != null) {
+            // handle clientPath = "/"
+            if (clientPath.length() == 1) {
+                return cnxn.chrootPath;
+            }
+            return cnxn.chrootPath + clientPath;
+        } else {
+            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.
@@ -495,16 +538,18 @@ public class ZooKeeper {
      *                and/or sequential
      * @return the actual path of the created node
      * @throws KeeperException if the server returns a non-zero error code
-     * @throws org.apache.zookeeper.KeeperException.InvalidACLException if the ACL is invalid
+     * @throws KeeperException.InvalidACLException if the ACL is invalid
      * @throws InterruptedException if the transaction is interrupted
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public String create(String path, byte data[], List<ACL> acl,
+    public String create(final String path, byte data[], List<ACL> acl,
             CreateMode createMode)
         throws KeeperException, InterruptedException
     {
-        // also handle the case where server will append name suffix
-        PathUtils.validatePath(path, createMode.isSequential());
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath, createMode.isSequential());
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.create);
@@ -512,7 +557,7 @@ public class ZooKeeper {
         CreateResponse response = new CreateResponse();
         request.setData(data);
         request.setFlags(createMode.toFlag());
-        request.setPath(path);
+        request.setPath(serverPath);
         if (acl != null && acl.size() == 0) {
             throw new KeeperException.InvalidACLException();
         }
@@ -520,9 +565,13 @@ public class ZooKeeper {
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
+        }
+        if (cnxn.chrootPath == null) {
+            return response.getPath();
+        } else {
+            return response.getPath().substring(cnxn.chrootPath.length());
         }
-        return response.getPath();
     }
 
     /**
@@ -532,11 +581,13 @@ public class ZooKeeper {
      * @see #create(String, byte[], List, CreateMode)
      */
 
-    public void create(String path, byte data[], List<ACL> acl,
+    public void create(final String path, byte data[], List<ACL> acl,
             CreateMode createMode,  StringCallback cb, Object ctx)
     {
-        // also handle the case where server will append name suffix
-        PathUtils.validatePath(path, createMode.isSequential());
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath, createMode.isSequential());
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.create);
@@ -545,9 +596,10 @@ public class ZooKeeper {
         ReplyHeader r = new ReplyHeader();
         request.setData(data);
         request.setFlags(createMode.toFlag());
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setAcl(acl);
-        cnxn.queuePacket(h, r, request, response, cb, path, ctx, null);
+        cnxn.queuePacket(h, r, request, response, cb, clientPath,
+                serverPath, ctx, null);
     }
 
     /**
@@ -573,22 +625,38 @@ public class ZooKeeper {
      * @param version
      *                the expected node version.
      * @throws InterruptedException IF the server transaction is interrupted
-     * @throws KeeperException If the server signals an error with a non-zero return code.
+     * @throws KeeperException If the server signals an error with a non-zero
+     *   return code.
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public void delete(String path, int version) throws
-            InterruptedException, KeeperException {
-        PathUtils.validatePath(path);
+    public void delete(final String path, int version)
+        throws InterruptedException, KeeperException
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+        
+        final String serverPath;
+
+        // maintain semantics even in chroot case
+        // specifically - root cannot be deleted
+        // I think this makes sense even in chroot case.
+        if (clientPath.equals("/")) {
+            // a bit of a hack, but delete(/) will never succeed and ensures
+            // that the same semantics are maintained
+            serverPath = clientPath;
+        } else {
+            serverPath = prependChroot(clientPath);
+        }
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.delete);
         DeleteRequest request = new DeleteRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setVersion(version);
         ReplyHeader r = cnxn.submitRequest(h, request, null, null);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
     }
 
@@ -598,15 +666,32 @@ public class ZooKeeper {
      *
      * @see #delete(String, int)
      */
-    public void delete(String path, int version, VoidCallback cb, Object ctx) {
-        PathUtils.validatePath(path);
+    public void delete(final String path, int version, VoidCallback cb,
+            Object ctx)
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath;
+
+        // maintain semantics even in chroot case
+        // specifically - root cannot be deleted
+        // I think this makes sense even in chroot case.
+        if (clientPath.equals("/")) {
+            // a bit of a hack, but delete(/) will never succeed and ensures
+            // that the same semantics are maintained
+            serverPath = clientPath;
+        } else {
+            serverPath = prependChroot(clientPath);
+        }
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.delete);
         DeleteRequest request = new DeleteRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setVersion(version);
-        cnxn.queuePacket(h, new ReplyHeader(), request, null, cb, path, ctx, null);
+        cnxn.queuePacket(h, new ReplyHeader(), request, null, cb, clientPath,
+                serverPath, ctx, null);
     }
 
     /**
@@ -626,28 +711,33 @@ public class ZooKeeper {
      * @throws InterruptedException If the server transaction is interrupted.
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public Stat exists(String path, Watcher watcher) throws KeeperException,
-        InterruptedException
+    public Stat exists(final String path, Watcher watcher)
+        throws KeeperException, InterruptedException
     {
-        PathUtils.validatePath(path);
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        // the watch contains the un-chroot path
+        WatchRegistration wcb = null;
+        if (watcher != null) {
+            wcb = new ExistsWatchRegistration(watcher, clientPath);
+        }
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.exists);
         ExistsRequest request = new ExistsRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setWatch(watcher != null);
         SetDataResponse response = new SetDataResponse();
-        WatchRegistration wcb = null;
-        if (watcher != null) {
-            wcb = new ExistsWatchRegistration(watcher, path);
-        }
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         if (r.getErr() != 0) {
             if (r.getErr() == KeeperException.Code.NONODE.intValue()) {
                 return null;
             }
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
 
         return response.getStat().getCzxid() == -1 ? null : response.getStat();
@@ -683,23 +773,28 @@ public class ZooKeeper {
      *
      * @see #exists(String, boolean)
      */
-    public void exists(String path, Watcher watcher, StatCallback cb,
-            Object ctx)
+    public void exists(final String path, Watcher watcher,
+            StatCallback cb, Object ctx)
     {
-        PathUtils.validatePath(path);
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        // the watch contains the un-chroot path
+        WatchRegistration wcb = null;
+        if (watcher != null) {
+            wcb = new ExistsWatchRegistration(watcher, clientPath);
+        }
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.exists);
         ExistsRequest request = new ExistsRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setWatch(watcher != null);
         SetDataResponse response = new SetDataResponse();
-        WatchRegistration wcb = null;
-        if (watcher != null) {
-            wcb = new ExistsWatchRegistration(watcher, path);
-        }
-        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx, wcb);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, wcb);
     }
 
     /**
@@ -731,24 +826,30 @@ public class ZooKeeper {
      * @throws InterruptedException If the server transaction is interrupted.
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public byte[] getData(String path, Watcher watcher, Stat stat)
-            throws KeeperException, InterruptedException {
-        PathUtils.validatePath(path);
+    public byte[] getData(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 DataWatchRegistration(watcher, clientPath);
+        }
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.getData);
         GetDataRequest request = new GetDataRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setWatch(watcher != null);
         GetDataResponse response = new GetDataResponse();
-        WatchRegistration wcb = null;
-        if (watcher != null) {
-            wcb = new DataWatchRegistration(watcher, path);
-        }
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
         if (stat != null) {
             DataTree.copyStat(response.getStat(), stat);
@@ -785,21 +886,28 @@ public class ZooKeeper {
      *
      * @see #getData(String, Watcher, Stat)
      */
-    public void getData(String path, Watcher watcher, DataCallback cb, Object ctx) {
-        PathUtils.validatePath(path);
+    public void getData(final String path, Watcher watcher,
+            DataCallback 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 DataWatchRegistration(watcher, clientPath);
+        }
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.getData);
         GetDataRequest request = new GetDataRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setWatch(watcher != null);
         GetDataResponse response = new GetDataResponse();
-        WatchRegistration wcb = null;
-        if (watcher != null) {
-            wcb = new DataWatchRegistration(watcher, path);
-        }
-        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx, wcb);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, wcb);
     }
 
     /**
@@ -840,21 +948,25 @@ public class ZooKeeper {
      * @throws KeeperException If the server signals an error with a non-zero error code.
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public Stat setData(String path, byte data[], int version)
-            throws KeeperException, InterruptedException {
-        PathUtils.validatePath(path);
+    public Stat setData(final String path, byte data[], int version)
+        throws KeeperException, InterruptedException
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.setData);
         SetDataRequest request = new SetDataRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setData(data);
         request.setVersion(version);
         SetDataResponse response = new SetDataResponse();
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
         return response.getStat();
     }
@@ -865,20 +977,23 @@ public class ZooKeeper {
      *
      * @see #setData(String, byte[], int)
      */
-    public void setData(String path, byte data[], int version, StatCallback cb,
-            Object ctx) {
-        PathUtils.validatePath(path);
+    public void setData(final String path, byte data[], int version,
+            StatCallback cb, Object ctx)
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.setData);
         SetDataRequest request = new SetDataRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setData(data);
         request.setVersion(version);
         SetDataResponse response = new SetDataResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx, null);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, null);
     }
 
     /**
@@ -896,19 +1011,23 @@ public class ZooKeeper {
      * @throws KeeperException If the server signals an error with a non-zero error code.
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public List<ACL> getACL(String path, Stat stat)
-            throws KeeperException, InterruptedException {
-        PathUtils.validatePath(path);
+    public List<ACL> getACL(final String path, Stat stat)
+        throws KeeperException, InterruptedException
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.getACL);
         GetACLRequest request = new GetACLRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         GetACLResponse response = new GetACLResponse();
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
         DataTree.copyStat(response.getStat(), stat);
         return response.getAcl();
@@ -920,17 +1039,21 @@ public class ZooKeeper {
      *
      * @see #getACL(String, Stat)
      */
-    public void getACL(String path, Stat stat, ACLCallback cb, Object ctx) {
-        PathUtils.validatePath(path);
+    public void getACL(final String path, Stat stat, ACLCallback cb,
+            Object ctx)
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.getACL);
         GetACLRequest request = new GetACLRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         GetACLResponse response = new GetACLResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx, null);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, null);
     }
 
     /**
@@ -953,14 +1076,18 @@ public class ZooKeeper {
      * @throws org.apache.zookeeper.KeeperException.InvalidACLException If the acl is invalide.
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public Stat setACL(String path, List<ACL> acl, int version)
-            throws KeeperException, InterruptedException {
-        PathUtils.validatePath(path);
+    public Stat setACL(final String path, List<ACL> acl, int version)
+        throws KeeperException, InterruptedException
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.setACL);
         SetACLRequest request = new SetACLRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         if (acl != null && acl.size() == 0) {
             throw new KeeperException.InvalidACLException();
         }
@@ -970,7 +1097,7 @@ public class ZooKeeper {
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
         return response.getStat();
     }
@@ -981,20 +1108,23 @@ public class ZooKeeper {
      *
      * @see #setACL(String, List, int)
      */
-    public void setACL(String path, List<ACL> acl, int version,
-            StatCallback cb, Object ctx) {
-        PathUtils.validatePath(path);
+    public void setACL(final String path, List<ACL> acl, int version,
+            StatCallback cb, Object ctx)
+    {
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.setACL);
         SetACLRequest request = new SetACLRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setAcl(acl);
         request.setVersion(version);
         SetACLResponse response = new SetACLResponse();
-        cnxn
-                .queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx, null);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, null);
     }
 
     /**
@@ -1018,24 +1148,30 @@ public class ZooKeeper {
      * @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(String path, Watcher watcher)
-            throws KeeperException, InterruptedException {
-        PathUtils.validatePath(path);
+    public List<String> getChildren(final String path, Watcher watcher)
+        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.getChildren);
         GetChildrenRequest request = new GetChildrenRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setWatch(watcher != null);
         GetChildrenResponse response = new GetChildrenResponse();
-        WatchRegistration wcb = null;
-        if (watcher != null) {
-            wcb = new ChildWatchRegistration(watcher, path);
-        }
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         if (r.getErr() != 0) {
             throw KeeperException.create(KeeperException.Code.get(r.getErr()),
-                    path);
+                    clientPath);
         }
         return response.getChildren();
     }
@@ -1071,22 +1207,28 @@ public class ZooKeeper {
      *
      * @see #getChildren(String, Watcher)
      */
-    public void getChildren(String path, Watcher watcher, ChildrenCallback cb,
-            Object ctx) {
-        PathUtils.validatePath(path);
+    public void getChildren(final String path, Watcher watcher,
+            ChildrenCallback 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.getChildren);
         GetChildrenRequest request = new GetChildrenRequest();
-        request.setPath(path);
+        request.setPath(serverPath);
         request.setWatch(watcher != null);
         GetChildrenResponse response = new GetChildrenResponse();
-        WatchRegistration wcb = null;
-        if (watcher != null) {
-            wcb = new ChildWatchRegistration(watcher, path);
-        }
-        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path,
-                        ctx, wcb);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, wcb);
     }
 
     /**
@@ -1107,16 +1249,19 @@ public class ZooKeeper {
      * @param ctx context to be provided to the callback
      * @throws IllegalArgumentException if an invalid path is specified
      */
-    public void sync(String path, VoidCallback cb, Object ctx){
-        PathUtils.validatePath(path);
+    public void sync(final String path, VoidCallback cb, Object ctx){
+        final String clientPath = path;
+        PathUtils.validatePath(clientPath);
+
+        final String serverPath = prependChroot(clientPath);
 
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.sync);
         SyncRequest request = new SyncRequest();
         SyncResponse response = new SyncResponse();
-        request.setPath(path);
-        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb, path, ctx,
-                null);
+        request.setPath(serverPath);
+        cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
+                clientPath, serverPath, ctx, null);
     }
 
     public States getState() {

+ 48 - 0
src/java/test/org/apache/zookeeper/test/ChrootAsyncTest.java

@@ -0,0 +1,48 @@
+/**
+ * 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 org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.junit.Before;
+
+public class ChrootAsyncTest extends AsyncOpsTest {
+    private static final Logger LOG = Logger.getLogger(ChrootAsyncTest.class);
+
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        String hp = hostPort;
+        hostPort = hostPort + "/chrootasynctest";
+
+        super.setUp();
+
+        LOG.info("STARTING " + getName());
+
+        ZooKeeper zk = createClient(hp);
+        try {
+            zk.create("/chrootasynctest", null, Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } finally {
+            zk.close();
+        }
+    }
+}

+ 48 - 0
src/java/test/org/apache/zookeeper/test/ChrootClientTest.java

@@ -0,0 +1,48 @@
+/**
+ * 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 org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.junit.Before;
+
+public class ChrootClientTest extends ClientTest {
+    private static final Logger LOG = Logger.getLogger(ChrootClientTest.class);
+
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        String hp = hostPort;
+        hostPort = hostPort + "/chrootclienttest";
+
+        super.setUp();
+
+        LOG.info("STARTING " + getName());
+
+        ZooKeeper zk = createClient(hp);
+        try {
+            zk.create("/chrootclienttest", null, Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } finally {
+            zk.close();
+        }
+    }
+}

+ 132 - 0
src/java/test/org/apache/zookeeper/test/ChrootTest.java

@@ -0,0 +1,132 @@
+/**
+ * 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.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.test.AsyncOps.ACLCB;
+import org.apache.zookeeper.test.AsyncOps.ChildrenCB;
+import org.apache.zookeeper.test.AsyncOps.DataCB;
+import org.apache.zookeeper.test.AsyncOps.StatCB;
+import org.apache.zookeeper.test.AsyncOps.StringCB;
+import org.apache.zookeeper.test.AsyncOps.VoidCB;
+import org.junit.Test;
+
+public class ChrootTest extends ClientBase {
+    private class MyWatcher implements Watcher {
+        private final String path;
+        private String eventPath;
+        private CountDownLatch latch = new CountDownLatch(1);
+
+        public MyWatcher(String path) {
+            this.path = path;
+        }
+        public void process(WatchedEvent event) {
+            System.out.println("latch:" + path + " " + event.getPath());
+            this.eventPath = event.getPath();
+            latch.countDown();
+        }
+        public boolean matches() throws InterruptedException {
+            if (!latch.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)) {
+                fail("No watch received within timeout period " + path);
+            }
+            return path.equals(eventPath);
+        }
+    }
+
+    public void testChrootSynchronous()
+        throws IOException, InterruptedException, KeeperException
+    {
+        ZooKeeper zk1 = createClient();
+        try {
+            zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } finally {
+            if(zk1 != null)
+                zk1.close();
+        }
+        ZooKeeper zk2 = createClient(hostPort + "/ch1");
+        try {
+            assertEquals("/ch2",
+                    zk2.create("/ch2", null, Ids.OPEN_ACL_UNSAFE,
+                            CreateMode.PERSISTENT));
+        } finally {
+            if(zk2 != null)
+                zk2.close();
+        }
+
+        zk1 = createClient();
+        zk2 = createClient(hostPort + "/ch1");
+        try {
+            // check get
+            MyWatcher w1 = new MyWatcher("/ch1");
+            assertNotNull(zk1.exists("/ch1", w1));
+            MyWatcher w2 = new MyWatcher("/ch1/ch2");
+            assertNotNull(zk1.exists("/ch1/ch2", w2));
+
+            MyWatcher w3 = new MyWatcher("/ch2");
+            assertNotNull(zk2.exists("/ch2", w3));
+
+            // check set
+            zk1.setData("/ch1", "1".getBytes(), -1);
+            zk2.setData("/ch2", "2".getBytes(), -1);
+
+            // check watches
+            assertTrue(w1.matches());
+            assertTrue(w2.matches());
+            assertTrue(w3.matches());
+
+            // check exceptions
+            try {
+                zk2.setData("/ch3", "3".getBytes(), -1);
+            } catch (KeeperException.NoNodeException e) {
+                assertEquals("/ch3", e.getPath());
+            }
+
+            assertTrue(Arrays.equals("1".getBytes(),
+                    zk1.getData("/ch1", false, null)));
+            assertTrue(Arrays.equals("2".getBytes(),
+                    zk1.getData("/ch1/ch2", false, null)));
+            assertTrue(Arrays.equals("2".getBytes(),
+                    zk2.getData("/ch2", false, null)));
+
+            // check delete
+            zk2.delete("/ch2", -1);
+            zk1.delete("/ch1", -1);
+            assertNull(zk1.exists("/ch1", false));
+            assertNull(zk1.exists("/ch1/ch2", false));
+            assertNull(zk2.exists("/ch2", false));
+        } finally {
+            if(zk1 != null)
+                zk1.close();
+            if(zk2 != null)
+                zk2.close();
+        }
+    }
+}

+ 11 - 3
src/java/test/org/apache/zookeeper/test/ClientBase.java

@@ -254,13 +254,21 @@ public abstract class ClientBase extends TestCase {
         
         return tmpDir;
     }
-        
+    private static int getPort(String hostPort) {
+        String portstr = hostPort.split(":")[1];
+        String[] pc = portstr.split("/");
+        if (pc.length > 1) {
+            portstr = pc[0];
+        }
+        return Integer.parseInt(portstr);
+    }
+    
     static NIOServerCnxn.Factory createNewServerInstance(File dataDir,
             NIOServerCnxn.Factory factory, String hostPort, int maxCnxns)
         throws IOException, InterruptedException 
     {
         ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
-        final int PORT = Integer.parseInt(hostPort.split(":")[1]);
+        final int PORT = getPort(hostPort);
         if (factory == null) {
             factory = new NIOServerCnxn.Factory(PORT,maxCnxns);
         }
@@ -278,7 +286,7 @@ public abstract class ClientBase extends TestCase {
     {
         if (factory != null) {
             factory.shutdown();
-            final int PORT = Integer.parseInt(hostPort.split(":")[1]);
+            final int PORT = getPort(hostPort);
 
             assertTrue("waiting for server down",
                        ClientBase.waitForServerDown("127.0.0.1:" + PORT,

+ 31 - 24
src/java/test/org/apache/zookeeper/test/ClientTest.java

@@ -287,9 +287,9 @@ public class ClientTest extends ClientBase {
         try {
             MyWatcher watcher = new MyWatcher();
             zk = createClient(watcher, hostPort);
-            //LOG.info("Created client: " + zk.describeCNXN());
             LOG.info("Before create /benwashere");
-            zk.create("/benwashere", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/benwashere", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
             LOG.info("After create /benwashere");
             try {
                 zk.setData("/benwashere", "hi".getBytes(), 57);
@@ -301,7 +301,7 @@ public class ClientTest extends ClientBase {
             }
             LOG.info("Before delete /benwashere");
             zk.delete("/benwashere", 0);
-            LOG.info("Before delete /benwashere");
+            LOG.info("After delete /benwashere");
             zk.close();
             //LOG.info("Closed client: " + zk.describeCNXN());
             Thread.sleep(2000);
@@ -318,13 +318,16 @@ public class ClientTest extends ClientBase {
             }
             Stat stat = new Stat();
             // Test basic create, ls, and getData
+            zk.create("/pat", "Pat was here".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
             LOG.info("Before create /ben");
-            zk.create("/ben", "Ben was here".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            LOG.info("Before getChildren /");
-            List<String> children = zk.getChildren("/", false);
-            assertEquals(2, children.size());
-            assertEquals("ben", children.get(1));
-            String value = new String(zk.getData("/ben", false, stat));
+            zk.create("/pat/ben", "Ben was here".getBytes(),
+                    Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            LOG.info("Before getChildren /pat");
+            List<String> children = zk.getChildren("/pat", false);
+            assertEquals(1, children.size());
+            assertEquals("ben", children.get(0));
+            String value = new String(zk.getData("/pat/ben", false, stat));
             assertEquals("Ben was here", value);
             // Test stat and watch of non existent node
 
@@ -348,12 +351,12 @@ public class ClientTest extends ClientBase {
             assertEquals(EventType.NodeCreated, event.getType());
             assertEquals(KeeperState.SyncConnected, event.getState());
             // Test child watch and create with sequence
-            zk.getChildren("/ben", true);
+            zk.getChildren("/pat/ben", true);
             for (int i = 0; i < 10; i++) {
-                zk.create("/ben/" + i + "-", Integer.toString(i).getBytes(),
+                zk.create("/pat/ben/" + i + "-", Integer.toString(i).getBytes(),
                         Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
             }
-            children = zk.getChildren("/ben", false);
+            children = zk.getChildren("/pat/ben", false);
             Collections.sort(children);
             assertEquals(10, children.size());
             for (int i = 0; i < 10; i++) {
@@ -361,39 +364,43 @@ public class ClientTest extends ClientBase {
                 assertTrue("starts with -", name.startsWith(i + "-"));
                 byte b[];
                 if (withWatcherObj) {
-                    b = zk.getData("/ben/" + name, watcher, stat);
+                    b = zk.getData("/pat/ben/" + name, watcher, stat);
                 } else {
-                    b = zk.getData("/ben/" + name, true, stat);
+                    b = zk.getData("/pat/ben/" + name, true, stat);
                 }
                 assertEquals(Integer.toString(i), new String(b));
-                zk.setData("/ben/" + name, "new".getBytes(), stat.getVersion());
+                zk.setData("/pat/ben/" + name, "new".getBytes(),
+                        stat.getVersion());
                 if (withWatcherObj) {
-                    stat = zk.exists("/ben/" + name, watcher);
+                    stat = zk.exists("/pat/ben/" + name, watcher);
                 } else {
-                stat = zk.exists("/ben/" + name, true);
+                stat = zk.exists("/pat/ben/" + name, true);
                 }
-                zk.delete("/ben/" + name, stat.getVersion());
+                zk.delete("/pat/ben/" + name, stat.getVersion());
             }
             event = watcher.events.poll(10, TimeUnit.SECONDS);
-            assertEquals("/ben", event.getPath());
+            assertEquals("/pat/ben", event.getPath());
             assertEquals(EventType.NodeChildrenChanged, event.getType());
             assertEquals(KeeperState.SyncConnected, event.getState());
             for (int i = 0; i < 10; i++) {
                 event = watcher.events.poll(10, TimeUnit.SECONDS);
                 final String name = children.get(i);
-                assertEquals("/ben/" + name, event.getPath());
+                assertEquals("/pat/ben/" + name, event.getPath());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
                 event = watcher.events.poll(10, TimeUnit.SECONDS);
-                assertEquals("/ben/" + name, event.getPath());
+                assertEquals("/pat/ben/" + name, event.getPath());
                 assertEquals(EventType.NodeDeleted, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
             }
-            zk.create("/good\u0040path", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/good\u0040path", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
 
-            zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
             try {
-                zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+                zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
                 fail("duplicate create allowed");
             } catch(KeeperException.NodeExistsException e) {
                 // OK, expected that

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác