You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by br...@apache.org on 2011/09/22 07:42:20 UTC
svn commit: r1173949 - in /zookeeper/trunk: CHANGES.txt
src/java/main/org/apache/zookeeper/server/quorum/Leader.java
src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java
Author: breed
Date: Thu Sep 22 05:42:19 2011
New Revision: 1173949
URL: http://svn.apache.org/viewvc?rev=1173949&view=rev
Log:
ZOOKEEPER-1191. Synchronization issue - wait not in guarded block
ZOOKEEPER-1192. Leader.waitForEpochAck() checks waitingForNewEpoch instead of checking electionFinished
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/Zab1_0Test.java
Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1173949&r1=1173948&r2=1173949&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Thu Sep 22 05:42:19 2011
@@ -11,6 +11,10 @@ BUGFIXES:
ZOOKEEPER-786. Exception in ZooKeeper.toString
(Thomas Koch via phunt)
+ ZOOKEEPER-1191. Synchronization issue - wait not in guarded block (Alex Shraer via breed)
+
+ ZOOKEEPER-1192. Leader.waitForEpochAck() checks waitingForNewEpoch instead of checking electionFinished (Alex Shraer via breed)
+
IMPROVEMENTS:
ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports,
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=1173949&r1=1173948&r2=1173949&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 Thu Sep 22 05:42:19 2011
@@ -781,9 +781,15 @@ public class Leader {
waitingForNewEpoch = false;
connectingFollowers.notifyAll();
} else {
- connectingFollowers.wait(self.getInitLimit()*self.getTickTime());
+ long start = System.currentTimeMillis();
+ long cur = start;
+ long end = start + self.getInitLimit()*self.getTickTime();
+ while(waitingForNewEpoch && cur < end) {
+ connectingFollowers.wait(end - cur);
+ cur = System.currentTimeMillis();
+ }
if (waitingForNewEpoch) {
- throw new InterruptedException("Out of time to propose an epoch");
+ throw new InterruptedException("Timeout while waiting for epoch from quorum");
}
}
return epoch;
@@ -807,10 +813,16 @@ public class Leader {
if (readyToStart && verifier.containsQuorum(electingFollowers)) {
electionFinished = true;
electingFollowers.notifyAll();
- } else {
- electingFollowers.wait(self.getInitLimit()*self.getTickTime());
- if (waitingForNewEpoch) {
- throw new InterruptedException("Out of time to propose an epoch");
+ } else {
+ long start = System.currentTimeMillis();
+ long cur = start;
+ long end = start + self.getInitLimit()*self.getTickTime();
+ while(!electionFinished && cur < end) {
+ electingFollowers.wait(end - cur);
+ cur = System.currentTimeMillis();
+ }
+ if (!electionFinished) {
+ throw new InterruptedException("Timeout while waiting for epoch to be acked by quorum");
}
}
}
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=1173949&r1=1173948&r2=1173949&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 Thu Sep 22 05:42:19 2011
@@ -44,7 +44,10 @@ import org.apache.zookeeper.server.ZooKe
import org.apache.zookeeper.server.ZooKeeperServer.DataTreeBuilder;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
import org.apache.zookeeper.server.quorum.Leader;
+import org.apache.zookeeper.server.quorum.LearnerInfo;
+import org.apache.zookeeper.server.quorum.QuorumPacket;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
+import org.apache.zookeeper.server.quorum.Zab1_0Test.LeaderConversation;
import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.junit.Assert;
@@ -124,7 +127,7 @@ public class Zab1_0Test {
}
static public interface LeaderConversation {
- void converseWithLeader(InputArchive ia, OutputArchive oa) throws Exception;
+ void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l) throws Exception;
}
static public interface FollowerConversation {
@@ -160,7 +163,7 @@ public class Zab1_0Test {
OutputArchive oa = BinaryOutputArchive.getArchive(followerSocket
.getOutputStream());
- conversation.converseWithLeader(ia, oa);
+ conversation.converseWithLeader(ia, oa, leader);
} finally {
recursiveDelete(tmpDir);
if (leader != null) {
@@ -176,7 +179,7 @@ public class Zab1_0Test {
@Test
public void testNormalRun() throws Exception {
testConversation(new LeaderConversation() {
- public void converseWithLeader(InputArchive ia, OutputArchive oa)
+ public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l)
throws IOException {
/* we test a normal run. everything should work out well. */
LearnerInfo li = new LearnerInfo(1, 0x10000);
@@ -209,7 +212,7 @@ public class Zab1_0Test {
@Test
public void testLeaderBehind() throws Exception {
testConversation(new LeaderConversation() {
- public void converseWithLeader(InputArchive ia, OutputArchive oa)
+ public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l)
throws IOException {
/* we test a normal run. everything should work out well. */
LearnerInfo li = new LearnerInfo(1, 0x10000);
@@ -240,7 +243,38 @@ public class Zab1_0Test {
});
}
-
+ /**
+ * Tests that when a quorum of followers send LearnerInfo but do not ack the epoch (which is sent
+ * by the leader upon receipt of LearnerInfo from a quorum), the leader does not start using this epoch
+ * as it would in the normal case (when a quorum do ack the epoch). This tests ZK-1192
+ * @throws Exception
+ */
+ @Test
+ public void testAbandonBeforeACKEpoch() throws Exception {
+ testConversation(new LeaderConversation() {
+ public void converseWithLeader(InputArchive ia, OutputArchive oa, Leader l)
+ throws IOException, InterruptedException {
+ /* we test a normal run. everything should work out well. */
+ LearnerInfo li = new LearnerInfo(1, 0x10000);
+ byte liBytes[] = new byte[12];
+ ByteBufferOutputStream.record2ByteBuffer(li,
+ ByteBuffer.wrap(liBytes));
+ QuorumPacket qp = new QuorumPacket(Leader.FOLLOWERINFO, 0,
+ liBytes, null);
+ oa.writeRecord(qp, null);
+ readPacketSkippingPing(ia, qp);
+ Assert.assertEquals(Leader.LEADERINFO, qp.getType());
+ Assert.assertEquals(ZxidUtils.makeZxid(1, 0), qp.getZxid());
+ Assert.assertEquals(ByteBuffer.wrap(qp.getData()).getInt(),
+ 0x10000);
+ Thread.sleep(l.self.getInitLimit()*l.self.getTickTime() + 5000);
+
+ // The leader didn't get a quorum of acks - make sure that leader's current epoch is not advanced
+ Assert.assertEquals(0, l.self.getCurrentEpoch());
+ }
+ });
+ }
+
private void recursiveDelete(File file) {
if (file.isFile()) {
file.delete();