Repository: zookeeper
Updated Branches:
refs/heads/branch-3.5 e8c0ab924 -> 212201131
ZOOKEEPER-2722: fix flaky testSessionEstablishment test.
Make sure client is connected to a quorum before issuing a write operation to avoid possible
race condition between connected to a RO server and forming a new quorum.
Author: Michael Han <hanm@apache.org>
Reviewers: Rakesh Radhakrishnan <rakeshr@apache.org>, Camille Fournier <camille@apache.org>
Closes #191 from hanm/ZOOKEEPER-2722 and squashes the following commits:
99bf87f [Michael Han] Make sure syncConnected flag is set to false in read only connected
state.
9969979 [Michael Han] ZOOKEEPER-2722: fix flaky test testSessionEstablishment.
Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/21220113
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/21220113
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/21220113
Branch: refs/heads/branch-3.5
Commit: 212201131f8eb01dd2209deb37656ccc4096de77
Parents: e8c0ab9
Author: Michael Han <hanm@apache.org>
Authored: Mon Apr 17 16:37:19 2017 -0700
Committer: Michael Han <hanm@apache.org>
Committed: Mon Apr 17 16:37:19 2017 -0700
----------------------------------------------------------------------
.../org/apache/zookeeper/test/ClientBase.java | 34 +++++++++++++++++---
.../apache/zookeeper/test/ReadOnlyModeTest.java | 19 +++++++++--
2 files changed, 46 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/21220113/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 309b2b2..5a1e071 100644
--- a/src/java/test/org/apache/zookeeper/test/ClientBase.java
+++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java
@@ -96,7 +96,10 @@ public abstract class ClientBase extends ZKTestCase {
public static class CountdownWatcher implements Watcher {
// XXX this doesn't need to be volatile! (Should probably be final)
volatile CountDownLatch clientConnected;
+ // Set to true when connected to a read-only server, or a read-write (quorum) server.
volatile boolean connected;
+ // Set to true when connected to a quorum server.
+ volatile boolean syncConnected;
public CountdownWatcher() {
reset();
@@ -104,16 +107,24 @@ public abstract class ClientBase extends ZKTestCase {
synchronized public void reset() {
clientConnected = new CountDownLatch(1);
connected = false;
+ syncConnected = false;
}
synchronized public void process(WatchedEvent event) {
- if (event.getState() == KeeperState.SyncConnected ||
- event.getState() == KeeperState.ConnectedReadOnly) {
+ KeeperState state = event.getState();
+ if (state == KeeperState.SyncConnected) {
connected = true;
- notifyAll();
- clientConnected.countDown();
+ syncConnected = true;
+ } else if (state == KeeperState.ConnectedReadOnly) {
+ connected = true;
+ syncConnected = false;
} else {
connected = false;
- notifyAll();
+ syncConnected = false;
+ }
+
+ notifyAll();
+ if (connected) {
+ clientConnected.countDown();
}
}
synchronized public boolean isConnected() {
@@ -133,6 +144,19 @@ public abstract class ClientBase extends ZKTestCase {
}
}
+ synchronized public void waitForSyncConnected(long timeout)
+ throws InterruptedException, TimeoutException
+ {
+ long expire = Time.currentElapsedTime() + timeout;
+ long left = timeout;
+ while(!syncConnected && left > 0) {
+ wait(left);
+ left = expire - Time.currentElapsedTime();
+ }
+ if (!syncConnected) {
+ throw new TimeoutException("Failed to connect to read-write ZooKeeper server.");
+ }
+ }
synchronized public void waitForDisconnected(long timeout)
throws InterruptedException, TimeoutException
{
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/21220113/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 21ff9f3..5dfbb21 100644
--- a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
+++ b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
@@ -47,8 +47,11 @@ import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import org.slf4j.LoggerFactory;
public class ReadOnlyModeTest extends ZKTestCase {
+ private static final org.slf4j.Logger LOG = LoggerFactory
+ .getLogger(ReadOnlyModeTest.class);
private static int CONNECTION_TIMEOUT = QuorumBase.CONNECTION_TIMEOUT;
private QuorumUtil qu = new QuorumUtil(1);
@@ -229,17 +232,29 @@ public class ReadOnlyModeTest extends ZKTestCase {
Assert.assertSame("should be in r/o mode", States.CONNECTEDREADONLY, zk
.getState());
long fakeId = zk.getSessionId();
+ LOG.info("Connected as r/o mode with state {} and session id {}",
+ zk.getState(), fakeId);
watcher.reset();
qu.start(2);
Assert.assertTrue("waiting for server up", ClientBase.waitForServerUp(
"127.0.0.1:" + qu.getPeer(2).clientPort, CONNECTION_TIMEOUT));
- watcher.waitForConnected(CONNECTION_TIMEOUT);
+ LOG.info("Server 127.0.0.1:{} is up", qu.getPeer(2).clientPort);
+ // ZOOKEEPER-2722: wait until we can connect to a read-write server after the quorum
+ // is formed. Otherwise, it is possible that client first connects to a read-only
server,
+ // then drops the connection because of shutting down of the read-only server caused
+ // by leader election / quorum forming between the read-only server and the newly
started
+ // server. If we happen to execute the zk.create after the read-only server is shutdown
and
+ // before the quorum is formed, we will get a ConnectLossException.
+ watcher.waitForSyncConnected(CONNECTION_TIMEOUT);
+ Assert.assertEquals("Should be in read-write mode", States.CONNECTED,
+ zk.getState());
+ LOG.info("Connected as rw mode with state {} and session id {}",
+ zk.getState(), zk.getSessionId());
zk.create("/test", "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
Assert.assertFalse("fake session and real session have same id", zk
.getSessionId() == fakeId);
-
zk.close();
}
|