|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.zookeeper.test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.concurrent.BlockingQueue;
|
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
@@ -32,6 +33,7 @@ import org.apache.zookeeper.WatchedEvent;
|
|
|
import org.apache.zookeeper.ZooKeeper;
|
|
|
import org.apache.zookeeper.AsyncCallback.StatCallback;
|
|
|
import org.apache.zookeeper.AsyncCallback.VoidCallback;
|
|
|
+import org.apache.zookeeper.Watcher;
|
|
|
import org.apache.zookeeper.Watcher.Event;
|
|
|
import org.apache.zookeeper.Watcher.Event.EventType;
|
|
|
import org.apache.zookeeper.ZooDefs.Ids;
|
|
@@ -140,6 +142,57 @@ public class WatcherTest extends ClientBase {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testWatcherDisconnectOnClose()
|
|
|
+ throws IOException, InterruptedException, KeeperException
|
|
|
+ {
|
|
|
+ ZooKeeper zk = null;
|
|
|
+ try {
|
|
|
+ final BlockingQueue<WatchedEvent> queue = new LinkedBlockingQueue<>();
|
|
|
+
|
|
|
+ MyWatcher connWatcher = new MyWatcher();
|
|
|
+
|
|
|
+ Watcher watcher = new Watcher(){
|
|
|
+ @Override
|
|
|
+ public void process(WatchedEvent event) {
|
|
|
+ try {
|
|
|
+ queue.put(event);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ // Oh well, never mind
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ };
|
|
|
+
|
|
|
+ zk = createClient(connWatcher, hostPort);
|
|
|
+
|
|
|
+ StatCallback scb = new StatCallback() {
|
|
|
+ public void processResult(int rc, String path, Object ctx,
|
|
|
+ Stat stat) {
|
|
|
+ // don't do anything
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ // Register a watch on the node
|
|
|
+ zk.exists("/missing", watcher, scb, null);
|
|
|
+
|
|
|
+ // Close the client without changing the node
|
|
|
+ zk.close();
|
|
|
+
|
|
|
+
|
|
|
+ WatchedEvent event = queue.poll(10, TimeUnit.SECONDS);
|
|
|
+
|
|
|
+ Assert.assertNotNull("No watch event was received after closing the Zookeeper client. A 'Closed' event should have occurred", event);
|
|
|
+ Assert.assertEquals("Closed events are not generated by the server, and so should have a type of 'None'", Event.EventType.None, event.getType());
|
|
|
+ Assert.assertEquals("A 'Closed' event was expected as the Zookeeper client was closed without altering the node it was watching", Event.KeeperState.Closed, event.getState());
|
|
|
+ } finally {
|
|
|
+ if (zk != null) {
|
|
|
+ zk.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
public void testWatcherCount()
|