You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2017/06/08 15:51:13 UTC

zookeeper git commit: ZOOKEEPER-2798: Fix flaky test: org.apache.zookeeper.test.ReadOnlyModeTest.testConnectionEvents

Repository: zookeeper
Updated Branches:
  refs/heads/branch-3.5 beed6cb02 -> 643e551ea


ZOOKEEPER-2798: Fix flaky test: org.apache.zookeeper.test.ReadOnlyModeTest.testConnectionEvents

See https://github.com/apache/zookeeper/pull/270 for a description of the bug.

Author: Abraham Fine <af...@apache.org>

Reviewers: Michael Han <ha...@apache.org>

Closes #271 from afine/ZOOKEEPER-2798_3.5


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/643e551e
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/643e551e
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/643e551e

Branch: refs/heads/branch-3.5
Commit: 643e551eacc1fb76c40e04b5d857aaac77089343
Parents: beed6cb
Author: Abraham Fine <af...@apache.org>
Authored: Thu Jun 8 08:51:09 2017 -0700
Committer: Michael Han <ha...@apache.org>
Committed: Thu Jun 8 08:51:09 2017 -0700

----------------------------------------------------------------------
 .../org/apache/zookeeper/test/ClientBase.java   | 19 ++++++++++++++++++
 .../apache/zookeeper/test/ReadOnlyModeTest.java | 21 +++++---------------
 2 files changed, 24 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/643e551e/src/java/test/org/apache/zookeeper/test/ClientBase.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java
index 5a1e071..7f381d2 100644
--- a/src/java/test/org/apache/zookeeper/test/ClientBase.java
+++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java
@@ -100,6 +100,8 @@ public abstract class ClientBase extends ZKTestCase {
         volatile boolean connected;
         // Set to true when connected to a quorum server.
         volatile boolean syncConnected;
+        // Set to true when connected to a quorum server in read-only mode
+        volatile boolean readOnlyConnected;
 
         public CountdownWatcher() {
             reset();
@@ -108,18 +110,22 @@ public abstract class ClientBase extends ZKTestCase {
             clientConnected = new CountDownLatch(1);
             connected = false;
             syncConnected = false;
+            readOnlyConnected = false;
         }
         synchronized public void process(WatchedEvent event) {
             KeeperState state = event.getState();
             if (state == KeeperState.SyncConnected) {
                 connected = true;
                 syncConnected = true;
+                readOnlyConnected = false;
             } else if (state == KeeperState.ConnectedReadOnly) {
                 connected = true;
                 syncConnected = false;
+                readOnlyConnected = true;
             } else {
                 connected = false;
                 syncConnected = false;
+                readOnlyConnected = false;
             }
 
             notifyAll();
@@ -157,6 +163,19 @@ public abstract class ClientBase extends ZKTestCase {
                 throw new TimeoutException("Failed to connect to read-write ZooKeeper server.");
             }
         }
+        synchronized public void waitForReadOnlyConnected(long timeout)
+                throws InterruptedException, TimeoutException
+        {
+            long expire = System.currentTimeMillis() + timeout;
+            long left = timeout;
+            while(!readOnlyConnected && left > 0) {
+                wait(left);
+                left = expire - System.currentTimeMillis();
+            }
+            if (!readOnlyConnected) {
+                throw new TimeoutException("Failed to connect in read-only mode to ZooKeeper server.");
+            }
+        }
         synchronized public void waitForDisconnected(long timeout)
             throws InterruptedException, TimeoutException
         {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/643e551e/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
index 5dfbb21..68c7182 100644
--- a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
+++ b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
@@ -169,13 +169,9 @@ public class ReadOnlyModeTest extends ZKTestCase {
      */
     @Test(timeout = 90000)
     public void testConnectionEvents() throws Exception {
-        final List<KeeperState> states = new ArrayList<KeeperState>();
+        CountdownWatcher watcher = new CountdownWatcher();
         ZooKeeper zk = new ZooKeeper(qu.getConnString(), CONNECTION_TIMEOUT,
-                new Watcher() {
-                    public void process(WatchedEvent event) {
-                        states.add(event.getState());
-                    }
-                }, true);
+                watcher, true);
         boolean success = false;
         for (int i = 0; i < 30; i++) {
             try {
@@ -188,6 +184,7 @@ public class ReadOnlyModeTest extends ZKTestCase {
             }            
         }
         Assert.assertTrue("Did not succeed in connecting in 30s", success);
+        Assert.assertFalse("The connection should not be read-only yet", watcher.readOnlyConnected);
 
         // kill peer and wait no more than 5 seconds for read-only server
         // to be started (which should take one tickTime (2 seconds))
@@ -195,12 +192,7 @@ public class ReadOnlyModeTest extends ZKTestCase {
 
         // Re-connect the client (in case we were connected to the shut down
         // server and the local session was not persisted).
-        zk = new ZooKeeper(qu.getConnString(), CONNECTION_TIMEOUT,
-                new Watcher() {
-                    public void process(WatchedEvent event) {
-                        states.add(event.getState());
-                    }
-                }, true);
+        zk = new ZooKeeper(qu.getConnString(), CONNECTION_TIMEOUT, watcher, true);
         long start = Time.currentElapsedTime();
         while (!(zk.getState() == States.CONNECTEDREADONLY)) {
             Thread.sleep(200);
@@ -209,10 +201,7 @@ public class ReadOnlyModeTest extends ZKTestCase {
                               Time.currentElapsedTime() - start < 30000);
         }
 
-        // At this point states list should contain, in the given order,
-        // SyncConnected, Disconnected, and ConnectedReadOnly states
-        Assert.assertTrue("ConnectedReadOnly event wasn't received", states
-                .get(2) == KeeperState.ConnectedReadOnly);
+        watcher.waitForReadOnlyConnected(5000);
         zk.close();
     }