Author: phunt
Date: Thu Sep 25 15:25:28 2008
New Revision: 699114
URL: http://svn.apache.org/viewvc?rev=699114&view=rev
Log:
cleanup event type/state fields, incompatible API change, ZOOKEEPER-139
Added:
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/WatchedEvent.java (with props)
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java (with props)
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/KeeperStateTest.java (with props)
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java (with props)
Modified:
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Watcher.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/WatchManager.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GenerateLoad.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RecoveryTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SledgeHammer.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Thu Sep 25 15:25:28 2008
@@ -45,6 +45,8 @@
import org.apache.zookeeper.AsyncCallback.StringCallback;
import org.apache.zookeeper.AsyncCallback.VoidCallback;
import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.ZooKeeper.States;
import org.apache.zookeeper.ZooKeeper.WatchRegistration;
@@ -253,7 +255,7 @@
eventThread.start();
}
- WatcherEvent eventOfDeath = new WatcherEvent();
+ Object eventOfDeath = new Object();
final static UncaughtExceptionHandler uncaughtExceptionHandler = new UncaughtExceptionHandler() {
public void uncaughtException(Thread t, Throwable e) {
@@ -263,9 +265,9 @@
private class WatcherSetEventPair {
private final Set<Watcher> watchers;
- private final WatcherEvent event;
+ private final WatchedEvent event;
- public WatcherSetEventPair(Set<Watcher> watchers, WatcherEvent event) {
+ public WatcherSetEventPair(Set<Watcher> watchers, WatchedEvent event) {
this.watchers = watchers;
this.event = event;
}
@@ -281,7 +283,7 @@
setDaemon(true);
}
- public void queueEvent(WatcherEvent event) {
+ public void queueEvent(WatchedEvent event) {
// materialize the watchers based on the event
WatcherSetEventPair pair = new WatcherSetEventPair(
watcher.materialize(event.getState(), event.getType(),
@@ -307,6 +309,7 @@
if (event == eventOfDeath) {
break;
}
+
if (event instanceof WatcherSetEventPair) {
// each watcher will process the event
WatcherSetEventPair pair = (WatcherSetEventPair)event;
@@ -459,16 +462,17 @@
int sessionTimeout = conRsp.getTimeOut();
if (sessionTimeout <= 0) {
zooKeeper.state = States.CLOSED;
- eventThread.queueEvent(new WatcherEvent(Watcher.Event.EventNone,
- Watcher.Event.KeeperStateExpired, null));
+
+ eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None,
+ Watcher.Event.KeeperState.Expired, null));
throw new IOException("Session Expired");
}
readTimeout = sessionTimeout * 2 / 3;
connectTimeout = sessionTimeout / serverAddrs.size();
sessionId = conRsp.getSessionId();
sessionPasswd = conRsp.getPasswd();
- eventThread.queueEvent(new WatcherEvent(Watcher.Event.EventNone,
- Watcher.Event.KeeperStateSyncConnected, null));
+ eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None,
+ Watcher.Event.KeeperState.SyncConnected, null));
}
void readResponse() throws IOException {
@@ -497,7 +501,7 @@
+ Long.toHexString(sessionId));
}
- eventThread.queueEvent(event);
+ eventThread.queueEvent( new WatchedEvent(event) );
return;
}
if (pendingQueue.size() == 0) {
@@ -797,9 +801,9 @@
e);
cleanup();
if (zooKeeper.state.isAlive()) {
- eventThread.queueEvent(new WatcherEvent(
- Event.EventNone,
- Event.KeeperStateDisconnected,
+ eventThread.queueEvent(new WatchedEvent(
+ Event.EventType.None,
+ Event.KeeperState.Disconnected,
null));
}
Modified: 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=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientWatchManager.java Thu Sep 25 15:25:28 2008
@@ -20,7 +20,8 @@
import java.util.Set;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
/**
*/
@@ -37,5 +38,5 @@
* @param path event path
* @return
*/
- public Set<Watcher> materialize(int state, int type, String path);
+ public Set<Watcher> materialize(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path);
}
Added: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/WatchedEvent.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/WatchedEvent.java?rev=699114&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/WatchedEvent.java (added)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/WatchedEvent.java Thu Sep 25 15:25:28 2008
@@ -0,0 +1,73 @@
+/**
+ * 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 org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+
+/**
+ * A WatchedEvent represents a change on the ZooKeeper that a Watcher
+ * is able to respond to. The WatchedEvent includes exactly what happened,
+ * the current state of the ZooKeeper, and the path of the znode that
+ * was involved in the event.
+ */
+public class WatchedEvent {
+ final private KeeperState keeperState;
+ final private EventType eventType;
+ private String path;
+
+ /**
+ * Create a WatchedEvent with specified type, state and path
+ */
+ public WatchedEvent(EventType eventType, KeeperState keeperState, String path) {
+ this.keeperState = keeperState;
+ this.eventType = eventType;
+ this.path = path;
+ }
+
+ /**
+ * Convert a WatcherEvent sent over the wire into a full-fledged WatcherEvent
+ */
+ public WatchedEvent(WatcherEvent eventMessage) {
+ keeperState = KeeperState.fromInt(eventMessage.getState());
+ eventType = EventType.fromInt(eventMessage.getType());
+ path = eventMessage.getPath();
+ }
+
+ public KeeperState getState() {
+ return keeperState;
+ }
+
+ public EventType getType() {
+ return eventType;
+ }
+
+ public String getPath() {
+ return path;
+ }
+
+ /**
+ * Convert WatchedEvent to type that can be sent over network
+ */
+ public WatcherEvent getWrapper() {
+ return new WatcherEvent(eventType.getIntValue(),
+ keeperState.getIntValue(),
+ path);
+ }
+}
Propchange: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/WatchedEvent.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Watcher.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Watcher.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Watcher.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/Watcher.java Thu Sep 25 15:25:28 2008
@@ -31,36 +31,79 @@
public interface Watcher {
/**
- *
- * This interface defines the list of possible event codes
- *
+ * This interface defines the possible states an Event may represent
*/
public interface Event {
-
- // constants for connection states
- final public static int KeeperStateChanged = 0;
-
- final public static int KeeperStateUnknown = -1;
-
- final public static int KeeperStateDisconnected = 0;
-
- final public static int KeeperStateNoSyncConnected = 1;
-
- final public static int KeeperStateSyncConnected = 3;
-
- public static final int KeeperStateExpired = -112;
-
- // constants for event types
- final public static int EventNone = -1;
-
- final public static int EventNodeCreated = 1;
-
- final public static int EventNodeDeleted = 2;
-
- final public static int EventNodeDataChanged = 3;
-
- final public static int EventNodeChildrenChanged = 4;
+ /**
+ * Enumeration of states the ZooKeeper may be at the event
+ */
+ public enum KeeperState {
+ Unknown (-1),
+ Disconnected (0),
+ NoSyncConnected (1),
+ SyncConnected (3),
+ Expired (-112);
+
+ private final int intValue; // Integer representation of value
+ // for sending over wire
+
+ KeeperState(int intValue) {
+ this.intValue = intValue;
+ }
+
+ public int getIntValue() {
+ return intValue;
+ }
+
+ public static KeeperState fromInt(int intValue) {
+ switch(intValue) {
+ case -1: return KeeperState.Unknown;
+ case 0: return KeeperState.Disconnected;
+ case 1: return KeeperState.NoSyncConnected;
+ case 3: return KeeperState.SyncConnected;
+ case -112: return KeeperState.Expired;
+
+ default:
+ throw new RuntimeException("Invalid integer value for conversion to KeeperState");
+ }
+ }
+ }
+
+ /**
+ * Enumeration of types of events that may occur on the ZooKeeper
+ */
+ public enum EventType {
+ None (-1),
+ NodeCreated (1),
+ NodeDeleted (2),
+ NodeDataChanged (3),
+ NodeChildrenChanged (4);
+
+ private final int intValue; // Integer representation of value
+ // for sending over wire
+
+ EventType(int intValue) {
+ this.intValue = intValue;
+ }
+
+ public int getIntValue() {
+ return intValue;
+ }
+
+ public static EventType fromInt(int intValue) {
+ switch(intValue) {
+ case -1: return EventType.None;
+ case 1: return EventType.NodeCreated;
+ case 2: return EventType.NodeDeleted;
+ case 3: return EventType.NodeDataChanged;
+ case 4: return EventType.NodeChildrenChanged;
+
+ default:
+ throw new RuntimeException("Invalid integer value for conversion to EventType");
+ }
+ }
+ }
}
- abstract public void process(WatcherEvent event);
+ abstract public void process(WatchedEvent event);
}
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Thu Sep 25 15:25:28 2008
@@ -54,6 +54,8 @@
import org.apache.zookeeper.proto.SyncResponse;
import org.apache.zookeeper.proto.WatcherEvent;
import org.apache.zookeeper.server.DataTree;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
/**
* This is the main class of ZooKeeper client library. To use a ZooKeeper
@@ -124,13 +126,15 @@
private volatile Watcher defaultWatcher;
/* (non-Javadoc)
- * @see org.apache.zookeeper.ClientWatchManager#materialize(int, int, java.lang.String)
+ * @see org.apache.zookeeper.ClientWatchManager#materialize(Event.KeeperState, Event.EventType, java.lang.String)
*/
- public Set<Watcher> materialize(int state, int type, String path) {
+ public Set<Watcher> materialize(Watcher.Event.KeeperState state,
+ Watcher.Event.EventType type, String path) {
Set<Watcher> result = new HashSet<Watcher>();
// clear the watches if we are not connected
- if (state != Watcher.Event.KeeperStateSyncConnected) {
+
+ if (state != Watcher.Event.KeeperState.SyncConnected) {
synchronized (dataWatches) {
for (Set<Watcher> watchers : dataWatches.values()) {
for (Watcher watcher : watchers) {
@@ -152,21 +156,21 @@
Set<Watcher> watchers = null;
switch (type) {
- case Watcher.Event.EventNone:
+ case None:
result.add(defaultWatcher);
return result;
- case Watcher.Event.EventNodeDataChanged:
- case Watcher.Event.EventNodeCreated:
+ case NodeDataChanged:
+ case NodeCreated:
synchronized (dataWatches) {
watchers = dataWatches.remove(path);
}
break;
- case Watcher.Event.EventNodeChildrenChanged:
+ case NodeChildrenChanged:
synchronized (childWatches) {
watchers = childWatches.remove(path);
}
break;
- case Watcher.Event.EventNodeDeleted:
+ case NodeDeleted:
synchronized (dataWatches) {
watchers = dataWatches.remove(path);
}
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java Thu Sep 25 15:25:28 2008
@@ -51,7 +51,7 @@
}
static private class MyWatcher implements Watcher {
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
System.err.println(event.getPath() + ": " + event.getState() + "-"
+ event.getType());
}
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java Thu Sep 25 15:25:28 2008
@@ -32,6 +32,8 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.Watcher.Event;
import org.apache.zookeeper.ZooDefs.OpCode;
@@ -192,8 +194,8 @@
}
}
}
- dataWatches.triggerWatch(path, Event.EventNodeCreated);
- childWatches.triggerWatch(parentName.equals("")?"/":parentName, Event.EventNodeChildrenChanged);
+ dataWatches.triggerWatch(path, Event.EventType.NodeCreated);
+ childWatches.triggerWatch(parentName.equals("")?"/":parentName, Event.EventType.NodeChildrenChanged);
return path;
}
@@ -231,9 +233,9 @@
ZooTrace.EVENT_DELIVERY_TRACE_MASK,
"childWatches.triggerWatch " + parentName);
Set<Watcher> processed =
- dataWatches.triggerWatch(path, Event.EventNodeDeleted);
- childWatches.triggerWatch(path, Event.EventNodeDeleted, processed);
- childWatches.triggerWatch(parentName.equals("")?"/":parentName, Event.EventNodeChildrenChanged);
+ dataWatches.triggerWatch(path, EventType.NodeDeleted);
+ childWatches.triggerWatch(path, EventType.NodeDeleted, processed);
+ childWatches.triggerWatch(parentName.equals("")?"/":parentName, EventType.NodeChildrenChanged);
}
public Stat setData(String path, byte data[], int version, long zxid,
@@ -250,7 +252,7 @@
n.stat.setVersion(version);
copyStat(n.stat, s);
}
- dataWatches.triggerWatch(path, Event.EventNodeDataChanged);
+ dataWatches.triggerWatch(path, EventType.NodeDataChanged);
return s;
}
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java Thu Sep 25 15:25:28 2008
@@ -46,6 +46,7 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Version;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.proto.AuthPacket;
@@ -760,13 +761,17 @@
*
* @see org.apache.zookeeper.server.ServerCnxnIface#process(org.apache.zookeeper.proto.WatcherEvent)
*/
- synchronized public void process(WatcherEvent event) {
+ synchronized public void process(WatchedEvent event) {
ReplyHeader h = new ReplyHeader(-1, -1L, 0);
ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK,
"Deliver event " + event + " to 0x"
+ Long.toHexString(this.sessionId)
+ " through " + this);
- sendResponse(h, event, "notification");
+
+ // Convert WatchedEvent to a type that can be sent over the wire
+ WatcherEvent e = event.getWrapper();
+
+ sendResponse(h, e, "notification");
}
public void finishSessionInit(boolean valid) {
@@ -830,12 +835,15 @@
public long getOutstandingRequests() {
return outstandingRequests;
}
+
public long getPacketsReceived() {
return packetsReceived;
}
+
public long getPacketsSent() {
return packetsSent;
}
+
@Override
public String toString(){
StringBuilder sb=new StringBuilder();
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java Thu Sep 25 15:25:28 2008
@@ -25,6 +25,7 @@
import org.apache.jute.Record;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.proto.ReplyHeader;
import org.apache.zookeeper.proto.WatcherEvent;
@@ -57,7 +58,7 @@
public void finishSessionInit(boolean valid);
- public abstract void process(WatcherEvent event);
+ public abstract void process(WatchedEvent event);
public abstract long getSessionId();
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/WatchManager.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/WatchManager.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/WatchManager.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/WatchManager.java Thu Sep 25 15:25:28 2008
@@ -25,6 +25,9 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.proto.WatcherEvent;
/**
@@ -76,13 +79,13 @@
}
}
- Set<Watcher> triggerWatch(String path, int type) {
+ Set<Watcher> triggerWatch(String path, EventType type) {
return triggerWatch(path, type, null);
}
- Set<Watcher> triggerWatch(String path, int type, Set<Watcher> supress) {
- WatcherEvent e = new WatcherEvent(type,
- Watcher.Event.KeeperStateSyncConnected, path);
+ Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
+ WatchedEvent e = new WatchedEvent(type,
+ KeeperState.SyncConnected, path);
HashSet<Watcher> watchers;
synchronized (this) {
watchers = watchTable.remove(path);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java Thu Sep 25 15:25:28 2008
@@ -31,6 +31,8 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.AsyncCallback.DataCallback;
import org.apache.zookeeper.AsyncCallback.StringCallback;
@@ -39,7 +41,7 @@
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -78,8 +80,8 @@
private static class CountdownWatcher implements Watcher {
volatile CountDownLatch clientConnected = new CountDownLatch(1);
- public void process(WatcherEvent event) {
- if (event.getState() == Event.KeeperStateSyncConnected) {
+ public void process(WatchedEvent event) {
+ if (event.getState() == KeeperState.SyncConnected) {
clientConnected.countDown();
}
}
@@ -157,7 +159,7 @@
notifyAll();
}
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
// ignore for purposes of this test
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java Thu Sep 25 15:25:28 2008
@@ -32,9 +32,11 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.NIOServerCnxn;
import org.apache.zookeeper.server.ServerStats;
import org.apache.zookeeper.server.SyncRequestProcessor;
@@ -66,14 +68,14 @@
*
*/
protected class NullWatcher implements Watcher {
- public void process(WatcherEvent event) { /* nada */ }
+ public void process(WatchedEvent event) { /* nada */ }
}
protected static class CountdownWatcher implements Watcher {
volatile CountDownLatch clientConnected = new CountDownLatch(1);
- public void process(WatcherEvent event) {
- if (event.getState() == Event.KeeperStateSyncConnected) {
+ public void process(WatchedEvent event) {
+ if (event.getState() == KeeperState.SyncConnected) {
clientConnected.countDown();
}
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java Thu Sep 25 15:25:28 2008
@@ -33,12 +33,14 @@
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.KeeperException.InvalidACLException;
import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooDefs.Perms;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.junit.Test;
public class ClientTest extends ClientBase {
protected static final Logger LOG = Logger.getLogger(ClientTest.class);
@@ -143,12 +145,12 @@
}
private class MyWatcher extends CountdownWatcher {
- LinkedBlockingQueue<WatcherEvent> events =
- new LinkedBlockingQueue<WatcherEvent>();
+ LinkedBlockingQueue<WatchedEvent> events =
+ new LinkedBlockingQueue<WatchedEvent>();
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
super.process(event);
- if (event.getType() != Event.EventNone) {
+ if (event.getType() != EventType.None) {
try {
events.put(event);
} catch (InterruptedException e) {
@@ -194,11 +196,11 @@
zk.setData("/foo-" + i, ("foodata3-" + i).getBytes(), -1);
}
for (int i = 0; i < watchers.length; i++) {
- WatcherEvent event =
+ WatchedEvent event =
watchers[i].events.poll(10, TimeUnit.SECONDS);
assertEquals("/foo-" + i, event.getPath());
- assertEquals(Event.EventNodeDataChanged, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeDataChanged, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
// small chance that an unexpected message was delivered
// after this check, but we would catch that next time
@@ -220,11 +222,11 @@
zk.setData("/foo-" + i, ("foodata5-" + i).getBytes(), -1);
}
for (int i = 0; i < watchers.length; i++) {
- WatcherEvent event =
+ WatchedEvent event =
watchers[i].events.poll(10, TimeUnit.SECONDS);
assertEquals("/foo-" + i, event.getPath());
- assertEquals(Event.EventNodeDataChanged, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeDataChanged, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
// small chance that an unexpected message was delivered
// after this check, but we would catch that next time
@@ -245,11 +247,11 @@
zk.setData("/foo-" + i, ("foodata7-" + i).getBytes(), -1);
}
for (int i = 0; i < watchers.length; i++) {
- WatcherEvent event =
+ WatchedEvent event =
watchers[i].events.poll(10, TimeUnit.SECONDS);
assertEquals("/foo-" + i, event.getPath());
- assertEquals(Event.EventNodeDataChanged, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeDataChanged, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
// small chance that an unexpected message was delivered
// after this check, but we would catch that next time
@@ -257,11 +259,11 @@
assertEquals(0, watchers[i].events.size());
// watchers2
- WatcherEvent event2 =
+ WatchedEvent event2 =
watchers2[i].events.poll(10, TimeUnit.SECONDS);
assertEquals("/foo-" + i, event2.getPath());
- assertEquals(Event.EventNodeDataChanged, event2.getType());
- assertEquals(Event.KeeperStateSyncConnected, event2.getState());
+ assertEquals(EventType.NodeDataChanged, event2.getType());
+ assertEquals(KeeperState.SyncConnected, event2.getState());
// small chance that an unexpected message was delivered
// after this check, but we would catch that next time
@@ -339,10 +341,10 @@
// the first poll is just a session delivery
LOG.info("Comment: checking for events length "
+ watcher.events.size());
- WatcherEvent event = watcher.events.poll(10, TimeUnit.SECONDS);
+ WatchedEvent event = watcher.events.poll(10, TimeUnit.SECONDS);
assertEquals("/frog", event.getPath());
- assertEquals(Event.EventNodeCreated, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeCreated, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
// Test child watch and create with sequence
zk.getChildren("/ben", true);
for (int i = 0; i < 10; i++) {
@@ -372,18 +374,18 @@
}
event = watcher.events.poll(10, TimeUnit.SECONDS);
assertEquals("/ben", event.getPath());
- assertEquals(Event.EventNodeChildrenChanged, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeChildrenChanged, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
for (int i = 0; i < 10; i++) {
event = watcher.events.poll(10, TimeUnit.SECONDS);
final String name = children.get(i);
assertEquals("/ben/" + name, event.getPath());
- assertEquals(Event.EventNodeDataChanged, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeDataChanged, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
event = watcher.events.poll(10, TimeUnit.SECONDS);
assertEquals("/ben/" + name, event.getPath());
- assertEquals(Event.EventNodeDeleted, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(EventType.NodeDeleted, event.getType());
+ assertEquals(KeeperState.SyncConnected, event.getState());
}
zk.create("/good\u0001path", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
//try {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java Thu Sep 25 15:25:28 2008
@@ -23,7 +23,7 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.DataTree;
public class DataTreeTest extends TestCase {
@@ -46,7 +46,7 @@
public void testRootWatchTriggered() throws Exception {
class MyWatcher implements Watcher{
boolean fired=false;
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
if(event.getPath().equals("/"))
fired=true;
}
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java?rev=699114&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java Thu Sep 25 15:25:28 2008
@@ -0,0 +1,50 @@
+/**
+ * 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.util.EnumSet;
+
+import junit.framework.TestCase;
+import org.junit.Test;
+
+import org.apache.zookeeper.Watcher.Event.EventType;
+
+public class EventTypeTest extends TestCase {
+
+ @Test
+ public void testIntConversion() {
+ // Ensure that we can convert all valid integers to EventTypes
+ EnumSet<EventType> allTypes = EnumSet.allOf(EventType.class);
+
+ for(EventType et : allTypes) {
+ assertEquals(et, EventType.fromInt( et.getIntValue() ) );
+ }
+ }
+
+ @Test
+ public void testInvalidIntConversion() {
+ try {
+ EventType et = EventType.fromInt(324242);
+ fail("Was able to create an invalid EventType via an integer");
+ } catch(RuntimeException re) {
+ // we're good.
+ }
+
+ }
+}
Propchange: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GenerateLoad.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GenerateLoad.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GenerateLoad.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/GenerateLoad.java Thu Sep 25 15:25:28 2008
@@ -42,11 +42,13 @@
import org.apache.zookeeper.AsyncCallback.StatCallback;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
public class GenerateLoad {
protected static final Logger LOG = Logger.getLogger(GenerateLoad.class);
@@ -340,7 +342,7 @@
}
}
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
System.err.println(event);
synchronized(this) {
try {
@@ -350,7 +352,7 @@
e.printStackTrace();
}
}
- if (event.getType() == Watcher.Event.EventNone && event.getState() == Watcher.Event.KeeperStateExpired) {
+ if (event.getType() == EventType.None && event.getState() == KeeperState.Expired) {
try {
zk = new ZooKeeper(host, 10000, this);
} catch (IOException e) {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java Thu Sep 25 15:25:28 2008
@@ -38,6 +38,7 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.CreateMode;
@@ -45,7 +46,7 @@
import org.apache.zookeeper.AsyncCallback.DataCallback;
import org.apache.zookeeper.AsyncCallback.StatCallback;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
private static final Logger LOG = Logger.getLogger(IntegrityCheck.class);
@@ -147,8 +148,8 @@
}
// watcher callback
- public void process(WatcherEvent event) {
- if(event.getState()==Event.KeeperStateSyncConnected){
+ public void process(WatchedEvent event) {
+ if(event.getState()==KeeperState.SyncConnected){
synchronized(this){
notifyAll();
}
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/KeeperStateTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/KeeperStateTest.java?rev=699114&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/KeeperStateTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/KeeperStateTest.java Thu Sep 25 15:25:28 2008
@@ -0,0 +1,50 @@
+/**
+ * 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.util.EnumSet;
+
+import junit.framework.TestCase;
+import org.junit.Test;
+
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+
+public class KeeperStateTest extends TestCase {
+
+ @Test
+ public void testIntConversion() {
+ // Ensure that we can convert all valid integers to KeeperStates
+ EnumSet<KeeperState> allStates = EnumSet.allOf(KeeperState.class);
+
+ for(KeeperState as : allStates) {
+ assertEquals(as, KeeperState.fromInt( as.getIntValue() ) );
+ }
+ }
+
+ @Test
+ public void testInvalidIntConversion() {
+ try {
+ KeeperState ks = KeeperState.fromInt(324142);
+ fail("Was able to create an invalid KeeperState via an integer");
+ } catch(RuntimeException re) {
+ // we're good.
+ }
+
+ }
+}
Propchange: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/KeeperStateTest.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java Thu Sep 25 15:25:28 2008
@@ -32,7 +32,7 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.NIOServerCnxn;
import org.apache.zookeeper.server.ZooKeeperServer;
@@ -142,7 +142,7 @@
*
* @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.proto.WatcherEvent)
*/
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
System.err.println("Got event " + event.getType() + " "
+ event.getState() + " " + event.getPath());
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RecoveryTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RecoveryTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RecoveryTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RecoveryTest.java Thu Sep 25 15:25:28 2008
@@ -28,20 +28,18 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.NIOServerCnxn;
import org.apache.zookeeper.server.ServerStats;
import org.apache.zookeeper.server.SyncRequestProcessor;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.junit.Test;
-/**
- *
- */
public class RecoveryTest extends TestCase implements Watcher {
protected static final Logger LOG = Logger.getLogger(RecoveryTest.class);
@@ -199,11 +197,11 @@
/*
* (non-Javadoc)
*
- * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.proto.WatcherEvent)
+ * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatcherEvent)
*/
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
- if (event.getState() == Watcher.Event.KeeperStateSyncConnected
+ if (event.getState() == KeeperState.SyncConnected
&& startSignal != null && startSignal.getCount() > 0)
{
startSignal.countDown();
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java Thu Sep 25 15:25:28 2008
@@ -31,9 +31,10 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.NIOServerCnxn;
import org.apache.zookeeper.server.ServerStats;
import org.apache.zookeeper.server.ZooKeeperServer;
@@ -80,8 +81,8 @@
private static class CountdownWatcher implements Watcher {
volatile CountDownLatch clientConnected = new CountDownLatch(1);
- public void process(WatcherEvent event) {
- if (event.getState() == Event.KeeperStateSyncConnected) {
+ public void process(WatchedEvent event) {
+ if (event.getState() == KeeperState.SyncConnected) {
clientConnected.countDown();
}
}
@@ -174,9 +175,9 @@
zk.close();
}
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
- if (event.getState() == Watcher.Event.KeeperStateSyncConnected
+ if (event.getState() == KeeperState.SyncConnected
&& startSignal.getCount() > 0)
{
startSignal.countDown();
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SledgeHammer.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SledgeHammer.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SledgeHammer.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SledgeHammer.java Thu Sep 25 15:25:28 2008
@@ -28,7 +28,7 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
public class SledgeHammer extends Thread implements Watcher {
ZooKeeper zk;
@@ -106,7 +106,7 @@
System.exit(0);
}
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
synchronized (this) {
notifyAll();
}
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java?rev=699114&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java Thu Sep 25 15:25:28 2008
@@ -0,0 +1,95 @@
+/**
+ * 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.util.EnumSet;
+
+import junit.framework.TestCase;
+import org.junit.Test;
+
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.proto.WatcherEvent;
+
+public class WatchedEventTest extends TestCase {
+
+ @Test
+ public void testCreatingWatchedEvent() {
+ // EventWatch is a simple, immutable type, so all we need to do
+ // is make sure we can create all possible combinations of values.
+
+ EnumSet<EventType> allTypes = EnumSet.allOf(EventType.class);
+ EnumSet<KeeperState> allStates = EnumSet.allOf(KeeperState.class);
+ WatchedEvent we;
+
+ for(EventType et : allTypes) {
+ for(KeeperState ks : allStates) {
+ we = new WatchedEvent(et, ks, "blah");
+ assertEquals(et, we.getType());
+ assertEquals(ks, we.getState());
+ assertEquals("blah", we.getPath());
+ }
+
+ }
+ }
+
+ @Test
+ public void testCreatingWatchedEventFromWrapper() {
+ // Make sure we can handle any type of correct wrapper
+
+ EnumSet<EventType> allTypes = EnumSet.allOf(EventType.class);
+ EnumSet<KeeperState> allStates = EnumSet.allOf(KeeperState.class);
+ WatchedEvent we;
+ WatcherEvent wep;
+
+ for(EventType et : allTypes) {
+ for(KeeperState ks : allStates) {
+ wep = new WatcherEvent(et.getIntValue(), ks.getIntValue(), "blah");
+ we = new WatchedEvent(wep);
+ assertEquals(et, we.getType());
+ assertEquals(ks, we.getState());
+ assertEquals("blah", we.getPath());
+ }
+ }
+ }
+
+ @Test
+ public void testCreatingWatchedEventFromInvalidWrapper() {
+ // Make sure we can't convert from an invalid wrapper
+
+ try {
+ WatcherEvent wep = new WatcherEvent(-2342, -252352, "foo");
+ WatchedEvent we = new WatchedEvent(wep);
+ fail("Was able to create WatchedEvent from bad wrapper");
+ } catch (RuntimeException re) {
+ // we're good
+ }
+ }
+
+ @Test
+ public void testConvertingToEventWrapper() {
+ WatchedEvent we = new WatchedEvent(EventType.NodeCreated, KeeperState.Expired, "blah");
+ WatcherEvent wew = we.getWrapper();
+
+ assertEquals(EventType.NodeCreated.getIntValue(), wew.getType());
+ assertEquals(KeeperState.Expired.getIntValue(), wew.getState());
+ assertEquals("blah", wew.getPath());
+ }
+}
Propchange: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java Thu Sep 25 15:25:28 2008
@@ -27,29 +27,31 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
public class WatcherFuncTest extends ClientBase {
private static class SimpleWatcher implements Watcher {
- private LinkedBlockingQueue<WatcherEvent> events =
- new LinkedBlockingQueue<WatcherEvent>();
+ private LinkedBlockingQueue<WatchedEvent> events =
+ new LinkedBlockingQueue<WatchedEvent>();
private CountDownLatch latch;
public SimpleWatcher(CountDownLatch latch) {
this.latch = latch;
}
- public void process(WatcherEvent event) {
- if (event.getState() == Event.KeeperStateSyncConnected) {
+ public void process(WatchedEvent event) {
+ if (event.getState() == KeeperState.SyncConnected) {
if (latch != null) {
latch.countDown();
}
}
- if (event.getType() == Watcher.Event.EventNone) {
+ if (event.getType() == EventType.None) {
return;
}
try {
@@ -58,13 +60,13 @@
assertTrue("interruption unexpected", false);
}
}
- public void verify(List<Integer> expected) throws InterruptedException{
- WatcherEvent event;
+ public void verify(List<EventType> expected) throws InterruptedException{
+ WatchedEvent event;
int count = 0;
while (count < expected.size()
&& (event = events.poll(30, TimeUnit.SECONDS)) != null)
{
- assertEquals(expected.get(count).intValue(), event.getType());
+ assertEquals(expected.get(count), event.getType());
count++;
}
assertEquals(expected.size(), count);
@@ -78,7 +80,7 @@
private volatile CountDownLatch lsnr_latch;
private ZooKeeper lsnr;
- private List<Integer> expected;
+ private List<EventType> expected;
@Override
protected void setUp() throws Exception {
@@ -93,7 +95,7 @@
lsnr_dwatch = new SimpleWatcher(lsnr_latch);
lsnr = createClient(lsnr_dwatch, lsnr_latch);
- expected = new ArrayList<Integer>();
+ expected = new ArrayList<EventType>();
}
@Override
protected void tearDown() throws Exception {
@@ -124,9 +126,9 @@
assertNull(lsnr.exists("/foo/bar", true));
client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- expected.add(Watcher.Event.EventNodeCreated);
+ expected.add(EventType.NodeCreated);
client.create("/foo/bar", "child".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- expected.add(Watcher.Event.EventNodeCreated);
+ expected.add(EventType.NodeCreated);
verify();
@@ -150,9 +152,9 @@
}
client.setData("/foo", "parent".getBytes(), -1);
- expected.add(Watcher.Event.EventNodeDataChanged);
+ expected.add(EventType.NodeDataChanged);
client.setData("/foo/bar", "child".getBytes(), -1);
- expected.add(Watcher.Event.EventNodeDataChanged);
+ expected.add(EventType.NodeDataChanged);
verify();
@@ -160,9 +162,9 @@
assertNotNull(lsnr.exists("/foo/bar", true));
client.delete("/foo/bar", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
client.delete("/foo", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
verify();
}
@@ -189,9 +191,9 @@
assertNotNull(lsnr.getData("/foo/bar", true, null));
client.setData("/foo", "parent".getBytes(), -1);
- expected.add(Watcher.Event.EventNodeDataChanged);
+ expected.add(EventType.NodeDataChanged);
client.setData("/foo/bar", "child".getBytes(), -1);
- expected.add(Watcher.Event.EventNodeDataChanged);
+ expected.add(EventType.NodeDataChanged);
verify();
@@ -199,9 +201,9 @@
assertNotNull(lsnr.getData("/foo/bar", true, null));
client.delete("/foo/bar", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
client.delete("/foo", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
verify();
}
@@ -226,7 +228,7 @@
assertNotNull(lsnr.getChildren("/foo", true));
client.create("/foo/bar", "child".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- expected.add(Watcher.Event.EventNodeChildrenChanged); // /foo
+ expected.add(EventType.NodeChildrenChanged); // /foo
assertNotNull(lsnr.getChildren("/foo/bar", true));
@@ -240,10 +242,10 @@
assertNotNull(lsnr.getChildren("/foo/bar", true));
client.delete("/foo/bar", -1);
- expected.add(Watcher.Event.EventNodeDeleted); // /foo/bar childwatch
- expected.add(Watcher.Event.EventNodeChildrenChanged); // /foo
+ expected.add(EventType.NodeDeleted); // /foo/bar childwatch
+ expected.add(EventType.NodeChildrenChanged); // /foo
client.delete("/foo", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
verify();
}
@@ -256,7 +258,7 @@
SimpleWatcher w3 = new SimpleWatcher(null);
SimpleWatcher w4 = new SimpleWatcher(null);
- List<Integer> e2 = new ArrayList<Integer>();
+ List<EventType> e2 = new ArrayList<EventType>();
assertNull(lsnr.exists("/foo", true));
assertNull(lsnr.exists("/foo", w1));
@@ -267,9 +269,9 @@
assertNull(lsnr.exists("/foo/bar", w4));
client.create("/foo", "parent".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- expected.add(Watcher.Event.EventNodeCreated);
+ expected.add(EventType.NodeCreated);
client.create("/foo/bar", "child".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- e2.add(Watcher.Event.EventNodeCreated);
+ e2.add(EventType.NodeCreated);
lsnr_dwatch.verify(expected);
w1.verify(expected);
@@ -288,11 +290,11 @@
assertNotNull(lsnr.exists("/foo/bar", w4));
client.setData("/foo", "parent".getBytes(), -1);
- expected.add(Watcher.Event.EventNodeDataChanged);
+ expected.add(EventType.NodeDataChanged);
client.setData("/foo/bar", "child".getBytes(), -1);
- e2.add(Watcher.Event.EventNodeDataChanged);
+ e2.add(EventType.NodeDataChanged);
- lsnr_dwatch.verify(new ArrayList<Integer>()); // not reg so should = 0
+ lsnr_dwatch.verify(new ArrayList<EventType>()); // not reg so should = 0
w1.verify(expected);
w2.verify(e2);
w3.verify(e2);
@@ -310,9 +312,9 @@
assertNotNull(lsnr.exists("/foo/bar", w4));
client.delete("/foo/bar", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
client.delete("/foo", -1);
- e2.add(Watcher.Event.EventNodeDeleted);
+ e2.add(EventType.NodeDeleted);
lsnr_dwatch.verify(expected);
w1.verify(expected);
@@ -332,7 +334,7 @@
SimpleWatcher w3 = new SimpleWatcher(null);
SimpleWatcher w4 = new SimpleWatcher(null);
- List<Integer> e2 = new ArrayList<Integer>();
+ List<EventType> e2 = new ArrayList<EventType>();
try {
lsnr.getData("/foo", w1, null);
@@ -357,9 +359,9 @@
assertNotNull(lsnr.getData("/foo/bar", w4, null));
client.setData("/foo", "parent".getBytes(), -1);
- expected.add(Watcher.Event.EventNodeDataChanged);
+ expected.add(EventType.NodeDataChanged);
client.setData("/foo/bar", "child".getBytes(), -1);
- e2.add(Watcher.Event.EventNodeDataChanged);
+ e2.add(EventType.NodeDataChanged);
lsnr_dwatch.verify(expected);
w1.verify(expected);
@@ -377,9 +379,9 @@
assertNotNull(lsnr.getData("/foo/bar", w4, null));
client.delete("/foo/bar", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
client.delete("/foo", -1);
- e2.add(Watcher.Event.EventNodeDeleted);
+ e2.add(EventType.NodeDeleted);
lsnr_dwatch.verify(expected);
w1.verify(expected);
@@ -398,7 +400,7 @@
SimpleWatcher w3 = new SimpleWatcher(null);
SimpleWatcher w4 = new SimpleWatcher(null);
- List<Integer> e2 = new ArrayList<Integer>();
+ List<EventType> e2 = new ArrayList<EventType>();
try {
lsnr.getChildren("/foo", true);
@@ -418,7 +420,7 @@
assertNotNull(lsnr.getChildren("/foo", w1));
client.create("/foo/bar", "child".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- expected.add(Watcher.Event.EventNodeChildrenChanged); // /foo
+ expected.add(EventType.NodeChildrenChanged); // /foo
assertNotNull(lsnr.getChildren("/foo/bar", w2));
assertNotNull(lsnr.getChildren("/foo/bar", w2));
assertNotNull(lsnr.getChildren("/foo/bar", w3));
@@ -442,10 +444,10 @@
assertNotNull(lsnr.getChildren("/foo/bar", w4));
client.delete("/foo/bar", -1);
- e2.add(Watcher.Event.EventNodeDeleted); // /foo/bar childwatch
- expected.add(Watcher.Event.EventNodeChildrenChanged); // /foo
+ e2.add(EventType.NodeDeleted); // /foo/bar childwatch
+ expected.add(EventType.NodeChildrenChanged); // /foo
client.delete("/foo", -1);
- expected.add(Watcher.Event.EventNodeDeleted);
+ expected.add(EventType.NodeDeleted);
lsnr_dwatch.verify(expected);
w1.verify(expected);
Modified: 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=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java Thu Sep 25 15:25:28 2008
@@ -29,21 +29,23 @@
import org.apache.zookeeper.AsyncCallback.StatCallback;
import org.apache.zookeeper.AsyncCallback.VoidCallback;
import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
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>();
+ LinkedBlockingQueue<WatchedEvent> events =
+ new LinkedBlockingQueue<WatchedEvent>();
- public void process(WatcherEvent event) {
+ public void process(WatchedEvent event) {
super.process(event);
- if (event.getType() != Event.EventNone) {
+ if (event.getType() != Event.EventType.None) {
try {
events.put(event);
} catch (InterruptedException e) {
@@ -100,14 +102,14 @@
for (int i = 0; i < names.length; i++) {
String name = names[i];
- WatcherEvent event = watcher.events.poll(10, TimeUnit.SECONDS);
+ WatchedEvent event = watcher.events.poll(10, TimeUnit.SECONDS);
assertEquals(name, event.getPath());
- assertEquals(Event.EventNodeDataChanged, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(Event.EventType.NodeDataChanged, event.getType());
+ assertEquals(Event.KeeperState.SyncConnected, event.getState());
event = watcher.events.poll(10, TimeUnit.SECONDS);
assertEquals(name, event.getPath());
- assertEquals(Event.EventNodeDeleted, event.getType());
- assertEquals(Event.KeeperStateSyncConnected, event.getState());
+ assertEquals(Event.EventType.NodeDeleted, event.getType());
+ assertEquals(Event.KeeperState.SyncConnected, event.getState());
}
} finally {
if (zk != null) {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java?rev=699114&r1=699113&r2=699114&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java Thu Sep 25 15:25:28 2008
@@ -30,10 +30,11 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.EventType;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.WatchedEvent;
import org.junit.Test;
public class ZooKeeperTestClient extends TestCase implements Watcher {
@@ -43,10 +44,10 @@
protected String testDirOnZK = dirOnZK + "/" + System.currentTimeMillis();
- LinkedBlockingQueue<WatcherEvent> events = new LinkedBlockingQueue<WatcherEvent>();
+ LinkedBlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<WatchedEvent>();
- private WatcherEvent getEvent(int numTries) throws InterruptedException {
- WatcherEvent event = null;
+ private WatchedEvent getEvent(int numTries) throws InterruptedException {
+ WatchedEvent event = null;
for (int i = 0; i < numTries; i++) {
System.out.println("i = " + i);
event = events.poll(10, TimeUnit.SECONDS);
@@ -200,11 +201,11 @@
}
Thread.sleep(5000);
- WatcherEvent event = events.poll(10, TimeUnit.SECONDS);
+ WatchedEvent event = events.poll(10, TimeUnit.SECONDS);
if (event == null) {
throw new IOException("No event was delivered promptly");
}
- if (event.getType() != Watcher.Event.EventNodeChildrenChanged
+ if (event.getType() != EventType.NodeChildrenChanged
|| !event.getPath().equalsIgnoreCase(parentName)) {
fail("Unexpected event was delivered: " + event.toString());
}
@@ -263,12 +264,12 @@
if (event == null) {
throw new AssertionFailedError("First event was not delivered promptly");
}
- if (!((event.getType() == Watcher.Event.EventNodeChildrenChanged &&
+ if (!((event.getType() == EventType.NodeChildrenChanged &&
event.getPath().equalsIgnoreCase(parentName)) ||
- (event.getType() == Watcher.Event.EventNodeDeleted &&
+ (event.getType() == EventType.NodeDeleted &&
event.getPath().equalsIgnoreCase(nodeName)))) {
System.out.print(parentName + " "
- + Watcher.Event.EventNodeChildrenChanged + " " + nodeName + " " + Watcher.Event.EventNodeDeleted);
+ + EventType.NodeChildrenChanged + " " + nodeName + " " + EventType.NodeDeleted);
fail("Unexpected first event was delivered: " + event.toString());
}
@@ -277,12 +278,12 @@
if (event == null) {
throw new AssertionFailedError("Second event was not delivered promptly");
}
- if (!((event.getType() == Watcher.Event.EventNodeChildrenChanged &&
+ if (!((event.getType() == EventType.NodeChildrenChanged &&
event.getPath().equalsIgnoreCase(parentName)) ||
- (event.getType() == Watcher.Event.EventNodeDeleted &&
+ (event.getType() == EventType.NodeDeleted &&
event.getPath().equalsIgnoreCase(nodeName)))) {
System.out.print(parentName + " "
- + Watcher.Event.EventNodeChildrenChanged + " " + nodeName + " " + Watcher.Event.EventNodeDeleted);
+ + EventType.NodeChildrenChanged + " " + nodeName + " " + EventType.NodeDeleted);
fail("Unexpected second event was delivered: " + event.toString());
}
@@ -371,7 +372,7 @@
delete_create_get_set_test_1();
}
- synchronized public void process(WatcherEvent event) {
+ synchronized public void process(WatchedEvent event) {
try {
System.out.println("Got an event " + event.toString());
events.put(event);
|