You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2011/11/05 07:29:02 UTC
svn commit: r1197891 - in /zookeeper/trunk: ./
src/java/main/org/apache/zookeeper/server/quorum/
src/java/test/org/apache/zookeeper/server/quorum/
src/java/test/org/apache/zookeeper/test/
Author: mahadev
Date: Sat Nov 5 06:29:01 2011
New Revision: 1197891
URL: http://svn.apache.org/viewvc?rev=1197891&view=rev
Log:
ZOOKEEPER-1270. testEarlyLeaderAbandonment failing intermittently, quorum formed, no serving. (Flavio, Camille and Alexander Shraer via mahadev)
Modified:
zookeeper/trunk/CHANGES.txt
zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java
zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1197891&r1=1197890&r2=1197891&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sat Nov 5 06:29:01 2011
@@ -444,6 +444,9 @@ BUGFIXES:
ZOOKEEPER-1268. problems with read only mode, intermittent test failures
and ERRORs in the log (phunt via mahadev)
+ ZOOKEEPER-1270. testEarlyLeaderAbandonment failing intermittently,
+ quorum formed, no serving. (Flavio, Camille and Alexander Shraer via mahadev)
+
IMPROVEMENTS:
ZOOKEEPER-724. Improve junit test integration - log harness information
(phunt via mahadev)
Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java?rev=1197891&r1=1197890&r2=1197891&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java Sat Nov 5 06:29:01 2011
@@ -774,7 +774,8 @@ public class Leader {
}
connectingFollowers.add(sid);
QuorumVerifier verifier = self.getQuorumVerifier();
- if (verifier.containsQuorum(connectingFollowers)) {
+ if (connectingFollowers.contains(self.getId()) && verifier.containsQuorum(connectingFollowers))
+{
waitingForNewEpoch = false;
connectingFollowers.notifyAll();
} else {
@@ -807,7 +808,7 @@ public class Leader {
electingFollowers.add(id);
}
QuorumVerifier verifier = self.getQuorumVerifier();
- if (readyToStart && verifier.containsQuorum(electingFollowers)) {
+ if (electingFollowers.contains(self.getId()) && verifier.containsQuorum(electingFollowers)) {
electionFinished = true;
electingFollowers.notifyAll();
} else {
Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java?rev=1197891&r1=1197890&r2=1197891&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java Sat Nov 5 06:29:01 2011
@@ -28,6 +28,7 @@ import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.SocketChannel;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.regex.Pattern;
import org.apache.log4j.Layout;
@@ -144,7 +145,8 @@ public class QuorumPeerMainTest extends
mt[i].start();
}
- waitForAll(zk, States.CONNECTED);
+ waitForAll(zk, States.CONNECTED);
+
// ok lets find the leader and kill everything else, we have a few
// seconds, so it should be plenty of time
@@ -312,13 +314,14 @@ public class QuorumPeerMainTest extends
private void waitForAll(ZooKeeper[] zks, States state) throws InterruptedException {
int iterations = 10;
boolean someoneNotConnected = true;
- while (someoneNotConnected) {
+ while (someoneNotConnected) {
if (iterations-- == 0) {
+ ClientBase.logAllStackTraces();
throw new RuntimeException("Waiting too long");
}
someoneNotConnected = false;
- for (ZooKeeper zk : zks) {
+ for (ZooKeeper zk : zks) {
if (zk.getState() != state) {
someoneNotConnected = true;
}
Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java?rev=1197891&r1=1197890&r2=1197891&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java Sat Nov 5 06:29:01 2011
@@ -67,6 +67,112 @@ public class Zab1_0Test {
}
}
}
+
+ public static final class FollowerMockThread extends Thread {
+ private final Leader leader;
+ private final long followerSid;
+ public long epoch = -1;
+ public String msg = null;
+ private boolean onlyGetEpochToPropose;
+
+ private FollowerMockThread(long followerSid, Leader leader, boolean onlyGetEpochToPropose) {
+ this.leader = leader;
+ this.followerSid = followerSid;
+ this.onlyGetEpochToPropose = onlyGetEpochToPropose;
+ }
+
+ public void run() {
+ if (onlyGetEpochToPropose) {
+ try {
+ epoch = leader.getEpochToPropose(followerSid, 0);
+ } catch (Exception e) {
+ }
+ } else {
+ try{
+ leader.waitForEpochAck(followerSid, new StateSummary(0, 0));
+ msg = "FollowerMockThread (id = " + followerSid + ") returned from waitForEpochAck";
+ } catch (Exception e) {
+ }
+ }
+ }
+ }
+ @Test
+ public void testLeaderInConnectingFollowers() throws Exception {
+ File tmpDir = File.createTempFile("test", "dir");
+ tmpDir.delete();
+ tmpDir.mkdir();
+ Leader leader = null;
+ try {
+ QuorumPeer peer = createQuorumPeer(tmpDir);
+ leader = createLeader(tmpDir, peer);
+ peer.leader = leader;
+ peer.setAcceptedEpoch(5);
+
+ FollowerMockThread f1 = new FollowerMockThread(1, leader, true);
+ FollowerMockThread f2 = new FollowerMockThread(2, leader, true);
+ f1.start();
+ f2.start();
+
+ // wait until followers time out in getEpochToPropose - they shouldn't return
+ // normally because the leader didn't execute getEpochToPropose and so its epoch was not
+ // accounted for
+ f1.join(leader.self.getInitLimit()*leader.self.getTickTime() + 5000);
+ f2.join(leader.self.getInitLimit()*leader.self.getTickTime() + 5000);
+
+ // even though followers timed out, their ids are in connectingFollowers, and their
+ // epoch were accounted for, so the leader should not block and since it started with
+ // accepted epoch = 5 it should now have 6
+ try {
+ long epoch = leader.getEpochToPropose(leader.self.getId(), leader.self.getAcceptedEpoch());
+ Assert.assertEquals("leader got wrong epoch from getEpochToPropose", 6, epoch);
+ } catch (Exception e){
+ Assert.fail("leader timed out in getEpochToPropose");
+ }
+ } finally {
+ recursiveDelete(tmpDir);
+ if (leader != null) {
+ leader.shutdown("end of test");
+ }
+ }
+ }
+
+ @Test
+ public void testLeaderInElectingFollowers() throws Exception {
+ File tmpDir = File.createTempFile("test", "dir");
+ tmpDir.delete();
+ tmpDir.mkdir();
+ Leader leader = null;
+ try {
+ QuorumPeer peer = createQuorumPeer(tmpDir);
+ leader = createLeader(tmpDir, peer);
+ peer.leader = leader;
+
+ FollowerMockThread f1 = new FollowerMockThread(1, leader, false);
+ FollowerMockThread f2 = new FollowerMockThread(2, leader, false);
+
+ // things needed for waitForEpochAck to run (usually in leader.lead(), but we're not running leader here)
+ leader.readyToStart = true;
+ leader.leaderStateSummary = new StateSummary(leader.self.getCurrentEpoch(), leader.zk.getLastProcessedZxid());
+
+ f1.start();
+ f2.start();
+
+ // wait until followers time out in waitForEpochAck - they shouldn't return
+ // normally because the leader didn't execute waitForEpochAck
+ f1.join(leader.self.getInitLimit()*leader.self.getTickTime() + 5000);
+ f2.join(leader.self.getInitLimit()*leader.self.getTickTime() + 5000);
+
+ // make sure that they timed out and didn't return normally
+ Assert.assertTrue(f1.msg + " without waiting for leader", f1.msg == null);
+ Assert.assertTrue(f2.msg + " without waiting for leader", f2.msg == null);
+ } finally {
+ recursiveDelete(tmpDir);
+ if (leader != null) {
+ leader.shutdown("end of test");
+ }
+ }
+ }
+
private static final class NullServerCnxnFactory extends ServerCnxnFactory {
public void startup(ZooKeeperServer zkServer) throws IOException,
InterruptedException {
Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java?rev=1197891&r1=1197890&r2=1197891&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java Sat Nov 5 06:29:01 2011
@@ -513,7 +513,7 @@ public abstract class ClientBase extends
return d.delete();
}
- private static void logAllStackTraces() {
+ public static void logAllStackTraces() {
StringBuilder sb = new StringBuilder();
sb.append("Starting logAllStackTraces()\n");
Map<Thread, StackTraceElement[]> threads = Thread.getAllStackTraces();