You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by br...@apache.org on 2008/09/24 23:12:26 UTC

svn commit: r698738 - in /hadoop/zookeeper/trunk/src/java: main/org/apache/zookeeper/ClientWatchManager.java test/org/apache/zookeeper/test/WatcherTest.java

Author: breed
Date: Wed Sep 24 14:12:26 2008
New Revision: 698738

URL: http://svn.apache.org/viewvc?rev=698738&view=rev
Log:
Missed adding two files for ZOOKEEPER-137

Added:
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java

Added: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java?rev=698738&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java (added)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java Wed Sep 24 14:12:26 2008
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import java.util.Set;
+
+import org.apache.zookeeper.proto.WatcherEvent;
+
+/**
+ */
+public interface ClientWatchManager {
+    /**
+     * Return a set of watchers that should be notified of the event. The 
+     * manager must not notify the watcher(s), however it will update it's 
+     * internal structure as if the watches had triggered. The intent being 
+     * that the callee is now responsible for notifying the watchers of the 
+     * event, possibly at some later time.
+     * 
+     * @param state event state
+     * @param type event type
+     * @param path event path
+     * @return
+     */
+    public Set<Watcher> materialize(int state, int type, String path);
+}

Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java?rev=698738&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java Wed Sep 24 14:12:26 2008
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import java.io.IOException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
+import org.apache.zookeeper.AsyncCallback.VoidCallback;
+import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.proto.WatcherEvent;
+import org.junit.Test;
+
+public class WatcherTest extends ClientBase {
+    protected static final Logger LOG = Logger.getLogger(WatcherTest.class);
+
+    private class MyWatcher extends CountdownWatcher {
+        LinkedBlockingQueue<WatcherEvent> events =
+            new LinkedBlockingQueue<WatcherEvent>();
+
+        public void process(WatcherEvent event) {
+            super.process(event);
+            if (event.getType() != Event.EventNone) {
+                try {
+                    events.put(event);
+                } catch (InterruptedException e) {
+                    LOG.warn("ignoring interrupt during event.put");
+                }
+            }
+        }
+    }
+
+    /**
+     * Verify that we get all of the events we expect to get. This particular
+     * case verifies that we see all of the data events on a particular node.
+     * There was a bug (ZOOKEEPER-137) that resulted in events being dropped
+     * in some cases (timing).
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws KeeperException
+     */
+    @Test
+    public void testWatcherCorrectness()
+        throws IOException, InterruptedException, KeeperException
+    {
+        ZooKeeper zk = null;
+        try {
+            MyWatcher watcher = new MyWatcher();
+            zk = createClient(watcher, hostPort);
+            
+            StatCallback scb = new StatCallback() {
+                public void processResult(int rc, String path, Object ctx,
+                        Stat stat) {
+                    // don't do anything
+                }
+            };
+            VoidCallback vcb = new VoidCallback() {
+                public void processResult(int rc, String path, Object ctx) {
+                    // don't do anything
+                }
+            };
+            
+            String names[] = new String[10];
+            for (int i = 0; i < names.length; i++) {
+                String name = zk.create("/tc-", "initialvalue".getBytes(),
+                        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
+                names[i] = name;
+                System.out.println(name);
+    
+                Stat stat = new Stat();
+                zk.getData(name, watcher, stat);
+                zk.setData(name, "new".getBytes(), stat.getVersion(), scb, null);
+                stat = zk.exists(name, watcher);
+                zk.delete(name, stat.getVersion(), vcb, null);
+            }
+            
+            for (int i = 0; i < names.length; i++) {
+                String name = names[i];
+                WatcherEvent event = watcher.events.poll(10, TimeUnit.SECONDS);
+                assertEquals(name, event.getPath());
+                assertEquals(Event.EventNodeDataChanged, event.getType());
+                assertEquals(Event.KeeperStateSyncConnected, event.getState());
+                event = watcher.events.poll(10, TimeUnit.SECONDS);
+                assertEquals(name, event.getPath());
+                assertEquals(Event.EventNodeDeleted, event.getType());
+                assertEquals(Event.KeeperStateSyncConnected, event.getState());
+            }
+        } finally {
+            if (zk != null) {
+                zk.close();
+            }
+        }
+    }
+
+}