Author: breed
Date: Wed Jan 28 00:15:33 2009
New Revision: 738316
URL: http://svn.apache.org/viewvc?rev=738316&view=rev
Log:
ZOOKEEPER-267. java client incorrectly generating syncdisconnected event when in
disconnected state
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/docs/zookeeperAdmin.html
hadoop/zookeeper/trunk/docs/zookeeperAdmin.pdf
hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html
hadoop/zookeeper/trunk/docs/zookeeperProgrammers.pdf
hadoop/zookeeper/trunk/src/c/src/cli.c
hadoop/zookeeper/trunk/src/c/src/zookeeper.c
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed Jan 28 00:15:33 2009
@@ -69,8 +69,9 @@
ZOOKEEPER-273. Zookeeper c client build should not depend on CPPUNIT. (pat
and runping via mahadev)
- ZOOKEEPER-268. tostring on jute generated objects can cause NPE. (pat via
-mahadev)
+ ZOOKEEPER-268. tostring on jute generated objects can cause NPE. (pat via mahadev)
+
+ ZOOKEEPER-267. java client incorrectly generating syncdisconnected event when in disconnected
state. (pat via breed)
IMPROVEMENTS:
Modified: hadoop/zookeeper/trunk/docs/zookeeperAdmin.html
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperAdmin.html?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/docs/zookeeperAdmin.html (original)
+++ hadoop/zookeeper/trunk/docs/zookeeperAdmin.html Wed Jan 28 00:15:33 2009
@@ -371,7 +371,7 @@
</ul>
<a name="N10066"></a><a name="sc_requiredSoftware"></a>
<h4>Required Software </h4>
-<p>ZooKeeper runs in Java, release 1.5 or greater (JDK 5 or
+<p>ZooKeeper runs in Java, release 1.6 or greater (JDK 6 or
greater). It runs as an <em>ensemble</em> of
ZooKeeper servers. Three ZooKeeper servers is the minimum
recommended size for an ensemble, and we also recommend that
@@ -482,7 +482,7 @@
<p>
<span class="codefrag computeroutput">$ java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar:conf
\
- org.apache.zookeeper.server.quorum.QuorumPeerMain zoo.cfg</span>
+ org.apache.zookeeper.server.quorum.ManagedQuorumPeerMain zoo.cfg</span>
</p>
</li>
Modified: hadoop/zookeeper/trunk/docs/zookeeperAdmin.pdf
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperAdmin.pdf?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
Binary files - no diff available.
Modified: hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html (original)
+++ hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html Wed Jan 28 00:15:33 2009
@@ -750,6 +750,15 @@
implementation requires that the timeout be a minimum of 2 times
the tickTime (as set in the server configuration) and a maximum of
20 times the tickTime.</p>
+<p>Another parameter to the ZooKeeper session establishment
+ call is the default watcher. Watchers are notified when any state
+ change occurs in the client. For example if the client loses
+ connectivity to the server the client will be notified, or if the
+ client's session expires, etc... This watcher should consider the
+ initial state to be disconnected (i.e. before any state changes
+ events are sent to the watcher by the client lib). In the case of
+ a new connection, the first event sent to the watcher is typically
+ the session connection event.</p>
<p>The session is kept alive by requests sent by the client. If
the session is idle for a period of time that would timeout the
session, the client will send a PING request to keep the session
@@ -762,7 +771,7 @@
</div>
-<a name="N101B5"></a><a name="ch_zkWatches"></a>
+<a name="N101B8"></a><a name="ch_zkWatches"></a>
<h2 class="h3">ZooKeeper Watches</h2>
<div class="section">
<p>All of the read operations in ZooKeeper - <strong>getData()</strong>,
<strong>getChildren()</strong>, and <strong>exists()</strong> - have
the option of setting a watch as a
@@ -839,7 +848,7 @@
general this all occurs transparently. There is one case where a watch
may be missed: a watch for the existance of a znode not yet created will
be missed if the znode is created and deleted while disconnected.</p>
-<a name="N101EB"></a><a name="sc_WatchGuarantees"></a>
+<a name="N101EE"></a><a name="sc_WatchGuarantees"></a>
<h3 class="h4">What ZooKeeper Guarantees about Watches</h3>
<p>With regard to watches, ZooKeeper maintains these
guarantees:</p>
@@ -874,7 +883,7 @@
</li>
</ul>
-<a name="N10210"></a><a name="sc_WatchRememberThese"></a>
+<a name="N10213"></a><a name="sc_WatchRememberThese"></a>
<h3 class="h4">Things to Remember about Watches</h3>
<ul>
@@ -933,7 +942,7 @@
</div>
-<a name="N1023C"></a><a name="sc_ZooKeeperAccessControl"></a>
+<a name="N1023F"></a><a name="sc_ZooKeeperAccessControl"></a>
<h2 class="h3">ZooKeeper access control using ACLs</h2>
<div class="section">
<p>ZooKeeper uses ACLs to control access to its znodes (the
@@ -962,7 +971,7 @@
example, the pair <em>(ip:19.22.0.0/16, READ)</em>
gives the <em>READ</em> permission to any clients with
an IP address that starts with 19.22.</p>
-<a name="N10263"></a><a name="sc_ACLPermissions"></a>
+<a name="N10266"></a><a name="sc_ACLPermissions"></a>
<h3 class="h4">ACL Permissions</h3>
<p>ZooKeeper supports the following permissions:</p>
<ul>
@@ -1018,7 +1027,7 @@
node, but nothing more. (The problem is, if you want to call
zoo_exists() on a node that doesn't exist, there is no
permission to check.)</p>
-<a name="N102B9"></a><a name="sc_BuiltinACLSchemes"></a>
+<a name="N102BC"></a><a name="sc_BuiltinACLSchemes"></a>
<h4>Builtin ACL Schemes</h4>
<p>ZooKeeeper has the following built in schemes:</p>
<ul>
@@ -1079,7 +1088,7 @@
</ul>
-<a name="N1030F"></a><a name="ZooKeeper+C+client+API"></a>
+<a name="N10312"></a><a name="ZooKeeper+C+client+API"></a>
<h4>ZooKeeper C client API</h4>
<p>The following constants are provided by the ZooKeeper C
library:</p>
@@ -1301,7 +1310,7 @@
</div>
-<a name="N10426"></a><a name="ch_zkGuarantees"></a>
+<a name="N10429"></a><a name="ch_zkGuarantees"></a>
<h2 class="h3">Consistency Guarantees</h2>
<div class="section">
<p>ZooKeeper is a high performance, scalable service. Both reads and
@@ -1427,12 +1436,12 @@
</div>
-<a name="N1048D"></a><a name="ch_bindings"></a>
+<a name="N10490"></a><a name="ch_bindings"></a>
<h2 class="h3">Bindings</h2>
<div class="section">
<p>The ZooKeeper client libraries come in two languages: Java and C.
The following sections describe these.</p>
-<a name="N10496"></a><a name="Java+Binding"></a>
+<a name="N10499"></a><a name="Java+Binding"></a>
<h3 class="h4">Java Binding</h3>
<p>There are two packages that make up the ZooKeeper Java binding:
<strong>org.apache.zookeeper</strong> and <strong>org.apache.zookeeper.data</strong>.
The rest of the
@@ -1499,7 +1508,7 @@
(SESSION_EXPIRED and AUTH_FAILED), the ZooKeeper object becomes invalid,
the two threads shut down, and any further ZooKeeper calls throw
errors.</p>
-<a name="N104DF"></a><a name="C+Binding"></a>
+<a name="N104E2"></a><a name="C+Binding"></a>
<h3 class="h4">C Binding</h3>
<p>The C binding has a single-threaded and multi-threaded library.
The multi-threaded library is easiest to use and is most similar to the
@@ -1516,7 +1525,7 @@
(i.e. FreeBSD 4.x). In all other cases, application developers should
link with zookeeper_mt, as it includes support for both Sync and Async
API.</p>
-<a name="N104EE"></a><a name="Installation"></a>
+<a name="N104F1"></a><a name="Installation"></a>
<h4>Installation</h4>
<p>If you're building the client from a check-out from the Apache
repository, follow the steps outlined below. If you're building from a
@@ -1647,7 +1656,7 @@
</li>
</ol>
-<a name="N10597"></a><a name="Using+the+C+Client"></a>
+<a name="N1059A"></a><a name="Using+the+C+Client"></a>
<h4>Using the C Client</h4>
<p>You can test your client by running a ZooKeeper server (see
instructions on the project wiki page on how to run it) and connecting
@@ -1705,7 +1714,7 @@
</div>
-<a name="N105DD"></a><a name="ch_guideToZkOperations"></a>
+<a name="N105E0"></a><a name="ch_guideToZkOperations"></a>
<h2 class="h3">Building Blocks: A Guide to ZooKeeper Operations</h2>
<div class="section">
<p>This section surveys all the operations a developer can perform
@@ -1723,28 +1732,28 @@
</li>
</ul>
-<a name="N105F1"></a><a name="sc_errorsZk"></a>
+<a name="N105F4"></a><a name="sc_errorsZk"></a>
<h3 class="h4">Handling Errors</h3>
<p>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.</p>
-<a name="N105FB"></a><a name="sc_connectingToZk"></a>
+<a name="N105FE"></a><a name="sc_connectingToZk"></a>
<h3 class="h4">Connecting to ZooKeeper</h3>
<p></p>
-<a name="N10604"></a><a name="sc_readOps"></a>
+<a name="N10607"></a><a name="sc_readOps"></a>
<h3 class="h4">Read Operations</h3>
<p></p>
-<a name="N1060D"></a><a name="sc_writeOps"></a>
+<a name="N10610"></a><a name="sc_writeOps"></a>
<h3 class="h4">Write Operations</h3>
<p></p>
-<a name="N10616"></a><a name="sc_handlingWatches"></a>
+<a name="N10619"></a><a name="sc_handlingWatches"></a>
<h3 class="h4">Handling Watches</h3>
<p></p>
-<a name="N1061F"></a><a name="sc_miscOps"></a>
+<a name="N10622"></a><a name="sc_miscOps"></a>
<h3 class="h4">Miscelleaneous ZooKeeper Operations</h3>
<p></p>
</div>
-<a name="N10629"></a><a name="ch_programStructureWithExample"></a>
+<a name="N1062C"></a><a name="ch_programStructureWithExample"></a>
<h2 class="h3">Program Structure, with Simple Example</h2>
<div class="section">
<p>
@@ -1753,7 +1762,7 @@
</div>
-<a name="N10634"></a><a name="ch_gotchas"></a>
+<a name="N10637"></a><a name="ch_gotchas"></a>
<h2 class="h3">Gotchas: Common Problems and Troubleshooting</h2>
<div class="section">
<p>So now you know ZooKeeper. It's fast, simple, your application
Modified: hadoop/zookeeper/trunk/docs/zookeeperProgrammers.pdf
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperProgrammers.pdf?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
Binary files - no diff available.
Modified: hadoop/zookeeper/trunk/src/c/src/cli.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/cli.c?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/cli.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/cli.c Wed Jan 28 00:15:33 2009
@@ -48,7 +48,8 @@
static int shutdownThisThing=0;
static __attribute__ ((unused)) void
-printProfileInfo(struct timeval start, struct timeval end,int thres,const char* msg)
+printProfileInfo(struct timeval start, struct timeval end, int thres,
+ const char* msg)
{
int delay=(end.tv_sec*1000+end.tv_usec/1000)-
(start.tv_sec*1000+start.tv_usec/1000);
@@ -56,14 +57,23 @@
fprintf(stderr,"%s: execution time=%dms\n",msg,delay);
}
-void watcher(zhandle_t *zzh, int type, int state, const char *path,void* context) {
- fprintf(stderr,"Watcher %d state = %d for %s\n", type, state, (path ? path: "null"));
+void watcher(zhandle_t *zzh, int type, int state, const char *path,
+ void* context)
+{
+ /* Be careful using zh here rather than zzh - as this may be mt code
+ * the client lib may call the watcher before zookeeper_init returns */
+
+ fprintf(stderr,
+ "Watcher %d state = %d for %s\n",
+ type, state, (path ? path: "null"));
+
if (type == ZOO_SESSION_EVENT) {
if (state == ZOO_CONNECTED_STATE) {
- const clientid_t *id = zoo_client_id(zh);
- if (myid.client_id == 0|| myid.client_id != id->client_id) {
+ const clientid_t *id = zoo_client_id(zzh);
+ if (myid.client_id == 0 || myid.client_id != id->client_id) {
myid = *id;
- fprintf(stderr, "Got a new id: %llx\n", _LL_CAST_ myid.client_id);
+ fprintf(stderr, "Got a new id: %llx\n",
+ _LL_CAST_ myid.client_id);
if (clientIdFile) {
FILE *fh = fopen(clientIdFile, "w");
if (!fh) {
@@ -79,12 +89,12 @@
}
} else if (state == ZOO_AUTH_FAILED_STATE) {
fprintf(stderr, "Authentication failure. Shutting down...\n");
- zookeeper_close(zh);
+ zookeeper_close(zzh);
shutdownThisThing=1;
zh=0;
} else if (state == ZOO_EXPIRED_SESSION_STATE) {
fprintf(stderr, "Session expired. Shutting down...\n");
- zookeeper_close(zh);
+ zookeeper_close(zzh);
shutdownThisThing=1;
zh=0;
}
Modified: hadoop/zookeeper/trunk/src/c/src/zookeeper.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zookeeper.c?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/zookeeper.c Wed Jan 28 00:15:33 2009
@@ -1242,7 +1242,7 @@
}
}
zh->input_buffer = 0;
- }else{
+ } else {
// zookeeper_process was called but there was nothing to read
// from the socket
return ZNOTHING;
Modified: hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
(original)
+++ hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
Wed Jan 28 00:15:33 2009
@@ -377,6 +377,16 @@
the tickTime (as set in the server configuration) and a maximum of
20 times the tickTime.</para>
+ <para>Another parameter to the ZooKeeper session establishment
+ call is the default watcher. Watchers are notified when any state
+ change occurs in the client. For example if the client loses
+ connectivity to the server the client will be notified, or if the
+ client's session expires, etc... This watcher should consider the
+ initial state to be disconnected (i.e. before any state changes
+ events are sent to the watcher by the client lib). In the case of
+ a new connection, the first event sent to the watcher is typically
+ the session connection event.</para>
+
<para>The session is kept alive by requests sent by the client. If
the session is idle for a period of time that would timeout the
session, the client will send a PING request to keep the session
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=738316&r1=738315&r2=738316&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 Wed Jan 28 00:15:33
2009
@@ -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;
@@ -307,6 +309,12 @@
private final LinkedBlockingQueue<Object> waitingEvents =
new LinkedBlockingQueue<Object>();
+ /** This is really the queued session state until the event
+ * thread actually processes the event and hands it to the watcher.
+ * But for all intents and purposes this is the state.
+ */
+ private volatile KeeperState sessionState = KeeperState.Disconnected;
+
EventThread() {
super(currentThread().getName() + "-EventThread");
setUncaughtExceptionHandler(uncaughtExceptionHandler);
@@ -314,6 +322,12 @@
}
public void queueEvent(WatchedEvent event) {
+ if (event.getType() == EventType.None
+ && sessionState == event.getState()) {
+ return;
+ }
+ sessionState = event.getState();
+
// materialize the watchers based on the event
WatcherSetEventPair pair = new WatcherSetEventPair(
watcher.materialize(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=738316&r1=738315&r2=738316&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 Wed Jan
28 00:15:33 2009
@@ -45,7 +45,6 @@
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.data.StatPersisted;
-import org.apache.zookeeper.proto.WatcherEvent;
import org.apache.zookeeper.txn.CreateTxn;
import org.apache.zookeeper.txn.DeleteTxn;
import org.apache.zookeeper.txn.ErrorTxn;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java?rev=738316&r1=738315&r2=738316&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java
Wed Jan 28 00:15:33 2009
@@ -22,11 +22,9 @@
import java.util.LinkedList;
import org.apache.log4j.Logger;
-
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.server.Request;
import org.apache.zookeeper.server.RequestProcessor;
-import org.apache.zookeeper.server.ZooTrace;
/**
* This RequestProcessor matches the incoming committed requests with the
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=738316&r1=738315&r2=738316&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 Wed Jan
28 00:15:33 2009
@@ -22,6 +22,7 @@
import java.io.File;
import java.io.IOException;
+import java.util.LinkedList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@@ -32,6 +33,8 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+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;
@@ -88,11 +91,16 @@
throws IOException, InterruptedException
{
CountdownWatcher watcher = new CountdownWatcher();
+ return createClient(CONNECTION_TIMEOUT, watcher);
+ }
+
+ private DisconnectableZooKeeper createClient(int timeout,
+ CountdownWatcher watcher)
+ throws IOException, InterruptedException
+ {
DisconnectableZooKeeper zk =
- new DisconnectableZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, watcher);
- if(!watcher.clientConnected.await(CONNECTION_TIMEOUT,
- TimeUnit.MILLISECONDS))
- {
+ new DisconnectableZooKeeper(HOSTPORT, timeout, watcher);
+ if(!watcher.clientConnected.await(timeout, TimeUnit.MILLISECONDS)) {
fail("Unable to connect to server");
}
@@ -171,6 +179,49 @@
zk.close();
}
+ @Test
+ /**
+ * This test makes sure that duplicate state changes are not communicated
+ * to the client watcher. For example we should not notify state as
+ * "disconnected" if the watch has already been disconnected. In general
+ * we don't consider a dup state notification if the event type is
+ * not "None" (ie non-None communicates an event).
+ */
+ public void testSessionStateNoDupStateReporting()
+ throws IOException, InterruptedException, KeeperException
+ {
+ final int TIMEOUT = 3000;
+ DupWatcher watcher = new DupWatcher();
+ ZooKeeper zk = createClient(TIMEOUT, watcher);
+
+ // shutdown the server
+ serverFactory.shutdown();
+
+ try {
+ Thread.sleep(10000);
+ } catch (InterruptedException e) {
+ // ignore
+ }
+
+ // verify that the size is just 2 - ie connect then disconnect
+ // if the client attempts reconnect and we are not handling current
+ // state correctly (ie eventing on duplicate disconnects) then we'll
+ // see a disconnect for each failed connection attempt
+ assertEquals(2, watcher.states.size());
+
+ zk.close();
+ }
+
+ private class DupWatcher extends CountdownWatcher {
+ public LinkedList<WatchedEvent> states = new LinkedList<WatchedEvent>();
+ public void process(WatchedEvent event) {
+ super.process(event);
+ if (event.getType() == EventType.None) {
+ states.add(event);
+ }
+ }
+ }
+
public void process(WatchedEvent event) {
LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
if (event.getState() == KeeperState.SyncConnected
|