浏览代码

ZOOKEEPER-844. handle auth failure in java client

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@1005101 13f79535-47bb-0310-9956-ffa450edef68
Patrick D. Hunt 14 年之前
父节点
当前提交
b6c3c7a94b

+ 3 - 0
CHANGES.txt

@@ -104,6 +104,9 @@ BUGFIXES:
   ZOOKEEPER-804. c unit tests failing due to "assertion cptr failed" (michi
   mutsuzaki via mahadev)
 
+  ZOOKEEPER-844. handle auth failure in java client
+  (Camille Fournier via phunt)
+
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 
   (phunt via mahadev)

+ 6 - 2
src/java/main/org/apache/zookeeper/ClientCnxn.java

@@ -748,8 +748,12 @@ public class ClientCnxn {
                 return;
             }
             if (replyHdr.getXid() == -4) {
-                // -2 is the xid for AuthPacket
-                // TODO: process AuthPacket here
+                // -4 is the xid for AuthPacket               
+                if(replyHdr.getErr() == KeeperException.Code.AUTHFAILED.intValue()) {
+                    zooKeeper.state = States.AUTH_FAILED;                    
+                    eventThread.queueEvent( new WatchedEvent(Watcher.Event.EventType.None, 
+                            Watcher.Event.KeeperState.AuthFailed, null) );            		            		
+                }
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Got auth sessionid:0x"
                             + Long.toHexString(sessionId));

+ 8 - 1
src/java/main/org/apache/zookeeper/Watcher.java

@@ -58,7 +58,13 @@ public interface Watcher {
              * client connection (the session) is no longer valid. You must
              * create a new client connection (instantiate a new ZooKeeper
              * instance) if you with to access the ensemble. */
-            Expired (-112);
+            Expired (-112),
+
+            /**
+             * Auth failed state
+             * 
+             */
+            AuthFailed(4);
 
             private final int intValue;     // Integer representation of value
                                             // for sending over wire
@@ -77,6 +83,7 @@ public interface Watcher {
                     case    0: return KeeperState.Disconnected;
                     case    1: return KeeperState.NoSyncConnected;
                     case    3: return KeeperState.SyncConnected;
+                    case    4: return KeeperState.AuthFailed;
                     case -112: return KeeperState.Expired;
 
                     default:

+ 45 - 0
src/java/test/org/apache/zookeeper/test/AuthTest.java

@@ -18,9 +18,15 @@
 
 package org.apache.zookeeper.test;
 
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.TestableZooKeeper;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.junit.Assert;
 import org.junit.Test;
@@ -32,6 +38,45 @@ public class AuthTest extends ClientBase {
                 "super:D/InIHSb7yEEbrWz8b9l71RjZJU=");        
     }
 
+    private AtomicInteger authFailed = new AtomicInteger(0);
+    
+    @Override
+    protected TestableZooKeeper createClient(String hp)
+    throws IOException, InterruptedException
+    {
+        MyWatcher watcher = new MyWatcher();
+        return createClient(watcher, hp);
+    }
+
+    private class MyWatcher extends CountdownWatcher {
+        @Override
+        public synchronized void process(WatchedEvent event) {
+            if (event.getState() == KeeperState.AuthFailed) {
+                authFailed.incrementAndGet();
+            }
+            else {
+                super.process(event);
+            }
+        }
+    }
+
+    @Test
+    public void testBadAuthNotifiesWatch() throws Exception {
+        ZooKeeper zk = createClient();
+        try {
+            zk.addAuthInfo("FOO", "BAR".getBytes());
+            zk.getData("/path1", false, null);
+            Assert.fail("Should get auth state error");
+        } catch(KeeperException.AuthFailedException e) {
+            Assert.assertEquals("Should have called my watcher", 
+                    1, authFailed.get());
+        }
+        finally {
+            zk.close();
+        }
+    }
+
+    
     @Test
     public void testSuper() throws Exception {
         ZooKeeper zk = createClient();