Repository: zookeeper
Updated Branches:
refs/heads/branch-3.4 04b755369 -> 73ec93cfa
ZOOKEEPER-2759: Flaky test: org.apache.zookeeper.server.quorum.QuorumCnxManagerTest.testNoAuthLearnerConnectToAuthRequiredServerWithHigherSid
I noticed some inconsistent test results from `testNoAuthLearnerConnectToAuthRequiredServerWithHigherSid`.
It can be seen failing on Apache infra here: http://mail-archives.apache.org/mod_mbox/zookeeper-dev/201701.mbox/%3c374175863.2852.1485127554310.JavaMail.jenkinscrius%3e
The problem can be "reproduced" by adding a `Thread.sleep(4000)` as the first line of the
finally block for `RecvWorker.run` in `QuorumCnxManager.java`.
The issue is caused by a race condition between the removal of the relevant `sid` from `senderWorkerMap`
for `peer0` and the 3 second delay from `assertEventuallyNotConnected`.
Other tests in this class do not experience the same problem because `testNoAuthLearnerConnectToAuthRequiredServerWithHigherSid`
is unique in that it is the only test using `assertEventuallyNotConnected` where the peer
with the lower `sid` is not using SASL. This means that the peer with the lower `sid` will
always create send and receive workers in `handleConnection` (as it will not throw an exception
on `authServer.authenticate(sock, din)`) and may not destroy them in time for the test assertion.
Author: Abraham Fine <afine@apache.org>
Reviewers: Michael Han <hanm@apache.org>
Closes #229 from afine/ZOOKEEPER-2759 and squashes the following commits:
55869fa [Abraham Fine] add some comments
112a70b [Abraham Fine] fix tests with mockito
c567c49 [Abraham Fine] ZOOKEEPER-2759: Flaky test: org.apache.zookeeper.server.quorum.QuorumCnxManagerTest.testNoAuthLearnerConnectToAuthRequiredServerWithHigherSid
Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/73ec93cf
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/73ec93cf
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/73ec93cf
Branch: refs/heads/branch-3.4
Commit: 73ec93cfa07c860121d544e924fc5a425a7564d0
Parents: 04b7553
Author: Abraham Fine <afine@apache.org>
Authored: Thu Apr 27 14:15:04 2017 -0700
Committer: Michael Han <hanm@apache.org>
Committed: Thu Apr 27 14:15:04 2017 -0700
----------------------------------------------------------------------
ivy.xml | 2 +-
.../server/quorum/QuorumCnxManager.java | 18 +++++-
.../server/quorum/QuorumCnxManagerTest.java | 60 +++++++++++++++++---
3 files changed, 69 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73ec93cf/ivy.xml
----------------------------------------------------------------------
diff --git a/ivy.xml b/ivy.xml
index 1a9af76..8aa76c5 100644
--- a/ivy.xml
+++ b/ivy.xml
@@ -56,7 +56,7 @@
<dependency org="org.vafer" name="jdeb" rev="0.8" conf="package->master"/>
<dependency org="junit" name="junit" rev="4.8.1" conf="test->default"/>
- <dependency org="org.mockito" name="mockito-all" rev="1.8.2"
+ <dependency org="org.mockito" name="mockito-all" rev="1.8.5"
conf="test->default"/>
<dependency org="com.puppycrawl.tools" name="checkstyle" rev="6.1.1"
conf="test->default"/>
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73ec93cf/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
index 6a5ebb4..33f1943 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
@@ -175,11 +175,25 @@ public class QuorumCnxManager {
boolean listenOnAllIPs,
int quorumCnxnThreadsSize,
boolean quorumSaslAuthEnabled) {
+ this(mySid, view, authServer, authLearner, socketTimeout, listenOnAllIPs,
+ quorumCnxnThreadsSize, quorumSaslAuthEnabled, new ConcurrentHashMap<Long,
SendWorker>());
+ }
+
+ // visible for testing
+ public QuorumCnxManager(final long mySid,
+ Map<Long,QuorumPeer.QuorumServer> view,
+ QuorumAuthServer authServer,
+ QuorumAuthLearner authLearner,
+ int socketTimeout,
+ boolean listenOnAllIPs,
+ int quorumCnxnThreadsSize,
+ boolean quorumSaslAuthEnabled,
+ ConcurrentHashMap<Long, SendWorker> senderWorkerMap) {
+ this.senderWorkerMap = senderWorkerMap;
+
this.recvQueue = new ArrayBlockingQueue<Message>(RECV_CAPACITY);
this.queueSendMap = new ConcurrentHashMap<Long, ArrayBlockingQueue<ByteBuffer>>();
- this.senderWorkerMap = new ConcurrentHashMap<Long, SendWorker>();
this.lastMessageSent = new ConcurrentHashMap<Long, ByteBuffer>();
-
String cnxToValue = System.getProperty("zookeeper.cnxTimeout");
if(cnxToValue != null){
this.cnxTO = new Integer(cnxToValue);
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/73ec93cf/src/java/test/org/apache/zookeeper/server/quorum/QuorumCnxManagerTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumCnxManagerTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumCnxManagerTest.java
index 6b6c0b4..ecf59ac 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumCnxManagerTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumCnxManagerTest.java
@@ -34,6 +34,7 @@ import java.net.Socket;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
@@ -68,6 +69,14 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+
public class QuorumCnxManagerTest extends ZKTestCase {
private static final Logger LOG = LoggerFactory.getLogger(QuorumCnxManagerTest.class);
private int count;
@@ -182,9 +191,9 @@ public class QuorumCnxManagerTest extends ZKTestCase {
/**
* Peer0 has auth configured, Peer1 has no auth configured.
- * Peer1 connects to peer0, but is disconnected, because peer1's sid is
+ * Peer0 connects to peer1, but is disconnected, because peer1's sid is
* higher than peer0.
- * Peer0 connects to peer1, but is disconnected, because peer1 cannot
+ * Peer1 connects to peer0, but is disconnected, because peer1 cannot
* handle auth.
*/
@Test(timeout = 30000)
@@ -218,16 +227,36 @@ public class QuorumCnxManagerTest extends ZKTestCase {
* No auth learner connects to a server that requires auth, when the server
* has a higher sid.
* The connection should fail in both directions.
+ *
+ * peer0 should attempt to connect to peer1, but disconnect as its sid is lower
+ * peer1 should attempt to connect to peer0, peer0 will accept and add an entry to
+ * the senderWorkerMap but peer1 will disconnect because peer1 will start speaking SASL
+ * and peer0 will consider this invalid.
+ *
+ * Due to the unique behavior of peer0 creating an entry
+ * in senderWorkerMap for peer1 and then deleting it we use mockito spies to track
+ * this behavior.
*/
@Test(timeout = 30000)
public void testNoAuthLearnerConnectToAuthRequiredServerWithHigherSid()
throws Exception {
- QuorumCnxManager peer0 = createAndStartManager(0);
- QuorumCnxManager peer1 = createAndStartManager(1, "QuorumServer",
- "QuorumLearner", true, true);
+ ConcurrentHashMap<Long, QuorumCnxManager.SendWorker> senderWorkerMap0 =
+ spy(new ConcurrentHashMap<Long, QuorumCnxManager.SendWorker>());
+ ConcurrentHashMap<Long, QuorumCnxManager.SendWorker> senderWorkerMap1 =
+ spy(new ConcurrentHashMap<Long, QuorumCnxManager.SendWorker>());
+
+ QuorumCnxManager peer0 = createAndStartManager(0, senderWorkerMap0);
+ QuorumCnxManager peer1 = createAndStartManager(1, "QuorumServer", "QuorumLearner",
+ true, true, senderWorkerMap1);
peer0.connectOne(1);
peer1.connectOne(0);
+
assertEventuallyNotConnected(peer0, 1);
+
+ verify(senderWorkerMap0, timeout(10000)).put(eq(1L), any(QuorumCnxManager.SendWorker.class));
+ verify(senderWorkerMap0, timeout(10000)).remove(eq(1L), any(QuorumCnxManager.SendWorker.class));
+
+ verify(senderWorkerMap1, never()).put(anyLong(), any(QuorumCnxManager.SendWorker.class));
}
/**
@@ -679,9 +708,14 @@ public class QuorumCnxManagerTest extends ZKTestCase {
}
private QuorumCnxManager createAndStartManager(long sid) {
+ return createAndStartManager(sid, new ConcurrentHashMap<Long, QuorumCnxManager.SendWorker>());
+ }
+
+ private QuorumCnxManager createAndStartManager(long sid, ConcurrentHashMap<Long, QuorumCnxManager.SendWorker>
senderWorkerMap) {
QuorumCnxManager peer = new QuorumCnxManager(sid, peers,
new NullQuorumAuthServer(), new NullQuorumAuthLearner(), 10000,
- false, quorumCnxnThreadsSize, false);
+ false, quorumCnxnThreadsSize, false,
+ senderWorkerMap);
executor.submit(peer.listener);
InetSocketAddress electionAddr = peer.view.get(sid).electionAddr;
waitForElectionAddrBinding(electionAddr, 15);
@@ -692,14 +726,24 @@ public class QuorumCnxManagerTest extends ZKTestCase {
String serverLoginContext,
String learnerLoginContext,
boolean serverRequireSasl,
- boolean learnerRequireSasl)
+ boolean learnerRequireSasl) throws Exception
{
+ return createAndStartManager(sid, serverLoginContext, learnerLoginContext, serverRequireSasl,
learnerRequireSasl, new ConcurrentHashMap<Long, QuorumCnxManager.SendWorker>());
+
+ }
+
+ private QuorumCnxManager createAndStartManager(long sid,
+ String serverLoginContext,
+ String learnerLoginContext,
+ boolean serverRequireSasl,
+ boolean learnerRequireSasl,
+ ConcurrentHashMap<Long, QuorumCnxManager.SendWorker>
senderWorkerMap)
throws Exception {
QuorumAuthLearner authClient = new SaslQuorumAuthLearner(learnerRequireSasl,
"NOT_USING_KRB_PRINCIPAL", learnerLoginContext);
QuorumAuthServer authServer = new SaslQuorumAuthServer(serverRequireSasl,
serverLoginContext, authzHosts);
QuorumCnxManager peer = new QuorumCnxManager(sid, peers,
- authServer, authClient, 10000, false, quorumCnxnThreadsSize, true);
+ authServer, authClient, 10000, false, quorumCnxnThreadsSize, true, senderWorkerMap);
executor.submit(peer.listener);
InetSocketAddress electionAddr = peer.view.get(sid).electionAddr;
waitForElectionAddrBinding(electionAddr, 15);
|