Kaynağa Gözat

ZOOKEEPER-246. review error code definition in both source and docs. (pat via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@728812 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 16 yıl önce
ebeveyn
işleme
88fab67e8b
25 değiştirilmiş dosya ile 695 ekleme ve 393 silme
  1. 3 0
      CHANGES.txt
  2. 38 23
      src/c/include/zookeeper.h
  3. 26 26
      src/c/tests/TestClient.cc
  4. 40 40
      src/c/tests/TestOperations.cc
  5. 35 35
      src/c/tests/TestWatchers.cc
  6. 13 13
      src/c/tests/TestZookeeperClose.cc
  7. 6 0
      src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
  8. 5 4
      src/java/main/org/apache/zookeeper/ClientCnxn.java
  9. 383 122
      src/java/main/org/apache/zookeeper/KeeperException.java
  10. 11 10
      src/java/main/org/apache/zookeeper/ZooKeeper.java
  11. 2 1
      src/java/main/org/apache/zookeeper/server/DataTree.java
  12. 12 10
      src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
  13. 4 3
      src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
  14. 2 2
      src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
  15. 2 1
      src/java/main/org/apache/zookeeper/server/auth/AuthenticationProvider.java
  16. 5 4
      src/java/main/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java
  17. 4 2
      src/java/main/org/apache/zookeeper/server/auth/HostAuthenticationProvider.java
  18. 4 2
      src/java/main/org/apache/zookeeper/server/auth/IPAuthenticationProvider.java
  19. 2 1
      src/java/main/org/apache/zookeeper/server/upgrade/DataTreeV1.java
  20. 19 15
      src/java/test/org/apache/zookeeper/test/AsyncOps.java
  21. 4 4
      src/java/test/org/apache/zookeeper/test/AsyncTest.java
  22. 39 39
      src/java/test/org/apache/zookeeper/test/ClientTest.java
  23. 2 2
      src/java/test/org/apache/zookeeper/test/CreateModeTest.java
  24. 10 10
      src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java
  25. 24 24
      src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java

+ 3 - 0
CHANGES.txt

@@ -84,6 +84,9 @@ detailing connection parameters. (pat via mahadev)
 
 
    ZOOKEEPER-222.  print C client log message timestamp in human readable
    ZOOKEEPER-222.  print C client log message timestamp in human readable
 form. (pat via mahadev) 
 form. (pat via mahadev) 
+  
+   ZOOKEEPER-246. review error code definition in both source and docs. (pat
+via mahadev)
 
 
 Release 3.0.0 - 2008-10-21
 Release 3.0.0 - 2008-10-21
 
 

+ 38 - 23
src/c/include/zookeeper.h

@@ -69,29 +69,43 @@
 
 
 /** zookeeper return constants **/
 /** zookeeper return constants **/
 
 
-#define ZOK                                    0
-#define ZSYSTEMERROR                          -1
-#define ZRUNTIMEINCONSISTENCY (ZSYSTEMERROR - 1)
-#define ZDATAINCONSISTENCY    (ZSYSTEMERROR - 2)
-#define ZCONNECTIONLOSS       (ZSYSTEMERROR - 3)
-#define ZMARSHALLINGERROR     (ZSYSTEMERROR - 4)
-#define ZUNIMPLEMENTED        (ZSYSTEMERROR - 5)
-#define ZOPERATIONTIMEOUT     (ZSYSTEMERROR - 6)
-#define ZBADARGUMENTS         (ZSYSTEMERROR - 7)
-#define ZINVALIDSTATE         (ZSYSTEMERROR - 8)
-#define ZAPIERROR                           -100
-#define ZNONODE                  (ZAPIERROR - 1)
-#define ZNOAUTH                  (ZAPIERROR - 2)
-#define ZBADVERSION              (ZAPIERROR - 3)
-#define ZNOCHILDRENFOREPHEMERALS (ZAPIERROR - 8)
-#define ZNODEEXISTS             (ZAPIERROR - 10)
-#define ZNOTEMPTY               (ZAPIERROR - 11)
-#define ZSESSIONEXPIRED         (ZAPIERROR - 12)
-#define ZINVALIDCALLBACK        (ZAPIERROR - 13)
-#define ZINVALIDACL             (ZAPIERROR - 14)
-#define ZAUTHFAILED             (ZAPIERROR - 15)
-#define ZCLOSING                (ZAPIERROR - 16)
-#define ZNOTHING                (ZAPIERROR - 17)
+enum ZOO_ERRORS {
+  ZOK = 0, /*!< Everything is OK */
+
+  /** System and server-side errors.
+   * This is never thrown by the server, it shouldn't be used other than
+   * to indicate a range. Specifically error codes greater than this
+   * value, but lesser than {@link #ZAPIERROR}, are system errors. */
+  ZSYSTEMERROR = -1,
+  ZRUNTIMEINCONSISTENCY = -2, /*!< A runtime inconsistency was found */
+  ZDATAINCONSISTENCY = -3, /*!< A data inconsistency was found */
+  ZCONNECTIONLOSS = -4, /*!< Connection to the server has been lost */
+  ZMARSHALLINGERROR = -5, /*!< Error while marshalling or unmarshalling data */
+  ZUNIMPLEMENTED = -6, /*!< Operation is unimplemented */
+  ZOPERATIONTIMEOUT = -7, /*!< Operation timeout */
+  ZBADARGUMENTS = -8, /*!< Invalid arguments */
+  ZINVALIDSTATE = -9, /*!< Invliad zhandle state */
+
+  /** API errors.
+   * This is never thrown by the server, it shouldn't be used other than
+   * to indicate a range. Specifically error codes greater than this
+   * value are API errors (while values less than this indicate a 
+   * {@link #ZSYSTEMERROR}).
+   */
+  ZAPIERROR = -100,
+  ZNONODE = -101, /*!< Node does not exist */
+  ZNOAUTH = -102, /*!< Not authenticated */
+  ZBADVERSION = -103, /*!< Version conflict */
+  ZNOCHILDRENFOREPHEMERALS = -108, /*!< Ephemeral nodes may not have children */
+  ZNODEEXISTS = -110, /*!< The node already exists */
+  ZNOTEMPTY = -111, /*!< The node has children */
+  ZSESSIONEXPIRED = -112, /*!< The session has been expired by the server */
+  ZINVALIDCALLBACK = -113, /*!< Invalid callback specified */
+  ZINVALIDACL = -114, /*!< Invalid ACL specified */
+  ZAUTHFAILED = -115, /*!< Client authentication failed */
+  ZCLOSING = -116, /*!< ZooKeeper is closing */
+  ZNOTHING = -117 /*!< (not error) no server responses to process */
+};
 
 
 #ifdef __cplusplus
 #ifdef __cplusplus
 extern "C" {
 extern "C" {
@@ -101,6 +115,7 @@ extern "C" {
 *  @name Debug levels
 *  @name Debug levels
 */
 */
 typedef enum {ZOO_LOG_LEVEL_ERROR=1,ZOO_LOG_LEVEL_WARN=2,ZOO_LOG_LEVEL_INFO=3,ZOO_LOG_LEVEL_DEBUG=4} ZooLogLevel;
 typedef enum {ZOO_LOG_LEVEL_ERROR=1,ZOO_LOG_LEVEL_WARN=2,ZOO_LOG_LEVEL_INFO=3,ZOO_LOG_LEVEL_DEBUG=4} ZooLogLevel;
+
 /**
 /**
  * @name ACL Consts
  * @name ACL Consts
  */
  */

+ 26 - 26
src/c/tests/TestClient.cc

@@ -235,24 +235,24 @@ public:
         for(int i = 0; i < 30; i++) {
         for(int i = 0; i < 30; i++) {
             sprintf(path, "/%i", i);
             sprintf(path, "/%i", i);
             rc = zoo_create(zkWatchCreator, path, "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
             rc = zoo_create(zkWatchCreator, path, "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
 
 
         for(int i = 0; i < 30; i++) {
         for(int i = 0; i < 30; i++) {
             sprintf(path, "/%i", i);
             sprintf(path, "/%i", i);
             struct Stat stat;
             struct Stat stat;
             rc = zoo_exists(zkIdle, path, 1, &stat);
             rc = zoo_exists(zkIdle, path, 1, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
 
 
         for(int i = 0; i < 30; i++) {
         for(int i = 0; i < 30; i++) {
             sprintf(path, "/%i", i);
             sprintf(path, "/%i", i);
             usleep(500000);
             usleep(500000);
             rc = zoo_delete(zkWatchCreator, path, -1);
             rc = zoo_delete(zkWatchCreator, path, -1);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
         struct Stat stat;
         struct Stat stat;
-        CPPUNIT_ASSERT_EQUAL(ZNONODE, zoo_exists(zkIdle, "/0", 0, &stat));
+        CPPUNIT_ASSERT_EQUAL((int)ZNONODE, zoo_exists(zkIdle, "/0", 0, &stat));
     }
     }
 
 
     bool waitForEvent(zhandle_t *zh, watchctx_t *ctx, int seconds) {
     bool waitForEvent(zhandle_t *zh, watchctx_t *ctx, int seconds) {
@@ -299,7 +299,7 @@ public:
         for(i = 0; i < COUNT; i++) {
         for(i = 0; i < COUNT; i++) {
             sprintf(path, "/%d", i);
             sprintf(path, "/%d", i);
             rc = zoo_awexists(zk, path, watcher, &lctx[i], statCompletion, (void*)ZNONODE);
             rc = zoo_awexists(zk, path, watcher, &lctx[i], statCompletion, (void*)ZNONODE);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
 
 
         yield(zk, 0);
         yield(zk, 0);
@@ -307,7 +307,7 @@ public:
         for(i = 0; i < COUNT/2; i++) {
         for(i = 0; i < COUNT/2; i++) {
             sprintf(path, "/%d", i);
             sprintf(path, "/%d", i);
             rc = zoo_acreate(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0, stringCompletion, strdup(path));
             rc = zoo_acreate(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0, stringCompletion, strdup(path));
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
 
 
         yield(zk, 3);
         yield(zk, 3);
@@ -322,7 +322,7 @@ public:
         for(i = COUNT/2 + 1; i < COUNT*10; i++) {
         for(i = COUNT/2 + 1; i < COUNT*10; i++) {
             sprintf(path, "/%d", i);
             sprintf(path, "/%d", i);
             rc = zoo_acreate(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0, stringCompletion, strdup(path));
             rc = zoo_acreate(zk, path, "", 0,  &ZOO_OPEN_ACL_UNSAFE, 0, stringCompletion, strdup(path));
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
 
 
         yield(zk, 1);
         yield(zk, 1);
@@ -353,33 +353,33 @@ public:
 
 
         rc = zoo_create(zk, "/watchtest", "", 0, 
         rc = zoo_create(zk, "/watchtest", "", 0, 
                         &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
                         &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         rc = zoo_create(zk, "/watchtest/child", "", 0,
         rc = zoo_create(zk, "/watchtest/child", "", 0,
                         &ZOO_OPEN_ACL_UNSAFE, ZOO_EPHEMERAL, 0, 0);
                         &ZOO_OPEN_ACL_UNSAFE, ZOO_EPHEMERAL, 0, 0);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         if (isGlobal) {
         if (isGlobal) {
             testName = "GlobalTest";
             testName = "GlobalTest";
             rc = zoo_get_children(zk, "/watchtest", 1, &strings);
             rc = zoo_get_children(zk, "/watchtest", 1, &strings);
             deallocate_String_vector(&strings);
             deallocate_String_vector(&strings);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             blen = sizeof(buf);
             blen = sizeof(buf);
             rc = zoo_get(zk, "/watchtest/child", 1, buf, &blen, &stat);
             rc = zoo_get(zk, "/watchtest/child", 1, buf, &blen, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             rc = zoo_exists(zk, "/watchtest/child2", 1, &stat);
             rc = zoo_exists(zk, "/watchtest/child2", 1, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZNONODE, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZNONODE, rc);
         } else {
         } else {
             testName = "LocalTest";
             testName = "LocalTest";
             rc = zoo_wget_children(zk, "/watchtest", watcher, ctxLocal,
             rc = zoo_wget_children(zk, "/watchtest", watcher, ctxLocal,
                                  &strings);
                                  &strings);
             deallocate_String_vector(&strings);
             deallocate_String_vector(&strings);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             blen = sizeof(buf);
             blen = sizeof(buf);
             rc = zoo_wget(zk, "/watchtest/child", watcher, ctxLocal,
             rc = zoo_wget(zk, "/watchtest/child", watcher, ctxLocal,
                          buf, &blen, &stat);
                          buf, &blen, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             rc = zoo_wexists(zk, "/watchtest/child2", watcher, ctxLocal,
             rc = zoo_wexists(zk, "/watchtest/child2", watcher, ctxLocal,
                             &stat);
                             &stat);
-            CPPUNIT_ASSERT_EQUAL(ZNONODE, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZNONODE, rc);
         }
         }
         
         
         CPPUNIT_ASSERT(ctxLocal->countEvents() == 0);
         CPPUNIT_ASSERT(ctxLocal->countEvents() == 0);
@@ -392,18 +392,18 @@ public:
         CPPUNIT_ASSERT(ctxLocal->countEvents() == 0);
         CPPUNIT_ASSERT(ctxLocal->countEvents() == 0);
 
 
         rc = zoo_set(zk, "/watchtest/child", "1", 1, -1, 0);
         rc = zoo_set(zk, "/watchtest/child", "1", 1, -1, 0);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         struct Stat stat1, stat2;
         struct Stat stat1, stat2;
         rc = zoo_set(zk, "/watchtest/child", "1", 1, -1, &stat1);
         rc = zoo_set(zk, "/watchtest/child", "1", 1, -1, &stat1);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         CPPUNIT_ASSERT(stat1.version >= 0);
         CPPUNIT_ASSERT(stat1.version >= 0);
         rc = zoo_set(zk, "/watchtest/child", "1", 1, stat1.version, &stat2);
         rc = zoo_set(zk, "/watchtest/child", "1", 1, stat1.version, &stat2);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         rc = zoo_set(zk, "/watchtest/child", "1", 1, stat2.version, 0);
         rc = zoo_set(zk, "/watchtest/child", "1", 1, stat2.version, 0);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         rc = zoo_create(zk, "/watchtest/child2", "", 0,
         rc = zoo_create(zk, "/watchtest/child2", "", 0,
                         &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
                         &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0);
-        CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
 
 
         CPPUNIT_ASSERT_MESSAGE(testName, waitForEvent(zk, ctxLocal, 5));
         CPPUNIT_ASSERT_MESSAGE(testName, waitForEvent(zk, ctxLocal, 5));
         
         
@@ -434,25 +434,25 @@ public:
             testName = "GlobalTest";
             testName = "GlobalTest";
             rc = zoo_get_children(zk, "/watchtest", 1, &strings);
             rc = zoo_get_children(zk, "/watchtest", 1, &strings);
             deallocate_String_vector(&strings);
             deallocate_String_vector(&strings);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             blen = sizeof(buf);
             blen = sizeof(buf);
             rc = zoo_get(zk, "/watchtest/child", 1, buf, &blen, &stat);
             rc = zoo_get(zk, "/watchtest/child", 1, buf, &blen, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             rc = zoo_exists(zk, "/watchtest/child2", 1, &stat);
             rc = zoo_exists(zk, "/watchtest/child2", 1, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         } else {
         } else {
             testName = "LocalTest";
             testName = "LocalTest";
             rc = zoo_wget_children(zk, "/watchtest", watcher, ctxLocal,
             rc = zoo_wget_children(zk, "/watchtest", watcher, ctxLocal,
                                  &strings);
                                  &strings);
             deallocate_String_vector(&strings);
             deallocate_String_vector(&strings);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             blen = sizeof(buf);
             blen = sizeof(buf);
             rc = zoo_wget(zk, "/watchtest/child", watcher, ctxLocal,
             rc = zoo_wget(zk, "/watchtest/child", watcher, ctxLocal,
                          buf, &blen, &stat);
                          buf, &blen, &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
             rc = zoo_wexists(zk, "/watchtest/child2", watcher, ctxLocal,
             rc = zoo_wexists(zk, "/watchtest/child2", watcher, ctxLocal,
                             &stat);
                             &stat);
-            CPPUNIT_ASSERT_EQUAL(ZOK, rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK, rc);
         }
         }
 
 
         zoo_delete(zk, "/watchtest/child2", -1);
         zoo_delete(zk, "/watchtest/child2", -1);

+ 40 - 40
src/c/tests/TestOperations.cc

@@ -99,22 +99,22 @@ public:
         AsyncGetOperationCompletion res1;
         AsyncGetOperationCompletion res1;
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // second operation
         // second operation
         AsyncGetOperationCompletion res2;
         AsyncGetOperationCompletion res2;
         zkServer.addOperationResponse(new ZooGetResponse("2",1));
         zkServer.addOperationResponse(new ZooGetResponse("2",1));
         rc=zoo_aget(zh,"/x/y/2",0,asyncCompletion,&res2);
         rc=zoo_aget(zh,"/x/y/2",0,asyncCompletion,&res2);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // process the send queue
         // process the send queue
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         while((rc=zookeeper_process(zh,interest))==ZOK) printf("%d\n", rc);
         while((rc=zookeeper_process(zh,interest))==ZOK) printf("%d\n", rc);
 	printf("RC = %d", rc);
 	printf("RC = %d", rc);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
-        CPPUNIT_ASSERT_EQUAL(ZOK,res1.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,res1.rc_);
         CPPUNIT_ASSERT_EQUAL(string("1"),res1.value_);
         CPPUNIT_ASSERT_EQUAL(string("1"),res1.value_);
-        CPPUNIT_ASSERT_EQUAL(ZOK,res2.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,res2.rc_);
         CPPUNIT_ASSERT_EQUAL(string("2"),res2.value_);
         CPPUNIT_ASSERT_EQUAL(string("2"),res2.value_);
     }
     }
     // send two getData requests and disconnect while the second request is
     // send two getData requests and disconnect while the second request is
@@ -139,25 +139,25 @@ public:
         AsyncGetOperationCompletion res1;
         AsyncGetOperationCompletion res1;
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // second operation
         // second operation
         AsyncGetOperationCompletion res2;
         AsyncGetOperationCompletion res2;
         zkServer.addOperationResponse(new ZooGetResponse("2",1));
         zkServer.addOperationResponse(new ZooGetResponse("2",1));
         rc=zoo_aget(zh,"/x/y/2",0,asyncCompletion,&res2);
         rc=zoo_aget(zh,"/x/y/2",0,asyncCompletion,&res2);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // process the send queue
         // process the send queue
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // simulate a disconnect
         // simulate a disconnect
         zkServer.setConnectionLost();
         zkServer.setConnectionLost();
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
-        CPPUNIT_ASSERT_EQUAL(ZOK,res1.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZCONNECTIONLOSS,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,res1.rc_);
         CPPUNIT_ASSERT_EQUAL(string("1"),res1.value_);
         CPPUNIT_ASSERT_EQUAL(string("1"),res1.value_);
-        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,res2.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZCONNECTIONLOSS,res2.rc_);
         CPPUNIT_ASSERT_EQUAL(string(""),res2.value_);
         CPPUNIT_ASSERT_EQUAL(string(""),res2.value_);
     }
     }
     // send two getData requests and simulate timeout while the both request
     // send two getData requests and simulate timeout while the both request
@@ -182,20 +182,20 @@ public:
         AsyncGetOperationCompletion res1;
         AsyncGetOperationCompletion res1;
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // second operation
         // second operation
         AsyncGetOperationCompletion res2;
         AsyncGetOperationCompletion res2;
         zkServer.addOperationResponse(new ZooGetResponse("2",1));
         zkServer.addOperationResponse(new ZooGetResponse("2",1));
         rc=zoo_aget(zh,"/x/y/2",0,asyncCompletion,&res2);
         rc=zoo_aget(zh,"/x/y/2",0,asyncCompletion,&res2);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // simulate timeout
         // simulate timeout
         timeMock.tick(+10); // advance system time by 10 secs
         timeMock.tick(+10); // advance system time by 10 secs
         // the next call to zookeeper_interest should return ZOPERATIONTIMEOUT
         // the next call to zookeeper_interest should return ZOPERATIONTIMEOUT
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOPERATIONTIMEOUT,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOPERATIONTIMEOUT,rc);
         // make sure the completions have been called
         // make sure the completions have been called
-        CPPUNIT_ASSERT_EQUAL(ZOPERATIONTIMEOUT,res1.rc_);
-        CPPUNIT_ASSERT_EQUAL(ZOPERATIONTIMEOUT,res2.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZOPERATIONTIMEOUT,res1.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZOPERATIONTIMEOUT,res2.rc_);
     }
     }
 
 
     class PingCountingServer: public ZookeeperServer{
     class PingCountingServer: public ZookeeperServer{
@@ -231,24 +231,24 @@ public:
         timeval tv;
         timeval tv;
         // Round 1.
         // Round 1.
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // simulate waiting for the select() call to timeout; 
         // simulate waiting for the select() call to timeout; 
         // advance the system clock accordingly
         // advance the system clock accordingly
         timeMock.tick(tv);  
         timeMock.tick(tv);  
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
         // verify no ping sent
         // verify no ping sent
         CPPUNIT_ASSERT(zkServer.pingCount_==0);
         CPPUNIT_ASSERT(zkServer.pingCount_==0);
         
         
         // Round 2.
         // Round 2.
         // the client should have the idle threshold exceeded, by now
         // the client should have the idle threshold exceeded, by now
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // assume the socket is writable, so no idling here; move on to 
         // assume the socket is writable, so no idling here; move on to 
         // zookeeper_process immediately
         // zookeeper_process immediately
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
         // ZNOTHING means the client hasn't received a ping response yet
         // ZNOTHING means the client hasn't received a ping response yet
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
         // verify a ping is sent
         // verify a ping is sent
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);
         
         
@@ -257,11 +257,11 @@ public:
         // that the client has updated its last_recv timestamp 
         // that the client has updated its last_recv timestamp 
         zkServer.addRecvResponse(new PingResponse);
         zkServer.addRecvResponse(new PingResponse);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // pseudo-sleep for a short while (10 ms)
         // pseudo-sleep for a short while (10 ms)
         timeMock.millitick(10);
         timeMock.millitick(10);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // only one ping so far?
         // only one ping so far?
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);
         CPPUNIT_ASSERT(timeMock==zh->last_recv);
         CPPUNIT_ASSERT(timeMock==zh->last_recv);
@@ -288,7 +288,7 @@ public:
         timeval tv;
         timeval tv;
         // Round 1.
         // Round 1.
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // simulate waiting for the select() call to timeout; 
         // simulate waiting for the select() call to timeout; 
         // advance the system clock accordingly
         // advance the system clock accordingly
         timeMock.tick(tv);
         timeMock.tick(tv);
@@ -296,7 +296,7 @@ public:
         // trigger a watch now
         // trigger a watch now
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/x/y/z"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/x/y/z"));
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // arrival of a watch sets the last_recv to the current time
         // arrival of a watch sets the last_recv to the current time
         CPPUNIT_ASSERT(timeMock==zh->last_recv);
         CPPUNIT_ASSERT(timeMock==zh->last_recv);
         // spend 1 millisecond by processing the watch
         // spend 1 millisecond by processing the watch
@@ -305,10 +305,10 @@ public:
         // Round 2.
         // Round 2.
         // a ping is due; zookeeper_interest() must send it now
         // a ping is due; zookeeper_interest() must send it now
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // no delay here -- as if the socket is immediately writable
         // no delay here -- as if the socket is immediately writable
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
         // verify a ping is sent
         // verify a ping is sent
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);        
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);        
     }
     }
@@ -341,29 +341,29 @@ public:
         // send the queued up zoo_aget() request
         // send the queued up zoo_aget() request
         Mock_gettimeofday beginningOfTimes(now); // remember when we started
         Mock_gettimeofday beginningOfTimes(now); // remember when we started
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // no delay -- the socket is writable
         // no delay -- the socket is writable
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc); // ZNOTHING -- no response yet
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc); // ZNOTHING -- no response yet
         
         
         // Round 2.
         // Round 2.
         // what's next?
         // what's next?
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // no response from the server yet -- waiting in the select() call
         // no response from the server yet -- waiting in the select() call
         now.tick(tv);
         now.tick(tv);
         // a watch has arrived, thus preventing the connection from timing out 
         // a watch has arrived, thus preventing the connection from timing out 
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/x/y/z"));        
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/x/y/z"));        
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc); // read the watch message
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc); // read the watch message
         CPPUNIT_ASSERT_EQUAL(0,zkServer.pingCount_); // not yet!
         CPPUNIT_ASSERT_EQUAL(0,zkServer.pingCount_); // not yet!
         
         
         //Round 3.
         //Round 3.
         // now is the time to send a ping; make sure it's actually sent
         // now is the time to send a ping; make sure it's actually sent
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
         // verify a ping is sent
         // verify a ping is sent
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);
         CPPUNIT_ASSERT_EQUAL(1,zkServer.pingCount_);
         // make sure only 1/3 of the timeout has passed
         // make sure only 1/3 of the timeout has passed
@@ -407,7 +407,7 @@ public:
             return new TestConcurrentOpJob(svr_,zh_);
             return new TestConcurrentOpJob(svr_,zh_);
         }
         }
         virtual void validate(const char* file, int line) const{
         virtual void validate(const char* file, int line) const{
-            CPPUNIT_ASSERT_EQUAL_MESSAGE_LOC("ZOK != rc",ZOK,rc_,file,line);
+            CPPUNIT_ASSERT_EQUAL_MESSAGE_LOC("ZOK != rc",(int)ZOK,rc_,file,line);
         }
         }
     };
     };
     void testConcurrentOperations1()
     void testConcurrentOperations1()
@@ -454,7 +454,7 @@ public:
             //TEST_TRACE(("Finished %d iterations",i));
             //TEST_TRACE(("Finished %d iterations",i));
         }
         }
         virtual void validate(const char* file, int line) const{
         virtual void validate(const char* file, int line) const{
-            CPPUNIT_ASSERT_EQUAL_MESSAGE_LOC("ZOK != rc",ZOK,rc_,file,line);
+            CPPUNIT_ASSERT_EQUAL_MESSAGE_LOC("ZOK != rc",(int)ZOK,rc_,file,line);
         }
         }
         zhandle_t* zh_;
         zhandle_t* zh_;
         int rc_;
         int rc_;
@@ -514,7 +514,7 @@ public:
             return new TestConcurrentOpWithDisconnectJob(svr_,zh_);
             return new TestConcurrentOpWithDisconnectJob(svr_,zh_);
         }
         }
         virtual void validate(const char* file, int line) const{
         virtual void validate(const char* file, int line) const{
-            CPPUNIT_ASSERT_EQUAL_MESSAGE_LOC("ZCONNECTIONLOSS != rc",ZCONNECTIONLOSS,rc_,file,line);
+            CPPUNIT_ASSERT_EQUAL_MESSAGE_LOC("ZCONNECTIONLOSS != rc",(int)ZCONNECTIONLOSS,rc_,file,line);
         }
         }
     };
     };
 
 
@@ -570,10 +570,10 @@ public:
         AsyncGetOperationCompletion res1;
         AsyncGetOperationCompletion res1;
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&res1);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         CPPUNIT_ASSERT(ensureCondition(res1,1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(res1,1000)<1000);
-        CPPUNIT_ASSERT_EQUAL(ZOK,res1.rc_);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,res1.rc_);
         CPPUNIT_ASSERT_EQUAL(string("1"),res1.value_);        
         CPPUNIT_ASSERT_EQUAL(string("1"),res1.value_);        
     }
     }
     class ChangeNodeWatcher: public WatcherAction{
     class ChangeNodeWatcher: public WatcherAction{
@@ -625,7 +625,7 @@ public:
         // prepare a response for the zoo_aexists() request
         // prepare a response for the zoo_aexists() request
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_aexists(zh,"/x/y/z",1,asyncCompletion,&completion);
         int rc=zoo_aexists(zh,"/x/y/z",1,asyncCompletion,&completion);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         CPPUNIT_ASSERT(ensureCondition(action.isNodeChangedTriggered(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(action.isNodeChangedTriggered(),1000)<1000);
         CPPUNIT_ASSERT_EQUAL(string("/x/y/z"),action.path_);                
         CPPUNIT_ASSERT_EQUAL(string("/x/y/z"),action.path_);                

+ 35 - 35
src/c/tests/TestWatchers.cc

@@ -136,15 +136,15 @@ public:
         timeval tv;
         timeval tv;
         // open the socket
         // open the socket
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);        
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);        
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTING_STATE,zoo_state(zh));
         // send the handshake packet to the server
         // send the handshake packet to the server
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_ASSOCIATING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_ASSOCIATING_STATE,zoo_state(zh));
         // receive the server handshake response
         // receive the server handshake response
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // verify connected
         // verify connected
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTED_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTED_STATE,zoo_state(zh));
         CPPUNIT_ASSERT(watcher.connected_);
         CPPUNIT_ASSERT(watcher.connected_);
@@ -170,15 +170,15 @@ public:
         AsyncCompletion ignored;
         AsyncCompletion ignored;
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         zkServer.addOperationResponse(new ZooGetResponse("1",1));
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&ignored);
         int rc=zoo_aget(zh,"/x/y/1",0,asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // this will process the response and activate the watcher
         // this will process the response and activate the watcher
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         // now, disconnect
         // now, disconnect
         zkServer.setConnectionLost();
         zkServer.setConnectionLost();
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
-        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZCONNECTIONLOSS,rc);
         // verify disconnected
         // verify disconnected
         CPPUNIT_ASSERT(watcher.disconnected_);
         CPPUNIT_ASSERT(watcher.disconnected_);
         CPPUNIT_ASSERT_EQUAL(1,watcher.counter_);
         CPPUNIT_ASSERT_EQUAL(1,watcher.counter_);
@@ -205,15 +205,15 @@ public:
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_awexists(zh,"/x/y/1",activeWatcher,&wobject,
         int rc=zoo_awexists(zh,"/x/y/1",activeWatcher,&wobject,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // this will process the response and activate the watcher
         // this will process the response and activate the watcher
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         // now, disconnect
         // now, disconnect
         zkServer.setConnectionLost();
         zkServer.setConnectionLost();
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
-        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZCONNECTIONLOSS,rc);
 
 
         // verify the default watcher has been triggered
         // verify the default watcher has been triggered
         CPPUNIT_ASSERT(defWatcher.disconnected_);
         CPPUNIT_ASSERT(defWatcher.disconnected_);
@@ -246,22 +246,22 @@ public:
         AsyncCompletion ignored;
         AsyncCompletion ignored;
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_aexists(zh,"/a/b/c",1,asyncCompletion,&ignored);
         int rc=zoo_aexists(zh,"/a/b/c",1,asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         CountingDataWatcher wobject;
         CountingDataWatcher wobject;
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         // this will process the response and activate the watcher
         // this will process the response and activate the watcher
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
         // disconnect now
         // disconnect now
         zkServer.setConnectionLost();
         zkServer.setConnectionLost();
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
         rc=zookeeper_process(zh,ZOOKEEPER_READ);
-        CPPUNIT_ASSERT_EQUAL(ZCONNECTIONLOSS,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZCONNECTIONLOSS,rc);
 
 
         // verify the default watcher has been triggered
         // verify the default watcher has been triggered
         CPPUNIT_ASSERT(defWatcher.disconnected_);
         CPPUNIT_ASSERT(defWatcher.disconnected_);
@@ -294,24 +294,24 @@ public:
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_awexists(zh,"/a/b/c",activeWatcher,&wobject1,
         int rc=zoo_awexists(zh,"/a/b/c",activeWatcher,&wobject1,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         CountingDataWatcher wobject2;
         CountingDataWatcher wobject2;
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject2,
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject2,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         // this will process the response and activate the watcher
         // this will process the response and activate the watcher
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
         // we are all set now; let's trigger the watches
         // we are all set now; let's trigger the watches
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/a/b/c"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/a/b/c"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/x/y/z"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHANGED_EVENT,"/x/y/z"));
         // make sure all watchers have been processed
         // make sure all watchers have been processed
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
         
         
         CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
         CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
         CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
         CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
@@ -339,7 +339,7 @@ public:
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_awexists(zh,"/a",activeWatcher,&wobject1,
         int rc=zoo_awexists(zh,"/a",activeWatcher,&wobject1,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
@@ -348,17 +348,17 @@ public:
         DeletionCountingDataWatcher wobject2;
         DeletionCountingDataWatcher wobject2;
         rc=zoo_awget_children(zh,"/a",activeWatcher,
         rc=zoo_awget_children(zh,"/a",activeWatcher,
                 &wobject2,asyncCompletion,&ignored);
                 &wobject2,asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         // this will process the response and activate the watcher
         // this will process the response and activate the watcher
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
         // we are all set now; let's trigger the watches
         // we are all set now; let's trigger the watches
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_DELETED_EVENT,"/a"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_DELETED_EVENT,"/a"));
         // make sure the watchers have been processed
         // make sure the watchers have been processed
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
         CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
         CPPUNIT_ASSERT_EQUAL(1,wobject1.counter_);
         CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
         CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
@@ -385,7 +385,7 @@ public:
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_awexists(zh,"/a",activeWatcher,&wobject1,
         int rc=zoo_awexists(zh,"/a",activeWatcher,&wobject1,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
@@ -394,17 +394,17 @@ public:
         ChildEventCountingWatcher wobject2;
         ChildEventCountingWatcher wobject2;
         rc=zoo_awget_children(zh,"/a",activeWatcher,
         rc=zoo_awget_children(zh,"/a",activeWatcher,
                 &wobject2,asyncCompletion,&ignored);
                 &wobject2,asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         // this will process the response and activate the watcher
         // this will process the response and activate the watcher
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
         // we are all set now; let's trigger the watches
         // we are all set now; let's trigger the watches
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHILD_EVENT,"/a"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHILD_EVENT,"/a"));
         // make sure the watchers have been processed
         // make sure the watchers have been processed
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
         while((rc=zookeeper_process(zh,ZOOKEEPER_READ))==ZOK);
-        CPPUNIT_ASSERT_EQUAL(ZNOTHING,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZNOTHING,rc);
 
 
         CPPUNIT_ASSERT_EQUAL(0,wobject1.counter_);
         CPPUNIT_ASSERT_EQUAL(0,wobject1.counter_);
         CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
         CPPUNIT_ASSERT_EQUAL(1,wobject2.counter_);        
@@ -461,7 +461,7 @@ public:
         // a successful server response will activate the watcher 
         // a successful server response will activate the watcher 
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         int rc=zoo_aexists(zh,"/x/y/z",1,asyncCompletion,&ignored);
         int rc=zoo_aexists(zh,"/x/y/z",1,asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         // now, initiate a disconnect
         // now, initiate a disconnect
         zkServer.setConnectionLost();
         zkServer.setConnectionLost();
@@ -504,7 +504,7 @@ public:
         // set a path-specific watcher
         // set a path-specific watcher
         int rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
         int rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // make sure the watcher gets activated before we continue
         // make sure the watcher gets activated before we continue
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         
         
@@ -553,7 +553,7 @@ public:
         zkServer.addOperationResponse(new ZooStatResponse);
         zkServer.addOperationResponse(new ZooStatResponse);
         activationTracker.track(&defWatcher);
         activationTracker.track(&defWatcher);
         int rc=zoo_aexists(zh,"/a/b/c",1,asyncCompletion,&ignored);
         int rc=zoo_aexists(zh,"/a/b/c",1,asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // make sure the watcher gets activated before we continue
         // make sure the watcher gets activated before we continue
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
 
 
@@ -564,7 +564,7 @@ public:
         // set a path-specific watcher
         // set a path-specific watcher
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // make sure the watcher gets activated before we continue
         // make sure the watcher gets activated before we continue
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         
         
@@ -615,7 +615,7 @@ public:
         activationTracker.track(&wobject1);
         activationTracker.track(&wobject1);
         int rc=zoo_awexists(zh,"/a/b/c",activeWatcher,&wobject1,
         int rc=zoo_awexists(zh,"/a/b/c",activeWatcher,&wobject1,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // make sure the watcher gets activated before we continue
         // make sure the watcher gets activated before we continue
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
 
 
@@ -626,7 +626,7 @@ public:
         // set a path-specific watcher
         // set a path-specific watcher
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject2,
         rc=zoo_awexists(zh,"/x/y/z",activeWatcher,&wobject2,
                 asyncCompletion,&ignored);
                 asyncCompletion,&ignored);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // make sure the watcher gets activated before we continue
         // make sure the watcher gets activated before we continue
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
         CPPUNIT_ASSERT(ensureCondition(activationTracker.isWatcherActivated(),1000)<1000);
 
 
@@ -668,7 +668,7 @@ public:
         Stat stat;
         Stat stat;
         // add a node watch
         // add a node watch
         int rc=zoo_wexists(zh,"/a",activeWatcher,&wobject1,&stat);
         int rc=zoo_wexists(zh,"/a",activeWatcher,&wobject1,&stat);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
@@ -678,7 +678,7 @@ public:
         String_vector children;
         String_vector children;
         rc=zoo_wget_children(zh,"/a",activeWatcher,&wobject2,&children);
         rc=zoo_wget_children(zh,"/a",activeWatcher,&wobject2,&children);
         deallocate_String_vector(&children);
         deallocate_String_vector(&children);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         // we are all set now; let's trigger the watches
         // we are all set now; let's trigger the watches
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_DELETED_EVENT,"/a"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_DELETED_EVENT,"/a"));
@@ -715,7 +715,7 @@ public:
         Stat stat;
         Stat stat;
         // add a node watch
         // add a node watch
         int rc=zoo_wexists(zh,"/a",activeWatcher,&wobject1,&stat);
         int rc=zoo_wexists(zh,"/a",activeWatcher,&wobject1,&stat);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         typedef ZooGetChildrenResponse::StringVector ZooVector;
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
         zkServer.addOperationResponse(new ZooGetChildrenResponse(
@@ -725,7 +725,7 @@ public:
         String_vector children;
         String_vector children;
         rc=zoo_wget_children(zh,"/a",activeWatcher,&wobject2,&children);
         rc=zoo_wget_children(zh,"/a",activeWatcher,&wobject2,&children);
         deallocate_String_vector(&children);
         deallocate_String_vector(&children);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
 
 
         // we are all set now; let's trigger the watches
         // we are all set now; let's trigger the watches
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHILD_EVENT,"/a"));
         zkServer.addRecvResponse(new ZNodeEvent(ZOO_CHILD_EVENT,"/a"));

+ 13 - 13
src/c/tests/TestZookeeperClose.cc

@@ -83,7 +83,7 @@ public:
         freeMock.disable(); // disable mock's fake free()- use libc's free() instead
         freeMock.disable(); // disable mock's fake free()- use libc's free() instead
         
         
         // verify that zookeeper_close has done its job
         // verify that zookeeper_close has done its job
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // memory
         // memory
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
@@ -109,7 +109,7 @@ public:
         freeMock.disable(); // disable mock's fake free()- use libc's free() instead
         freeMock.disable(); // disable mock's fake free()- use libc's free() instead
 
 
         // verify that zookeeper_close has done its job
         // verify that zookeeper_close has done its job
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // memory
         // memory
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
@@ -133,18 +133,18 @@ public:
         timeval tv;
         timeval tv;
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);
         
         
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOOKEEPER_READ|ZOOKEEPER_WRITE,interest);
         CPPUNIT_ASSERT_EQUAL(ZOOKEEPER_READ|ZOOKEEPER_WRITE,interest);
         
         
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_ASSOCIATING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_ASSOCIATING_STATE,zoo_state(zh));
         
         
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTED_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTED_STATE,zoo_state(zh));
         // do not actually free the memory while in zookeeper_close()
         // do not actually free the memory while in zookeeper_close()
         Mock_free_noop freeMock;
         Mock_free_noop freeMock;
@@ -182,15 +182,15 @@ public:
         timeval tv;
         timeval tv;
         // initiate connection
         // initiate connection
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);        
         int rc=zookeeper_interest(zh,&fd,&interest,&tv);        
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_CONNECTING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOOKEEPER_READ|ZOOKEEPER_WRITE,interest);
         CPPUNIT_ASSERT_EQUAL(ZOOKEEPER_READ|ZOOKEEPER_WRITE,interest);
         rc=zookeeper_process(zh,interest);
         rc=zookeeper_process(zh,interest);
         // make sure the handshake in progress 
         // make sure the handshake in progress 
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);        
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);        
         CPPUNIT_ASSERT_EQUAL(ZOO_ASSOCIATING_STATE,zoo_state(zh));
         CPPUNIT_ASSERT_EQUAL(ZOO_ASSOCIATING_STATE,zoo_state(zh));
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
         rc=zookeeper_interest(zh,&fd,&interest,&tv);
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         
         
         // do not actually free the memory while in zookeeper_close()
         // do not actually free the memory while in zookeeper_close()
         Mock_free_noop freeMock;
         Mock_free_noop freeMock;
@@ -230,7 +230,7 @@ public:
         freeMock.disable();
         freeMock.disable();
         
         
         // verify that zookeeper_close has done its job
         // verify that zookeeper_close has done its job
-        CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+        CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         // memory
         // memory
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
         CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh.hostname));
@@ -264,7 +264,7 @@ public:
             CPPUNIT_ASSERT(adaptor!=0);
             CPPUNIT_ASSERT(adaptor!=0);
             int rc=zookeeper_close(zh);
             int rc=zookeeper_close(zh);
             zh=0;
             zh=0;
-            CPPUNIT_ASSERT_EQUAL(ZOK,rc);
+            CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);
         }
         }
     }
     }
     void testCloseConnected1()
     void testCloseConnected1()
@@ -293,7 +293,7 @@ public:
             // we're done, disable mock's fake free(), use libc's free() instead
             // we're done, disable mock's fake free(), use libc's free() instead
             freeMock.disable();
             freeMock.disable();
             
             
-            CPPUNIT_ASSERT_EQUAL(ZOK,rc);            
+            CPPUNIT_ASSERT_EQUAL((int)ZOK,rc);            
             adaptor_threads* adaptor=(adaptor_threads*)lzh.adaptor_priv;
             adaptor_threads* adaptor=(adaptor_threads*)lzh.adaptor_priv;
             // memory
             // memory
             CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
             CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(savezh));
@@ -358,7 +358,7 @@ public:
             // we're done, disable mock's fake free(), use libc's free() instead
             // we're done, disable mock's fake free(), use libc's free() instead
             freeMock.disable();
             freeMock.disable();
             
             
-            CPPUNIT_ASSERT_EQUAL(ZOK,closeAction.rc);          
+            CPPUNIT_ASSERT_EQUAL((int)ZOK,closeAction.rc);          
             adaptor_threads* adaptor=(adaptor_threads*)closeAction.lzh.adaptor_priv;
             adaptor_threads* adaptor=(adaptor_threads*)closeAction.lzh.adaptor_priv;
             // memory
             // memory
             CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh));
             CPPUNIT_ASSERT_EQUAL(1,freeMock.getFreeCount(lzh));

+ 6 - 0
src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml

@@ -1181,6 +1181,12 @@ int main(int argc, char argv) {
       </listitem>
       </listitem>
     </itemizedlist>
     </itemizedlist>
 
 
+    <section id="sc_errorsZk">
+      <title>Handling Errors</title>
+
+      <para>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.</para>
+    </section>
+    
     <section id="sc_connectingToZk">
     <section id="sc_connectingToZk">
       <title>Connecting to ZooKeeper</title>
       <title>Connecting to ZooKeeper</title>
 
 

+ 5 - 4
src/java/main/org/apache/zookeeper/ClientCnxn.java

@@ -458,13 +458,13 @@ public class ClientCnxn {
         }
         }
         switch(zooKeeper.state) {
         switch(zooKeeper.state) {
         case AUTH_FAILED:
         case AUTH_FAILED:
-            p.replyHeader.setErr(KeeperException.Code.AuthFailed);
+            p.replyHeader.setErr(KeeperException.Code.AUTHFAILED.intValue());
             break;
             break;
         case CLOSED:
         case CLOSED:
-            p.replyHeader.setErr(KeeperException.Code.SessionExpired);
+            p.replyHeader.setErr(KeeperException.Code.SESSIONEXPIRED.intValue());
             break;
             break;
         default:
         default:
-            p.replyHeader.setErr(KeeperException.Code.ConnectionLoss);
+            p.replyHeader.setErr(KeeperException.Code.CONNECTIONLOSS.intValue());
         }
         }
         finishPacket(p);
         finishPacket(p);
     }
     }
@@ -566,7 +566,8 @@ public class ClientCnxn {
              */
              */
             try {
             try {
                 if (packet.header.getXid() != replyHdr.getXid()) {
                 if (packet.header.getXid() != replyHdr.getXid()) {
-                    packet.replyHeader.setErr(KeeperException.Code.ConnectionLoss);
+                    packet.replyHeader.setErr(
+                            KeeperException.Code.CONNECTIONLOSS.intValue());
                     throw new IOException("Xid out of order. Got "
                     throw new IOException("Xid out of order. Got "
                             + replyHdr.getXid() + " expected "
                             + replyHdr.getXid() + " expected "
                             + packet.header.getXid());
                             + packet.header.getXid());

+ 383 - 122
src/java/main/org/apache/zookeeper/KeeperException.java

@@ -18,196 +18,400 @@
 
 
 package org.apache.zookeeper;
 package org.apache.zookeeper;
 
 
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+
 @SuppressWarnings("serial")
 @SuppressWarnings("serial")
 public abstract class KeeperException extends Exception {
 public abstract class KeeperException extends Exception {
 
 
     /**
     /**
-     * All non-specific keeper exceptions should be constructed via this factory method
-     * in order to guarantee consistency in error codes and such.  If you know the error
-     * code, then you should construct the special purpose exception directly.  That will
-     * allow you to have the most specific possible declarations of what exceptions might
-     * actually be thrown.
-     * @param code  The error code.
-     * @param path  The zookeeper path being operated on.
-     * @return  The specialized exception, presumably to be thrown by the caller.
+     * All non-specific keeper exceptions should be constructed via
+     * this factory method in order to guarantee consistency in error
+     * codes and such.  If you know the error code, then you should
+     * construct the special purpose exception directly.  That will
+     * allow you to have the most specific possible declarations of
+     * what exceptions might actually be thrown.
+     *
+     * @param code The error code.
+     * @param path The ZooKeeper path being operated on.
+     * @return The specialized exception, presumably to be thrown by
+     *  the caller.
      */
      */
-    public static KeeperException create(int code, String path) {
+    public static KeeperException create(Code code, String path) {
         KeeperException r = create(code);
         KeeperException r = create(code);
         r.path = path;
         r.path = path;
         return r;
         return r;
     }
     }
 
 
     /**
     /**
-     * All non-specific keeper exceptions should be constructed via this factory method
-     * in order to guarantee consistency in error codes and such.  If you know the error
-     * code, then you should construct the special purpose exception directly.  That will
-     * allow you to have the most specific possible declarations of what exceptions might
-     * actually be thrown.
-     * @param code The error code of your new exception.  This will also determine the
-     * specific type of the exception that is returned.
-     * @return  The specialized exception, presumably to be thrown by the caller.
+     * @deprecated deprecated in 3.1.0, use {@link #create(Code, String)}
+     * instead
      */
      */
+    @Deprecated
+    public static KeeperException create(int code, String path) {
+        KeeperException r = create(Code.get(code));
+        r.path = path;
+        return r;
+    }
+
+    /**
+     * @deprecated deprecated in 3.1.0, use {@link #create(Code)}
+     * instead
+     */
+    @Deprecated
     public static KeeperException create(int code) {
     public static KeeperException create(int code) {
+        return create(Code.get(code));
+    }
+
+    /**
+     * All non-specific keeper exceptions should be constructed via
+     * this factory method in order to guarantee consistency in error
+     * codes and such.  If you know the error code, then you should
+     * construct the special purpose exception directly.  That will
+     * allow you to have the most specific possible declarations of
+     * what exceptions might actually be thrown.
+     *
+     * @param code The error code of your new exception.  This will
+     * also determine the specific type of the exception that is
+     * returned.
+     * @return The specialized exception, presumably to be thrown by
+     * the caller.
+     */
+    public static KeeperException create(Code code) {
         switch (code) {
         switch (code) {
-            case Code.SystemError:
+            case SYSTEMERROR:
                 return new SystemErrorException();
                 return new SystemErrorException();
-            case Code.RuntimeInconsistency:
+            case RUNTIMEINCONSISTENCY:
                 return new RuntimeInconsistencyException();
                 return new RuntimeInconsistencyException();
-            case Code.DataInconsistency:
+            case DATAINCONSISTENCY:
                 return new DataInconsistencyException();
                 return new DataInconsistencyException();
-            case Code.ConnectionLoss:
+            case CONNECTIONLOSS:
                 return new ConnectionLossException();
                 return new ConnectionLossException();
-            case Code.MarshallingError:
+            case MARSHALLINGERROR:
                 return new MarshallingErrorException();
                 return new MarshallingErrorException();
-            case Code.Unimplemented:
+            case UNIMPLEMENTED:
                 return new UnimplementedException();
                 return new UnimplementedException();
-            case Code.OperationTimeout:
+            case OPERATIONTIMEOUT:
                 return new OperationTimeoutException();
                 return new OperationTimeoutException();
-            case Code.BadArguments:
+            case BADARGUMENTS:
                 return new BadArgumentsException();
                 return new BadArgumentsException();
-            case Code.APIError:
+            case APIERROR:
                 return new APIErrorException();
                 return new APIErrorException();
-            case Code.NoNode:
+            case NONODE:
                 return new NoNodeException();
                 return new NoNodeException();
-            case Code.NoAuth:
+            case NOAUTH:
                 return new NoAuthException();
                 return new NoAuthException();
-            case Code.BadVersion:
+            case BADVERSION:
                 return new BadVersionException();
                 return new BadVersionException();
-            case Code.NoChildrenForEphemerals:
+            case NOCHILDRENFOREPHEMERALS:
                 return new NoChildrenForEphemeralsException();
                 return new NoChildrenForEphemeralsException();
-            case Code.NodeExists:
+            case NODEEXISTS:
                 return new NodeExistsException();
                 return new NodeExistsException();
-            case Code.InvalidACL:
+            case INVALIDACL:
                 return new InvalidACLException();
                 return new InvalidACLException();
-            case Code.AuthFailed:
+            case AUTHFAILED:
                 return new AuthFailedException();
                 return new AuthFailedException();
-            case Code.NotEmpty:
+            case NOTEMPTY:
                 return new NotEmptyException();
                 return new NotEmptyException();
-            case Code.SessionExpired:
+            case SESSIONEXPIRED:
                 return new SessionExpiredException();
                 return new SessionExpiredException();
-            case Code.InvalidCallback:
+            case INVALIDCALLBACK:
                 return new InvalidCallbackException();
                 return new InvalidCallbackException();
 
 
-            case 0:
+            case OK:
             default:
             default:
                 throw new IllegalArgumentException("Invalid exception code");
                 throw new IllegalArgumentException("Invalid exception code");
         }
         }
     }
     }
 
 
+    /**
+     * Set the code for this exception
+     * @param code error code
+     * @deprecated deprecated in 3.1.0, exceptions should be immutable, this
+     * method should not be used
+     */
+    @Deprecated
     public void setCode(int code) {
     public void setCode(int code) {
-        this.code = code;
+        this.code = Code.get(code);
     }
     }
 
 
-    public interface Code {
-        int Ok = 0;
-
-        // System and server-side errors
-        int SystemError = -1;
-
-        int RuntimeInconsistency = SystemError - 1;
-
-        int DataInconsistency = SystemError - 2;
-
-        int ConnectionLoss = SystemError - 3;
-
-        int MarshallingError = SystemError - 4;
-
-        int Unimplemented = SystemError - 5;
-
-        int OperationTimeout = SystemError - 6;
-
-        int BadArguments = SystemError - 7;
-
-        // API errors
-        int APIError = -100; // Catch all, shouldn't be used other
-        // than range start
-
-        int NoNode = APIError - 1; // Node does not exist
-
-        int NoAuth = APIError - 2; // Current operation not permitted
-
-        int BadVersion = APIError - 3; // Version conflict
-
-        int NoChildrenForEphemerals = APIError - 8;
-
-        int NodeExists = APIError - 10;
-
-        int NotEmpty = APIError - 11;
-
-        int SessionExpired = APIError - 12;
-
-        int InvalidCallback = APIError - 13;
+    public static enum Code {
+        /** Everything is OK */
+        OK (0),
+
+        /** System and server-side errors.
+         * This is never thrown by the server, it shouldn't be used other than
+         * to indicate a range. Specifically error codes greater than this
+         * value, but lesser than {@link #APIERROR}, are system errors.
+         */
+        SYSTEMERROR (-1),
+
+        /** A runtime inconsistency was found */
+        RUNTIMEINCONSISTENCY (-2),
+        /** A data inconsistency was found */
+        DATAINCONSISTENCY (-3),
+        /** Connection to the server has been lost */
+        CONNECTIONLOSS (-4),
+        /** Error while marshalling or unmarshalling data */
+        MARSHALLINGERROR (-5),
+        /** Operation is unimplemented */
+        UNIMPLEMENTED (-6),
+        /** Operation timeout */
+        OPERATIONTIMEOUT (-7),
+        /** Invalid arguments */
+        BADARGUMENTS (-8),
+
+        /** API errors.
+         * This is never thrown by the server, it shouldn't be used other than
+         * to indicate a range. Specifically error codes greater than this
+         * value are API errors (while values less than this indicate a 
+         * {@link #SYSTEMERROR}).
+         */
+        APIERROR (-100),
+
+        /** Node does not exist */
+        NONODE (-101),
+        /** Not authenticated */
+        NOAUTH (-102),
+        /** Version conflict */
+        BADVERSION (-103),
+        /** Ephemeral nodes may not have children */
+        NOCHILDRENFOREPHEMERALS (-108),
+        /** The node already exists */
+        NODEEXISTS (-110),
+        /** The node has children */
+        NOTEMPTY (-111),
+        /** The session has been expired by the server */
+        SESSIONEXPIRED (-112),
+        /** Invalid callback specified */
+        INVALIDCALLBACK (-113),
+        /** Invalid ACL specified */
+        INVALIDACL (-114),
+        /** Client authentication failed */
+        AUTHFAILED (-115);
+
+        private static final Map<Integer,Code> lookup
+            = new HashMap<Integer,Code>();
+
+        static {
+            for(Code c : EnumSet.allOf(Code.class))
+                lookup.put(c.code, c);
+        }
 
 
-        int InvalidACL = APIError - 14;
+        private final int code;
+        Code(int code) {
+            this.code = code;
+        }
 
 
-        int AuthFailed = APIError - 15; // client authentication failed
+        /**
+         * Get the int value for a particular Code.
+         * @return error code as integer
+         */
+        public int intValue() { return code; }
+
+        /**
+         * Get the Code value for a particular integer error code
+         * @param code int error code
+         * @return Code value corresponding to specified int code, or null
+         */
+        public static Code get(int code) {
+            return lookup.get(code);
+        }
 
 
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #OK} instead
+         */
+        @Deprecated
+        public static final int Ok = OK.code;
+
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #SYSTEMERROR} instead
+         */
+        @Deprecated
+        public static final int SystemError = SYSTEMERROR.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #RUNTIMEINCONSISTENCY} instead
+         */
+        @Deprecated
+        public static final int RuntimeInconsistency = RUNTIMEINCONSISTENCY.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #DATAINCONSISTENCY}
+         * instead
+         */
+        @Deprecated
+        public static final int DataInconsistency = DATAINCONSISTENCY.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #CONNECTIONLOSS}
+         * instead
+         */
+        @Deprecated
+        public static final int ConnectionLoss = CONNECTIONLOSS.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #MARSHALLINGERROR}
+         * instead
+         */
+        @Deprecated
+        public static final int MarshallingError = MARSHALLINGERROR.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #UNIMPLEMENTED} instead
+         */
+        @Deprecated
+        public static final int Unimplemented = UNIMPLEMENTED.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #OPERATIONTIMEOUT}
+         * instead
+         */
+        @Deprecated
+        public static final int OperationTimeout = OPERATIONTIMEOUT.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #BADARGUMENTS} instead
+         */
+        @Deprecated
+        public static final int BadArguments = BADARGUMENTS.code;
+
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #APIERROR} instead
+         */
+        @Deprecated
+        public static final int APIError = APIERROR.code;
+
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #NONODE} instead
+         */
+        @Deprecated
+        public static final int NoNode = NONODE.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #NOAUTH} instead
+         */
+        @Deprecated
+        public static final int NoAuth = NOAUTH.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #BADVERSION} instead
+         */
+        @Deprecated
+        public static final int BadVersion = BADVERSION.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #NOCHILDRENFOREPHEMERALS}
+         * instead
+         */
+        @Deprecated
+        public static final int
+            NoChildrenForEphemerals = NOCHILDRENFOREPHEMERALS.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #NODEEXISTS} instead
+         */
+        @Deprecated
+        public static final int NodeExists = NODEEXISTS.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #NOTEMPTY} instead
+         */
+        @Deprecated
+        public static final int NotEmpty = NOTEMPTY.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #SESSIONEXPIRED} instead
+         */
+        @Deprecated
+        public static final int SessionExpired = SESSIONEXPIRED.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #INVALIDCALLBACK} instead
+         */
+        @Deprecated
+        public static final int InvalidCallback = INVALIDCALLBACK.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #INVALIDACL} instead
+         */
+        @Deprecated
+        public static final int InvalidACL = INVALIDACL.code;
+        /**
+         * @deprecated deprecated in 3.1.0, use {@link #AUTHFAILED} instead
+         */
+        @Deprecated
+        public static final int AuthFailed = AUTHFAILED.code;
     }
     }
 
 
-    static String getCodeMessage(int code) {
+    static String getCodeMessage(Code code) {
         switch (code) {
         switch (code) {
-            case 0:
+            case OK:
                 return "ok";
                 return "ok";
-            case Code.SystemError:
+            case SYSTEMERROR:
                 return "SystemError";
                 return "SystemError";
-            case Code.RuntimeInconsistency:
+            case RUNTIMEINCONSISTENCY:
                 return "RuntimeInconsistency";
                 return "RuntimeInconsistency";
-            case Code.DataInconsistency:
+            case DATAINCONSISTENCY:
                 return "DataInconsistency";
                 return "DataInconsistency";
-            case Code.ConnectionLoss:
+            case CONNECTIONLOSS:
                 return "ConnectionLoss";
                 return "ConnectionLoss";
-            case Code.MarshallingError:
+            case MARSHALLINGERROR:
                 return "MarshallingError";
                 return "MarshallingError";
-            case Code.Unimplemented:
+            case UNIMPLEMENTED:
                 return "Unimplemented";
                 return "Unimplemented";
-            case Code.OperationTimeout:
+            case OPERATIONTIMEOUT:
                 return "OperationTimeout";
                 return "OperationTimeout";
-            case Code.BadArguments:
+            case BADARGUMENTS:
                 return "BadArguments";
                 return "BadArguments";
-            case Code.APIError:
+            case APIERROR:
                 return "APIError";
                 return "APIError";
-            case Code.NoNode:
+            case NONODE:
                 return "NoNode";
                 return "NoNode";
-            case Code.NoAuth:
+            case NOAUTH:
                 return "NoAuth";
                 return "NoAuth";
-            case Code.BadVersion:
+            case BADVERSION:
                 return "BadVersion";
                 return "BadVersion";
-            case Code.NoChildrenForEphemerals:
+            case NOCHILDRENFOREPHEMERALS:
                 return "NoChildrenForEphemerals";
                 return "NoChildrenForEphemerals";
-            case Code.NodeExists:
+            case NODEEXISTS:
                 return "NodeExists";
                 return "NodeExists";
-            case Code.InvalidACL:
+            case INVALIDACL:
                 return "InvalidACL";
                 return "InvalidACL";
-            case Code.AuthFailed:
+            case AUTHFAILED:
                 return "AuthFailed";
                 return "AuthFailed";
-            case Code.NotEmpty:
+            case NOTEMPTY:
                 return "Directory not empty";
                 return "Directory not empty";
-            case Code.SessionExpired:
+            case SESSIONEXPIRED:
                 return "Session expired";
                 return "Session expired";
-            case Code.InvalidCallback:
+            case INVALIDCALLBACK:
                 return "Invalid callback";
                 return "Invalid callback";
             default:
             default:
                 return "Unknown error " + code;
                 return "Unknown error " + code;
         }
         }
     }
     }
 
 
-    private int code;
+    private Code code;
 
 
     private String path;
     private String path;
 
 
-    public KeeperException(int code) {
+    public KeeperException(Code code) {
         this.code = code;
         this.code = code;
     }
     }
 
 
-    KeeperException(int code, String path) {
+    KeeperException(Code code, String path) {
         this.code = code;
         this.code = code;
         this.path = path;
         this.path = path;
     }
     }
 
 
+    /**
+     * Read the error code for this exception
+     * @return the error code for this exception
+     * @deprecated deprecated in 3.1.0, use {@link #code()} instead
+     */
+    @Deprecated
     public int getCode() {
     public int getCode() {
+        return code.code;
+    }
+
+    /**
+     * Read the error Code for this exception
+     * @return the error Code for this exception
+     */
+    public Code code() {
         return code;
         return code;
     }
     }
 
 
+    /**
+     * Read the path for this exception
+     * @return the path associated with this error, null if none
+     */
     public String getPath() {
     public String getPath() {
         return path;
         return path;
     }
     }
@@ -220,117 +424,174 @@ public abstract class KeeperException extends Exception {
         return "KeeperErrorCode = " + getCodeMessage(code) + " for " + path;
         return "KeeperErrorCode = " + getCodeMessage(code) + " for " + path;
     }
     }
 
 
+    /**
+     *  @see Code.APIERROR
+     */
     public static class APIErrorException extends KeeperException {
     public static class APIErrorException extends KeeperException {
         public APIErrorException() {
         public APIErrorException() {
-            super(Code.APIError);
+            super(Code.APIERROR);
         }
         }
     }
     }
 
 
+    /**
+     *  @see Code.AUTHFAILED
+     */
     public static class AuthFailedException extends KeeperException {
     public static class AuthFailedException extends KeeperException {
         public AuthFailedException() {
         public AuthFailedException() {
-            super(Code.AuthFailed);
+            super(Code.AUTHFAILED);
         }
         }
     }
     }
 
 
+    /**
+     *  @see Code.BADARGUMENTS
+     */
     public static class BadArgumentsException extends KeeperException {
     public static class BadArgumentsException extends KeeperException {
         public BadArgumentsException() {
         public BadArgumentsException() {
-            super(Code.BadArguments);
+            super(Code.BADARGUMENTS);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.BADVERSION
+     */
     public static class BadVersionException extends KeeperException {
     public static class BadVersionException extends KeeperException {
         public BadVersionException() {
         public BadVersionException() {
-            super(Code.BadVersion);
+            super(Code.BADVERSION);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.CONNECTIONLOSS
+     */
     public static class ConnectionLossException extends KeeperException {
     public static class ConnectionLossException extends KeeperException {
         public ConnectionLossException() {
         public ConnectionLossException() {
-            super(Code.ConnectionLoss);
+            super(Code.CONNECTIONLOSS);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.DATAINCONSISTENCY
+     */
     public static class DataInconsistencyException extends KeeperException {
     public static class DataInconsistencyException extends KeeperException {
         public DataInconsistencyException() {
         public DataInconsistencyException() {
-            super(Code.DataInconsistency);
+            super(Code.DATAINCONSISTENCY);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.INVALIDACL
+     */
     public static class InvalidACLException extends KeeperException {
     public static class InvalidACLException extends KeeperException {
         public InvalidACLException() {
         public InvalidACLException() {
-            super(Code.InvalidACL);
+            super(Code.INVALIDACL);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.INVALIDCALLBACK
+     */
     public static class InvalidCallbackException extends KeeperException {
     public static class InvalidCallbackException extends KeeperException {
         public InvalidCallbackException() {
         public InvalidCallbackException() {
-            super(Code.InvalidCallback);
+            super(Code.INVALIDCALLBACK);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.MARSHALLINGERROR
+     */
     public static class MarshallingErrorException extends KeeperException {
     public static class MarshallingErrorException extends KeeperException {
         public MarshallingErrorException() {
         public MarshallingErrorException() {
-            super(Code.MarshallingError);
+            super(Code.MARSHALLINGERROR);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.NOAUTH
+     */
     public static class NoAuthException extends KeeperException {
     public static class NoAuthException extends KeeperException {
         public NoAuthException() {
         public NoAuthException() {
-            super(Code.NoAuth);
+            super(Code.NOAUTH);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.NOCHILDRENFOREPHEMERALS
+     */
     public static class NoChildrenForEphemeralsException extends KeeperException {
     public static class NoChildrenForEphemeralsException extends KeeperException {
         public NoChildrenForEphemeralsException() {
         public NoChildrenForEphemeralsException() {
-            super(Code.NoChildrenForEphemerals);
+            super(Code.NOCHILDRENFOREPHEMERALS);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.NODEEXISTS
+     */
     public static class NodeExistsException extends KeeperException {
     public static class NodeExistsException extends KeeperException {
         public NodeExistsException() {
         public NodeExistsException() {
-            super(Code.NodeExists);
+            super(Code.NODEEXISTS);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.NONODE
+     */
     public static class NoNodeException extends KeeperException {
     public static class NoNodeException extends KeeperException {
         public NoNodeException() {
         public NoNodeException() {
-            super(Code.NoNode);
+            super(Code.NONODE);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.NOTEMPTY
+     */
     public static class NotEmptyException extends KeeperException {
     public static class NotEmptyException extends KeeperException {
         public NotEmptyException() {
         public NotEmptyException() {
-            super(Code.NotEmpty);
+            super(Code.NOTEMPTY);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.OPERATIONTIMEOUT
+     */
     public static class OperationTimeoutException extends KeeperException {
     public static class OperationTimeoutException extends KeeperException {
         public OperationTimeoutException() {
         public OperationTimeoutException() {
-            super(Code.OperationTimeout);
+            super(Code.OPERATIONTIMEOUT);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.RUNTIMEINCONSISTENCY
+     */
     public static class RuntimeInconsistencyException extends KeeperException {
     public static class RuntimeInconsistencyException extends KeeperException {
         public RuntimeInconsistencyException() {
         public RuntimeInconsistencyException() {
-            super(Code.RuntimeInconsistency);
+            super(Code.RUNTIMEINCONSISTENCY);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.SESSIONEXPIRED
+     */
     public static class SessionExpiredException extends KeeperException {
     public static class SessionExpiredException extends KeeperException {
         public SessionExpiredException() {
         public SessionExpiredException() {
-            super(Code.SessionExpired);
+            super(Code.SESSIONEXPIRED);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.SYSTEMERROR
+     */
     public static class SystemErrorException extends KeeperException {
     public static class SystemErrorException extends KeeperException {
         public SystemErrorException() {
         public SystemErrorException() {
-            super(Code.SystemError);
+            super(Code.SYSTEMERROR);
         }
         }
     }
     }
 
 
+    /**
+     * @see Code.UNIMPLEMENTED
+     */
     public static class UnimplementedException extends KeeperException {
     public static class UnimplementedException extends KeeperException {
         public UnimplementedException() {
         public UnimplementedException() {
-            super(Code.Unimplemented);
+            super(Code.UNIMPLEMENTED);
         }
         }
     }
     }
 }
 }

+ 11 - 10
src/java/main/org/apache/zookeeper/ZooKeeper.java

@@ -278,7 +278,7 @@ public class ZooKeeper {
         
         
         @Override
         @Override
         protected boolean shouldAddWatch(int rc) {
         protected boolean shouldAddWatch(int rc) {
-            return rc == 0 || rc == KeeperException.Code.NoNode;
+            return rc == 0 || rc == KeeperException.Code.NONODE.intValue();
         }
         }
     }
     }
 
 
@@ -496,7 +496,8 @@ public class ZooKeeper {
         request.setAcl(acl);
         request.setAcl(acl);
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr(), path);
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()),
+                    path);
         }
         }
         return response.getPath();
         return response.getPath();
     }
     }
@@ -557,7 +558,7 @@ public class ZooKeeper {
         request.setVersion(version);
         request.setVersion(version);
         ReplyHeader r = cnxn.submitRequest(h, request, null, null);
         ReplyHeader r = cnxn.submitRequest(h, request, null, null);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
     }
     }
 
 
@@ -607,10 +608,10 @@ public class ZooKeeper {
         }
         }
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            if (r.getErr() == KeeperException.Code.NoNode) {
+            if (r.getErr() == KeeperException.Code.NONODE.intValue()) {
                 return null;
                 return null;
             }
             }
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
 
 
         return response.getStat().getCzxid() == -1 ? null : response.getStat();
         return response.getStat().getCzxid() == -1 ? null : response.getStat();
@@ -705,7 +706,7 @@ public class ZooKeeper {
         }
         }
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
         if (stat != null) {
         if (stat != null) {
             DataTree.copyStat(response.getStat(), stat);
             DataTree.copyStat(response.getStat(), stat);
@@ -805,7 +806,7 @@ public class ZooKeeper {
         SetDataResponse response = new SetDataResponse();
         SetDataResponse response = new SetDataResponse();
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
         return response.getStat();
         return response.getStat();
     }
     }
@@ -853,7 +854,7 @@ public class ZooKeeper {
         GetACLResponse response = new GetACLResponse();
         GetACLResponse response = new GetACLResponse();
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
         DataTree.copyStat(response.getStat(), stat);
         DataTree.copyStat(response.getStat(), stat);
         return response.getAcl();
         return response.getAcl();
@@ -909,7 +910,7 @@ public class ZooKeeper {
         SetACLResponse response = new SetACLResponse();
         SetACLResponse response = new SetACLResponse();
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         ReplyHeader r = cnxn.submitRequest(h, request, response, null);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
         return response.getStat();
         return response.getStat();
     }
     }
@@ -968,7 +969,7 @@ public class ZooKeeper {
         }
         }
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         ReplyHeader r = cnxn.submitRequest(h, request, response, wcb);
         if (r.getErr() != 0) {
         if (r.getErr() != 0) {
-            throw KeeperException.create(r.getErr());
+            throw KeeperException.create(KeeperException.Code.get(r.getErr()));
         }
         }
         return response.getChildren();
         return response.getChildren();
     }
     }

+ 2 - 1
src/java/main/org/apache/zookeeper/server/DataTree.java

@@ -558,7 +558,8 @@ public class DataTree {
         } catch (KeeperException e) {
         } catch (KeeperException e) {
             // These are expected errors since we take a lazy snapshot
             // These are expected errors since we take a lazy snapshot
             if (initialized
             if (initialized
-                    || (e.getCode() != Code.NoNode && e.getCode() != Code.NodeExists)) {
+                    || (e.code() != Code.NONODE 
+                            && e.code() != Code.NODEEXISTS)) {
                 LOG.warn(debug);
                 LOG.warn(debug);
                 LOG.error("FIXMSG",e);
                 LOG.error("FIXMSG",e);
             }
             }

+ 12 - 10
src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java

@@ -125,11 +125,12 @@ public class FinalRequestProcessor implements RequestProcessor {
             return;
             return;
         }
         }
         zks.decInProcess();
         zks.decInProcess();
-        int err = 0;
+        Code err = Code.OK;
         Record rsp = null;
         Record rsp = null;
         try {
         try {
             if (request.hdr != null && request.hdr.getType() == OpCode.error) {
             if (request.hdr != null && request.hdr.getType() == OpCode.error) {
-                throw KeeperException.create(((ErrorTxn) request.txn).getErr());
+                throw KeeperException.create(KeeperException.Code.get((
+                        (ErrorTxn) request.txn).getErr()));
             }
             }
 
 
             if (LOG.isDebugEnabled()) {
             if (LOG.isDebugEnabled()) {
@@ -145,21 +146,21 @@ public class FinalRequestProcessor implements RequestProcessor {
                 return;
                 return;
             case OpCode.create:
             case OpCode.create:
                 rsp = new CreateResponse(rc.path);
                 rsp = new CreateResponse(rc.path);
-                err = rc.err;
+                err = Code.get(rc.err);
                 break;
                 break;
             case OpCode.delete:
             case OpCode.delete:
-                err = rc.err;
+                err = Code.get(rc.err);
                 break;
                 break;
             case OpCode.setData:
             case OpCode.setData:
                 rsp = new SetDataResponse(rc.stat);
                 rsp = new SetDataResponse(rc.stat);
-                err = rc.err;
+                err = Code.get(rc.err);
                 break;
                 break;
             case OpCode.setACL:
             case OpCode.setACL:
                 rsp = new SetACLResponse(rc.stat);
                 rsp = new SetACLResponse(rc.stat);
-                err = rc.err;
+                err = Code.get(rc.err);
                 break;
                 break;
             case OpCode.closeSession:
             case OpCode.closeSession:
-                err = rc.err;
+                err = Code.get(rc.err);
                 break;
                 break;
             case OpCode.sync:
             case OpCode.sync:
                 SyncRequest syncRequest = new SyncRequest();
                 SyncRequest syncRequest = new SyncRequest();
@@ -235,7 +236,7 @@ public class FinalRequestProcessor implements RequestProcessor {
                 break;
                 break;
             }
             }
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            err = e.getCode();
+            err = e.code();
         } catch (Exception e) {
         } catch (Exception e) {
             LOG.warn("****************************** " + request);
             LOG.warn("****************************** " + request);
             StringBuffer sb = new StringBuffer();
             StringBuffer sb = new StringBuffer();
@@ -246,9 +247,10 @@ public class FinalRequestProcessor implements RequestProcessor {
             }
             }
             LOG.warn(sb.toString());
             LOG.warn(sb.toString());
             LOG.error("FIXMSG",e);
             LOG.error("FIXMSG",e);
-            err = Code.MarshallingError;
+            err = Code.MARSHALLINGERROR;
         }
         }
-        ReplyHeader hdr = new ReplyHeader(request.cxid, request.zxid, err);
+        ReplyHeader hdr =
+            new ReplyHeader(request.cxid, request.zxid, err.intValue());
         ServerStats.getInstance().updateLatency(request.createTime);
         ServerStats.getInstance().updateLatency(request.createTime);
         try {
         try {
             request.cnxn.sendResponse(hdr, rsp, "response");
             request.cnxn.sendResponse(hdr, rsp, "response");

+ 4 - 3
src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

@@ -423,7 +423,8 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
             String scheme = authPacket.getScheme();
             String scheme = authPacket.getScheme();
             AuthenticationProvider ap = ProviderRegistry.getProvider(scheme);
             AuthenticationProvider ap = ProviderRegistry.getProvider(scheme);
             if (ap == null
             if (ap == null
-                    || ap.handleAuthentication(this, authPacket.getAuth()) != KeeperException.Code.Ok) {
+                    || (ap.handleAuthentication(this, authPacket.getAuth()) 
+                            != KeeperException.Code.OK)) {
                 if (ap == null)
                 if (ap == null)
                     LOG.error("No authentication provider for scheme: "
                     LOG.error("No authentication provider for scheme: "
                             + scheme + " has " + ProviderRegistry.listProviders());
                             + scheme + " has " + ProviderRegistry.listProviders());
@@ -432,7 +433,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
                             + scheme);
                             + scheme);
                 // send a response...
                 // send a response...
                 ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
                 ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
-                        KeeperException.Code.AuthFailed);
+                        KeeperException.Code.AUTHFAILED.intValue());
                 sendResponse(rh, null, null);
                 sendResponse(rh, null, null);
                 // ... and close connection
                 // ... and close connection
                 sendBuffer(NIOServerCnxn.closeConn);
                 sendBuffer(NIOServerCnxn.closeConn);
@@ -441,7 +442,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
                 LOG.debug("Authentication succeeded for scheme: "
                 LOG.debug("Authentication succeeded for scheme: "
                         + scheme);
                         + scheme);
                 ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
                 ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
-                        KeeperException.Code.Ok);
+                        KeeperException.Code.OK.intValue());
                 sendResponse(rh, null, null);
                 sendResponse(rh, null, null);
             }
             }
             return;
             return;

+ 2 - 2
src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

@@ -358,7 +358,7 @@ public class PrepRequestProcessor extends Thread implements RequestProcessor {
         } catch (KeeperException e) {
         } catch (KeeperException e) {
             if (txnHeader != null) {
             if (txnHeader != null) {
                 txnHeader.setType(OpCode.error);
                 txnHeader.setType(OpCode.error);
-                txn = new ErrorTxn(e.getCode());
+                txn = new ErrorTxn(e.code().intValue());
             }
             }
         } catch (Exception e) {
         } catch (Exception e) {
             LOG.error("*********************************" + request);
             LOG.error("*********************************" + request);
@@ -375,7 +375,7 @@ public class PrepRequestProcessor extends Thread implements RequestProcessor {
             LOG.error("Unexpected exception", e);
             LOG.error("Unexpected exception", e);
             if (txnHeader != null) {
             if (txnHeader != null) {
                 txnHeader.setType(OpCode.error);
                 txnHeader.setType(OpCode.error);
-                txn = new ErrorTxn(Code.MarshallingError);
+                txn = new ErrorTxn(Code.MARSHALLINGERROR.intValue());
             }
             }
         }
         }
         request.hdr = txnHeader;
         request.hdr = txnHeader;

+ 2 - 1
src/java/main/org/apache/zookeeper/server/auth/AuthenticationProvider.java

@@ -18,6 +18,7 @@
 
 
 package org.apache.zookeeper.server.auth;
 package org.apache.zookeeper.server.auth;
 
 
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerCnxn;
 
 
 /**
 /**
@@ -45,7 +46,7 @@ public interface AuthenticationProvider {
      *                the authentication data received.
      *                the authentication data received.
      * @return TODO
      * @return TODO
      */
      */
-    int handleAuthentication(ServerCnxn cnxn, byte authData[]);
+    KeeperException.Code handleAuthentication(ServerCnxn cnxn, byte authData[]);
 
 
     /**
     /**
      * This method is called to see if the given id matches the given id
      * This method is called to see if the given id matches the given id

+ 5 - 4
src/java/main/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java

@@ -23,7 +23,6 @@ import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
 
 
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
-
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerCnxn;
@@ -90,7 +89,9 @@ public class DigestAuthenticationProvider implements AuthenticationProvider {
         return parts[0] + ":" + base64Encode(digest);
         return parts[0] + ":" + base64Encode(digest);
     }
     }
 
 
-    public int handleAuthentication(ServerCnxn cnxn, byte[] authData) {
+    public KeeperException.Code 
+        handleAuthentication(ServerCnxn cnxn, byte[] authData)
+    {
         String id = new String(authData);
         String id = new String(authData);
         try {
         try {
             String digest = generateDigest(id);
             String digest = generateDigest(id);
@@ -98,11 +99,11 @@ public class DigestAuthenticationProvider implements AuthenticationProvider {
                 cnxn.getAuthInfo().add(new Id("super", ""));
                 cnxn.getAuthInfo().add(new Id("super", ""));
             }
             }
             cnxn.getAuthInfo().add(new Id(getScheme(), digest));
             cnxn.getAuthInfo().add(new Id(getScheme(), digest));
-            return KeeperException.Code.Ok;
+            return KeeperException.Code.OK;
         } catch (NoSuchAlgorithmException e) {
         } catch (NoSuchAlgorithmException e) {
             LOG.error("Missing algorithm",e);
             LOG.error("Missing algorithm",e);
         }
         }
-        return KeeperException.Code.AuthFailed;
+        return KeeperException.Code.AUTHFAILED;
     }
     }
 
 
     public boolean isAuthenticated() {
     public boolean isAuthenticated() {

+ 4 - 2
src/java/main/org/apache/zookeeper/server/auth/HostAuthenticationProvider.java

@@ -28,10 +28,12 @@ public class HostAuthenticationProvider implements AuthenticationProvider {
         return "host";
         return "host";
     }
     }
 
 
-    public int handleAuthentication(ServerCnxn cnxn, byte[] authData) {
+    public KeeperException.Code
+        handleAuthentication(ServerCnxn cnxn, byte[] authData)
+    {
         String id = cnxn.getRemoteAddress().getAddress().getCanonicalHostName();
         String id = cnxn.getRemoteAddress().getAddress().getCanonicalHostName();
         cnxn.getAuthInfo().add(new Id(getScheme(), id));
         cnxn.getAuthInfo().add(new Id(getScheme(), id));
-        return KeeperException.Code.Ok;
+        return KeeperException.Code.OK;
     }
     }
 
 
     public boolean matches(String id, String aclExpr) {
     public boolean matches(String id, String aclExpr) {

+ 4 - 2
src/java/main/org/apache/zookeeper/server/auth/IPAuthenticationProvider.java

@@ -28,10 +28,12 @@ public class IPAuthenticationProvider implements AuthenticationProvider {
         return "ip";
         return "ip";
     }
     }
 
 
-    public int handleAuthentication(ServerCnxn cnxn, byte[] authData) {
+    public KeeperException.Code
+        handleAuthentication(ServerCnxn cnxn, byte[] authData)
+    {
         String id = cnxn.getRemoteAddress().getAddress().getHostAddress();
         String id = cnxn.getRemoteAddress().getAddress().getHostAddress();
         cnxn.getAuthInfo().add(new Id(getScheme(), id));
         cnxn.getAuthInfo().add(new Id(getScheme(), id));
-        return KeeperException.Code.Ok;
+        return KeeperException.Code.OK;
     }
     }
 
 
     // This is a bit weird but we need to return the address and the number of
     // This is a bit weird but we need to return the address and the number of

+ 2 - 1
src/java/main/org/apache/zookeeper/server/upgrade/DataTreeV1.java

@@ -437,7 +437,8 @@ public class DataTreeV1 {
         } catch (KeeperException e) {
         } catch (KeeperException e) {
             // These are expected errors since we take a lazy snapshot
             // These are expected errors since we take a lazy snapshot
             if (initialized
             if (initialized
-                    || (e.getCode() != Code.NoNode && e.getCode() != Code.NodeExists)) {
+                    || (e.code() != Code.NONODE 
+                            && e.code() != Code.NODEEXISTS)) {
                 LOG.warn(debug);
                 LOG.warn(debug);
                 LOG.error("FIXMSG",e);
                 LOG.error("FIXMSG",e);
             }
             }

+ 19 - 15
src/java/test/org/apache/zookeeper/test/AsyncOps.java

@@ -84,7 +84,7 @@ public class AsyncOps {
         /** the latch is used to await the results from the server */
         /** the latch is used to await the results from the server */
         CountDownLatch latch;
         CountDownLatch latch;
 
 
-        int rc = 0;
+        Code rc = Code.OK;
         String path = "/foo";
         String path = "/foo";
         String expected;
         String expected;
         
         
@@ -93,7 +93,7 @@ public class AsyncOps {
             this.latch = latch;
             this.latch = latch;
         }
         }
         
         
-        public void setRC(int rc) {
+        public void setRC(Code rc) {
             this.rc = rc;
             this.rc = rc;
         }
         }
         
         
@@ -101,7 +101,7 @@ public class AsyncOps {
             this.path = path;
             this.path = path;
         }
         }
         
         
-        public void processResult(int rc, String path, Object ctx)
+        public void processResult(Code rc, String path, Object ctx)
         {
         {
             this.rc = rc;
             this.rc = rc;
             this.path = path;
             this.path = path;
@@ -157,7 +157,7 @@ public class AsyncOps {
         public void processResult(int rc, String path, Object ctx, String name)
         public void processResult(int rc, String path, Object ctx, String name)
         {
         {
             this.name = name;
             this.name = name;
-            super.processResult(rc, path, ctx);
+            super.processResult(Code.get(rc), path, ctx);
         }
         }
 
 
         public AsyncCB create() {
         public AsyncCB create() {
@@ -173,7 +173,7 @@ public class AsyncOps {
         public void verifyCreateFailure_NodeExists() {
         public void verifyCreateFailure_NodeExists() {
             new StringCB(zk).verifyCreate();
             new StringCB(zk).verifyCreate();
             
             
-            rc = Code.NodeExists;
+            rc = Code.NODEEXISTS;
             name = null;
             name = null;
             zk.create(path, data, acl, flags, this, toString());
             zk.create(path, data, acl, flags, this, toString());
             verify();
             verify();
@@ -208,7 +208,7 @@ public class AsyncOps {
         {
         {
             this.acl = acl;
             this.acl = acl;
             this.stat = stat;
             this.stat = stat;
-            super.processResult(rc, path, ctx);
+            super.processResult(Code.get(rc), path, ctx);
         }
         }
         
         
         public void verifyGetACL() {
         public void verifyGetACL() {
@@ -252,7 +252,7 @@ public class AsyncOps {
         {
         {
             this.children =
             this.children =
                 (children == null ? new ArrayList<String>() : children);
                 (children == null ? new ArrayList<String>() : children);
-            super.processResult(rc, path, ctx);
+            super.processResult(Code.get(rc), path, ctx);
         }
         }
         
         
         public StringCB createNode() {
         public StringCB createNode() {
@@ -306,7 +306,7 @@ public class AsyncOps {
         }
         }
         
         
         public void verifyGetChildrenFailure_NoNode() {
         public void verifyGetChildrenFailure_NoNode() {
-            rc = KeeperException.Code.NoNode;
+            rc = KeeperException.Code.NONODE;
             verify();
             verify();
         }
         }
         
         
@@ -344,7 +344,7 @@ public class AsyncOps {
         {
         {
             this.data = data;
             this.data = data;
             this.stat = stat;
             this.stat = stat;
-            super.processResult(rc, path, ctx);
+            super.processResult(Code.get(rc), path, ctx);
         }
         }
         
         
         public void verifyGetData() {
         public void verifyGetData() {
@@ -355,7 +355,7 @@ public class AsyncOps {
         }
         }
         
         
         public void verifyGetDataFailure_NoNode() {
         public void verifyGetDataFailure_NoNode() {
-            rc = KeeperException.Code.NoNode;
+            rc = KeeperException.Code.NONODE;
             data = null;
             data = null;
             stat = null;
             stat = null;
             zk.getData(path, false, this, toString());
             zk.getData(path, false, this, toString());
@@ -392,7 +392,7 @@ public class AsyncOps {
         
         
         public void processResult(int rc, String path, Object ctx, Stat stat) {
         public void processResult(int rc, String path, Object ctx, Stat stat) {
             this.stat = stat;
             this.stat = stat;
-            super.processResult(rc, path, ctx);
+            super.processResult(Code.get(rc), path, ctx);
         }
         }
         
         
         public void verifySetACL() {
         public void verifySetACL() {
@@ -404,7 +404,7 @@ public class AsyncOps {
         }
         }
         
         
         public void verifySetACLFailure_NoNode() {
         public void verifySetACLFailure_NoNode() {
-            rc = KeeperException.Code.NoNode;
+            rc = KeeperException.Code.NONODE;
             stat = null;
             stat = null;
             zk.setACL(path, acl, version, this, toString());
             zk.setACL(path, acl, version, this, toString());
             verify();
             verify();
@@ -423,7 +423,7 @@ public class AsyncOps {
         }
         }
         
         
         public void verifySetDataFailure_NoNode() {
         public void verifySetDataFailure_NoNode() {
-            rc = KeeperException.Code.NoNode;
+            rc = KeeperException.Code.NONODE;
             stat = null;
             stat = null;
             zk.setData(path, data, version, this, toString());
             zk.setData(path, data, version, this, toString());
             verify();
             verify();
@@ -437,7 +437,7 @@ public class AsyncOps {
         }
         }
         
         
         public void verifyExistsFailure_NoNode() {
         public void verifyExistsFailure_NoNode() {
-            rc = KeeperException.Code.NoNode;
+            rc = KeeperException.Code.NONODE;
             stat = null;
             stat = null;
             zk.exists(path, false, this, toString());
             zk.exists(path, false, this, toString());
             verify();
             verify();
@@ -464,6 +464,10 @@ public class AsyncOps {
             super(zk, latch);
             super(zk, latch);
         }
         }
         
         
+        public void processResult(int rc, String path, Object ctx) {
+            super.processResult(Code.get(rc), path, ctx);
+        }
+
         public void delete() {
         public void delete() {
             zk.delete(path, version, this, toString());
             zk.delete(path, version, this, toString());
         }
         }
@@ -476,7 +480,7 @@ public class AsyncOps {
         }
         }
         
         
         public void verifyDeleteFailure_NoNode() {
         public void verifyDeleteFailure_NoNode() {
-            rc = Code.NoNode;
+            rc = Code.NONODE;
             zk.delete(path, version, this, toString());
             zk.delete(path, version, this, toString());
             verify();
             verify();
         }
         }

+ 4 - 4
src/java/test/org/apache/zookeeper/test/AsyncTest.java

@@ -58,7 +58,7 @@ public class AsyncTest extends TestCase
     @Override
     @Override
     protected void setUp() throws Exception {
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
         LOG.info("STARTING " + getName());
-        
+
         ClientBase.setupTestEnv();
         ClientBase.setupTestEnv();
 
 
         quorumTest.setUp();
         quorumTest.setUp();
@@ -85,7 +85,7 @@ public class AsyncTest extends TestCase
             }
             }
         }
         }
     }
     }
-    
+
     private ZooKeeper createClient() throws IOException,InterruptedException {
     private ZooKeeper createClient() throws IOException,InterruptedException {
         return createClient(quorumTest.hostPort);
         return createClient(quorumTest.hostPort);
     }
     }
@@ -220,7 +220,7 @@ public class AsyncTest extends TestCase
                 }
                 }
             }
             }
             assertEquals(0, (int) results.get(0));
             assertEquals(0, (int) results.get(0));
-            assertEquals(Code.NoAuth, (int) results.get(1));
+            assertEquals(Code.NOAUTH, Code.get((int) results.get(1)));
             assertEquals(0, (int) results.get(2));
             assertEquals(0, (int) results.get(2));
             assertEquals(0, (int) results.get(3));
             assertEquals(0, (int) results.get(3));
             assertEquals(0, (int) results.get(4));
             assertEquals(0, (int) results.get(4));
@@ -235,7 +235,7 @@ public class AsyncTest extends TestCase
                 zk.getData("/ben2", false, new Stat());
                 zk.getData("/ben2", false, new Stat());
                 fail("Should have received a permission error");
                 fail("Should have received a permission error");
             } catch (KeeperException e) {
             } catch (KeeperException e) {
-                assertEquals(Code.NoAuth, e.getCode());
+                assertEquals(Code.NOAUTH, e.code());
             }
             }
         } finally {
         } finally {
             zk.close();
             zk.close();

+ 39 - 39
src/java/test/org/apache/zookeeper/test/ClientTest.java

@@ -125,7 +125,7 @@ public class ClientTest extends ClientBase {
                 zk.getData("/acltest", false, new Stat());
                 zk.getData("/acltest", false, new Stat());
                 fail("Should have received a permission error");
                 fail("Should have received a permission error");
             } catch (KeeperException e) {
             } catch (KeeperException e) {
-                assertEquals(Code.NoAuth, e.getCode());
+                assertEquals(Code.NOAUTH, e.code());
             }
             }
             zk.addAuthInfo("digest", "ben:passwd".getBytes());
             zk.addAuthInfo("digest", "ben:passwd".getBytes());
             zk.getData("/acltest", false, new Stat());
             zk.getData("/acltest", false, new Stat());
@@ -159,7 +159,7 @@ public class ClientTest extends ClientBase {
             }
             }
         }
         }
     }
     }
-    
+
     /**
     /**
      * Register multiple watchers and verify that they all get notified and
      * Register multiple watchers and verify that they all get notified and
      * in the right order.
      * in the right order.
@@ -201,13 +201,13 @@ public class ClientTest extends ClientBase {
                 assertEquals("/foo-" + i, event.getPath());
                 assertEquals("/foo-" + i, event.getPath());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
                 assertEquals(KeeperState.SyncConnected, event.getState());
-                
+
                 // small chance that an unexpected message was delivered
                 // small chance that an unexpected message was delivered
                 //  after this check, but we would catch that next time
                 //  after this check, but we would catch that next time
                 //  we check events
                 //  we check events
                 assertEquals(0, watchers[i].events.size());
                 assertEquals(0, watchers[i].events.size());
             }
             }
-            
+
             //
             //
             // test get/exists with single set of watchers
             // test get/exists with single set of watchers
             //  get/exists together
             //  get/exists together
@@ -227,13 +227,13 @@ public class ClientTest extends ClientBase {
                 assertEquals("/foo-" + i, event.getPath());
                 assertEquals("/foo-" + i, event.getPath());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
                 assertEquals(KeeperState.SyncConnected, event.getState());
-                
+
                 // small chance that an unexpected message was delivered
                 // small chance that an unexpected message was delivered
                 //  after this check, but we would catch that next time
                 //  after this check, but we would catch that next time
                 //  we check events
                 //  we check events
                 assertEquals(0, watchers[i].events.size());
                 assertEquals(0, watchers[i].events.size());
             }
             }
-            
+
             //
             //
             // test get/exists with two sets of watchers
             // test get/exists with two sets of watchers
             //
             //
@@ -252,7 +252,7 @@ public class ClientTest extends ClientBase {
                 assertEquals("/foo-" + i, event.getPath());
                 assertEquals("/foo-" + i, event.getPath());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(EventType.NodeDataChanged, event.getType());
                 assertEquals(KeeperState.SyncConnected, event.getState());
                 assertEquals(KeeperState.SyncConnected, event.getState());
-                
+
                 // small chance that an unexpected message was delivered
                 // small chance that an unexpected message was delivered
                 //  after this check, but we would catch that next time
                 //  after this check, but we would catch that next time
                 //  we check events
                 //  we check events
@@ -264,13 +264,13 @@ public class ClientTest extends ClientBase {
                 assertEquals("/foo-" + i, event2.getPath());
                 assertEquals("/foo-" + i, event2.getPath());
                 assertEquals(EventType.NodeDataChanged, event2.getType());
                 assertEquals(EventType.NodeDataChanged, event2.getType());
                 assertEquals(KeeperState.SyncConnected, event2.getState());
                 assertEquals(KeeperState.SyncConnected, event2.getState());
-                
+
                 // small chance that an unexpected message was delivered
                 // small chance that an unexpected message was delivered
                 //  after this check, but we would catch that next time
                 //  after this check, but we would catch that next time
                 //  we check events
                 //  we check events
                 assertEquals(0, watchers2[i].events.size());
                 assertEquals(0, watchers2[i].events.size());
             }
             }
-            
+
         } finally {
         } finally {
             if (zk != null) {
             if (zk != null) {
                 zk.close();
                 zk.close();
@@ -303,7 +303,7 @@ public class ClientTest extends ClientBase {
             zk.close();
             zk.close();
             //LOG.info("Closed client: " + zk.describeCNXN());
             //LOG.info("Closed client: " + zk.describeCNXN());
             Thread.sleep(2000);
             Thread.sleep(2000);
-            
+
             zk = createClient(watcher, hostPort);
             zk = createClient(watcher, hostPort);
             //LOG.info("Created a new client: " + zk.describeCNXN());
             //LOG.info("Created a new client: " + zk.describeCNXN());
             LOG.info("Before delete /");
             LOG.info("Before delete /");
@@ -392,7 +392,7 @@ public class ClientTest extends ClientBase {
             //    zk.create("/bad\u0000path", "".getBytes(), null, CreateMode.PERSISTENT);
             //    zk.create("/bad\u0000path", "".getBytes(), null, CreateMode.PERSISTENT);
             //    fail("created an invalid path");
             //    fail("created an invalid path");
             //} catch(KeeperException e) {
             //} catch(KeeperException e) {
-            //    assertEquals(KeeperException.Code.BadArguments, e.getCode());
+            //    assertEquals(KeeperException.Code.BadArguments, e.code());
             //}
             //}
 
 
             zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
             zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -479,19 +479,19 @@ public class ClientTest extends ClientBase {
             zk.delete("/parent", -1);
             zk.delete("/parent", -1);
             fail("Should have received a not equals message");
             fail("Should have received a not equals message");
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NotEmpty, e.getCode());
+            assertEquals(KeeperException.Code.NOTEMPTY, e.code());
         }
         }
         zk.delete("/parent/child", -1);
         zk.delete("/parent/child", -1);
         zk.delete("/parent", -1);
         zk.delete("/parent", -1);
         zk.close();
         zk.close();
     }
     }
-    
+
     private static final long HAMMERTHREAD_LATENCY = 5;
     private static final long HAMMERTHREAD_LATENCY = 5;
-    
+
     private static abstract class HammerThread extends Thread {
     private static abstract class HammerThread extends Thread {
         protected final int count;
         protected final int count;
         protected volatile int current = 0;
         protected volatile int current = 0;
-        
+
         HammerThread(String name, int count) {
         HammerThread(String name, int count) {
             super(name);
             super(name);
             this.count = count;
             this.count = count;
@@ -571,7 +571,7 @@ public class ClientTest extends ClientBase {
         try {
         try {
             final int threadCount = 10;
             final int threadCount = 10;
             final int childCount = 1000;
             final int childCount = 1000;
-            
+
             HammerThread[] threads = new HammerThread[threadCount];
             HammerThread[] threads = new HammerThread[threadCount];
             long start = System.currentTimeMillis();
             long start = System.currentTimeMillis();
             for (int i = 0; i < threads.length; i++) {
             for (int i = 0; i < threads.length; i++) {
@@ -579,21 +579,21 @@ public class ClientTest extends ClientBase {
                 String prefix = "/test-" + i;
                 String prefix = "/test-" + i;
                 zk.create(prefix, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
                 zk.create(prefix, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
                 prefix += "/";
                 prefix += "/";
-                HammerThread thread = 
+                HammerThread thread =
                     new BasicHammerThread("BasicHammerThread-" + i, zk, prefix,
                     new BasicHammerThread("BasicHammerThread-" + i, zk, prefix,
                             childCount);
                             childCount);
                 thread.start();
                 thread.start();
-                
+
                 threads[i] = thread;
                 threads[i] = thread;
             }
             }
-            
+
             verifyHammer(start, threads, childCount);
             verifyHammer(start, threads, childCount);
         } catch (Throwable t) {
         } catch (Throwable t) {
             LOG.error("test failed", t);
             LOG.error("test failed", t);
             throw t;
             throw t;
         }
         }
     }
     }
-    
+
     /**
     /**
      * Separate threads each creating a number of nodes. Each thread
      * Separate threads each creating a number of nodes. Each thread
      * is creating a new client for each node creation.
      * is creating a new client for each node creation.
@@ -604,7 +604,7 @@ public class ClientTest extends ClientBase {
         try {
         try {
             final int threadCount = 5;
             final int threadCount = 5;
             final int childCount = 10;
             final int childCount = 10;
-            
+
             HammerThread[] threads = new HammerThread[threadCount];
             HammerThread[] threads = new HammerThread[threadCount];
             long start = System.currentTimeMillis();
             long start = System.currentTimeMillis();
             for (int i = 0; i < threads.length; i++) {
             for (int i = 0; i < threads.length; i++) {
@@ -618,23 +618,23 @@ public class ClientTest extends ClientBase {
                     }
                     }
                 }
                 }
                 prefix += "/";
                 prefix += "/";
-                HammerThread thread = 
+                HammerThread thread =
                     new SuperHammerThread("SuperHammerThread-" + i, this,
                     new SuperHammerThread("SuperHammerThread-" + i, this,
                             prefix, childCount);
                             prefix, childCount);
                 thread.start();
                 thread.start();
-                
+
                 threads[i] = thread;
                 threads[i] = thread;
             }
             }
-            
+
             verifyHammer(start, threads, childCount);
             verifyHammer(start, threads, childCount);
         } catch (Throwable t) {
         } catch (Throwable t) {
             LOG.error("test failed", t);
             LOG.error("test failed", t);
             throw t;
             throw t;
         }
         }
     }
     }
-    
-    public void verifyHammer(long start, HammerThread[] threads, int childCount) 
-        throws IOException, InterruptedException, KeeperException 
+
+    public void verifyHammer(long start, HammerThread[] threads, int childCount)
+        throws IOException, InterruptedException, KeeperException
     {
     {
         // look for the clients to finish their create operations
         // look for the clients to finish their create operations
         LOG.info("Starting check for completed hammers");
         LOG.info("Starting check for completed hammers");
@@ -653,25 +653,25 @@ public class ClientTest extends ClientBase {
         }
         }
         if (workingCount > 0) {
         if (workingCount > 0) {
             for (HammerThread h : threads) {
             for (HammerThread h : threads) {
-                LOG.warn(h.getName() + " never finished creation, current:" 
+                LOG.warn(h.getName() + " never finished creation, current:"
                         + h.current);
                         + h.current);
             }
             }
         } else {
         } else {
             LOG.info("Hammer threads completed creation operations");
             LOG.info("Hammer threads completed creation operations");
         }
         }
-        
+
         for (HammerThread h : threads) {
         for (HammerThread h : threads) {
             final int safetyFactor = 3;
             final int safetyFactor = 3;
             verifyThreadTerminated(h,
             verifyThreadTerminated(h,
-                    threads.length * childCount 
+                    threads.length * childCount
                     * HAMMERTHREAD_LATENCY * safetyFactor);
                     * HAMMERTHREAD_LATENCY * safetyFactor);
         }
         }
         LOG.info(new Date() + " Total time "
         LOG.info(new Date() + " Total time "
                 + (System.currentTimeMillis() - start));
                 + (System.currentTimeMillis() - start));
-        
+
         ZooKeeper zk = createClient();
         ZooKeeper zk = createClient();
         try {
         try {
-            
+
             LOG.info("******************* Connected to ZooKeeper" + new Date());
             LOG.info("******************* Connected to ZooKeeper" + new Date());
             for (int i = 0; i < threads.length; i++) {
             for (int i = 0; i < threads.length; i++) {
                 LOG.info("Doing thread: " + i + " " + new Date());
                 LOG.info("Doing thread: " + i + " " + new Date());
@@ -688,16 +688,16 @@ public class ClientTest extends ClientBase {
             zk.close();
             zk.close();
         }
         }
     }
     }
-    
+
     private class VerifyClientCleanup extends Thread {
     private class VerifyClientCleanup extends Thread {
         int count;
         int count;
         int current = 0;
         int current = 0;
-        
+
         VerifyClientCleanup(String name, int count) {
         VerifyClientCleanup(String name, int count) {
             super(name);
             super(name);
             this.count = count;
             this.count = count;
         }
         }
-        
+
         public void run() {
         public void run() {
             try {
             try {
                 for (; current < count; current++) {
                 for (; current < count; current++) {
@@ -714,21 +714,21 @@ public class ClientTest extends ClientBase {
      * Verify that the client is cleaning up properly. Open/close a large
      * Verify that the client is cleaning up properly. Open/close a large
      * number of sessions. Essentially looking to see if sockets/selectors
      * number of sessions. Essentially looking to see if sockets/selectors
      * are being cleaned up properly during close.
      * are being cleaned up properly during close.
-     * 
+     *
      * @throws Throwable
      * @throws Throwable
      */
      */
     @Test
     @Test
     public void testClientCleanup() throws Throwable {
     public void testClientCleanup() throws Throwable {
         final int threadCount = 20;
         final int threadCount = 20;
         final int clientCount = 100;
         final int clientCount = 100;
-        
+
         VerifyClientCleanup threads[] = new VerifyClientCleanup[threadCount];
         VerifyClientCleanup threads[] = new VerifyClientCleanup[threadCount];
-        
+
         for (int i = 0; i < threads.length; i++) {
         for (int i = 0; i < threads.length; i++) {
             threads[i] = new VerifyClientCleanup("VCC" + i, clientCount);
             threads[i] = new VerifyClientCleanup("VCC" + i, clientCount);
             threads[i].start();
             threads[i].start();
         }
         }
-        
+
         for (int i = 0; i < threads.length; i++) {
         for (int i = 0; i < threads.length; i++) {
             threads[i].join(600000);
             threads[i].join(600000);
             assertTrue(threads[i].current == threads[i].count);
             assertTrue(threads[i].current == threads[i].count);

+ 2 - 2
src/java/test/org/apache/zookeeper/test/CreateModeTest.java

@@ -68,14 +68,14 @@ public class CreateModeTest extends TestCase {
             CreateMode cm = CreateMode.fromFlag(99);
             CreateMode cm = CreateMode.fromFlag(99);
             fail("Shouldn't be able to convert 99 to a CreateMode.");
             fail("Shouldn't be able to convert 99 to a CreateMode.");
         } catch(KeeperException ke) {
         } catch(KeeperException ke) {
-            assertEquals(Code.BadArguments, ke.getCode());
+            assertEquals(Code.BADARGUMENTS, ke.code());
         }
         }
 
 
         try {
         try {
             CreateMode cm = CreateMode.fromFlag(-1);
             CreateMode cm = CreateMode.fromFlag(-1);
             fail("Shouldn't be able to convert -1 to a CreateMode.");
             fail("Shouldn't be able to convert -1 to a CreateMode.");
         } catch(KeeperException ke) {
         } catch(KeeperException ke) {
-            assertEquals(Code.BadArguments, ke.getCode());
+            assertEquals(Code.BADARGUMENTS, ke.code());
         }
         }
     }
     }
 }
 }

+ 10 - 10
src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java

@@ -140,7 +140,7 @@ public class WatcherFuncTest extends ClientBase {
             client.setData("/car", "missing".getBytes(), -1);
             client.setData("/car", "missing".getBytes(), -1);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
 
 
         try {
         try {
@@ -148,7 +148,7 @@ public class WatcherFuncTest extends ClientBase {
             client.setData("/foo/car", "missing".getBytes(), -1);
             client.setData("/foo/car", "missing".getBytes(), -1);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
 
 
         client.setData("/foo", "parent".getBytes(), -1);
         client.setData("/foo", "parent".getBytes(), -1);
@@ -176,13 +176,13 @@ public class WatcherFuncTest extends ClientBase {
             lsnr.getData("/foo", true, null);
             lsnr.getData("/foo", true, null);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
         try {
         try {
             lsnr.getData("/foo/bar", true, null);
             lsnr.getData("/foo/bar", true, null);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
 
 
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -215,13 +215,13 @@ public class WatcherFuncTest extends ClientBase {
             lsnr.getChildren("/foo", true);
             lsnr.getChildren("/foo", true);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
         try {
         try {
             lsnr.getChildren("/foo/bar", true);
             lsnr.getChildren("/foo/bar", true);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
 
 
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -340,13 +340,13 @@ public class WatcherFuncTest extends ClientBase {
             lsnr.getData("/foo", w1, null);
             lsnr.getData("/foo", w1, null);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
         try {
         try {
             lsnr.getData("/foo/bar", w2, null);
             lsnr.getData("/foo/bar", w2, null);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
 
 
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -406,13 +406,13 @@ public class WatcherFuncTest extends ClientBase {
             lsnr.getChildren("/foo", true);
             lsnr.getChildren("/foo", true);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
         try {
         try {
             lsnr.getChildren("/foo/bar", true);
             lsnr.getChildren("/foo/bar", true);
             fail();
             fail();
         } catch (KeeperException e) {
         } catch (KeeperException e) {
-            assertEquals(KeeperException.Code.NoNode, e.getCode());
+            assertEquals(KeeperException.Code.NONODE, e.code());
         }
         }
 
 
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

+ 24 - 24
src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java

@@ -122,8 +122,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     try {
     try {
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NodeExists;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NODEEXISTS;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for createin: " + ke.getMessage());
         fail("Unexpected exception code for createin: " + ke.getMessage());
       }
       }
@@ -179,9 +179,9 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
       try {
       try {
         zk.delete(nodeName, -1);
         zk.delete(nodeName, -1);
       } catch (KeeperException ke) {
       } catch (KeeperException ke) {
-        int code = ke.getCode();
-        boolean valid = code == KeeperException.Code.NoNode
-            || code == KeeperException.Code.NotEmpty;
+        Code code = ke.code();
+        boolean valid = code == KeeperException.Code.NONODE
+            || code == KeeperException.Code.NOTEMPTY;
         if (!valid) {
         if (!valid) {
           fail("Unexpected exception code for delete: " + ke.getMessage());
           fail("Unexpected exception code for delete: " + ke.getMessage());
         }
         }
@@ -193,8 +193,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     try {
     try {
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NodeExists;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NODEEXISTS;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for createin: " + ke.getMessage());
         fail("Unexpected exception code for createin: " + ke.getMessage());
       }
       }
@@ -219,8 +219,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
       zk.delete(parentName, -1);
       zk.delete(parentName, -1);
       fail("Should be impossible to delete a non-empty node " + parentName);
       fail("Should be impossible to delete a non-empty node " + parentName);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NotEmpty;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NOTEMPTY;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for delete: " + code);
         fail("Unexpected exception code for delete: " + code);
       }
       }
@@ -230,8 +230,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
       zk.create(nodeName + "/def", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
       zk.create(nodeName + "/def", null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
       fail("Should be impossible to create child off Ephemeral node " + nodeName);
       fail("Should be impossible to create child off Ephemeral node " + nodeName);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NoChildrenForEphemerals;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NOCHILDRENFOREPHEMERALS;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for createin: " + code);
         fail("Unexpected exception code for createin: " + code);
       }
       }
@@ -243,8 +243,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
         fail("ephemeral node " + nodeName + " should not have children");
         fail("ephemeral node " + nodeName + " should not have children");
       }
       }
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NoNode;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NONODE;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for createin: " + code);
         fail("Unexpected exception code for createin: " + code);
       }
       }
@@ -308,9 +308,9 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     try {
     try {
       zk.delete(nodeName, -1);
       zk.delete(nodeName, -1);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NoNode
-          || code == KeeperException.Code.NotEmpty;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NONODE
+          || code == KeeperException.Code.NOTEMPTY;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for delete: " + ke.getMessage());
         fail("Unexpected exception code for delete: " + ke.getMessage());
       }
       }
@@ -318,8 +318,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     try {
     try {
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
       zk.create(nodeName, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NodeExists;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NODEEXISTS;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for create: " + ke.getMessage());
         fail("Unexpected exception code for create: " + ke.getMessage());
       }
       }
@@ -328,7 +328,7 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
       zk.setData(nodeName, "hi".getBytes(), 5700);
       zk.setData(nodeName, "hi".getBytes(), 5700);
       fail("Should have gotten BadVersion exception");
       fail("Should have gotten BadVersion exception");
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      if (ke.getCode() != Code.BadVersion) {
+      if (ke.code() != Code.BADVERSION) {
         fail("Should have gotten BadVersion exception");
         fail("Should have gotten BadVersion exception");
       }
       }
     }
     }
@@ -344,9 +344,9 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
       zk.delete(nodeName, 6800);
       zk.delete(nodeName, 6800);
       fail("Should have gotten BadVersion exception");
       fail("Should have gotten BadVersion exception");
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NotEmpty
-          || code == KeeperException.Code.BadVersion;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NOTEMPTY
+          || code == KeeperException.Code.BADVERSION;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for delete: " + ke.getMessage());
         fail("Unexpected exception code for delete: " + ke.getMessage());
       }
       }
@@ -354,8 +354,8 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     try {
     try {
       zk.delete(nodeName, -1);
       zk.delete(nodeName, -1);
     } catch (KeeperException ke) {
     } catch (KeeperException ke) {
-      int code = ke.getCode();
-      boolean valid = code == KeeperException.Code.NotEmpty;
+      Code code = ke.code();
+      boolean valid = code == KeeperException.Code.NOTEMPTY;
       if (!valid) {
       if (!valid) {
         fail("Unexpected exception code for delete: " + code);
         fail("Unexpected exception code for delete: " + code);
       }
       }