Skip to content

Commit

Permalink
ZOOKEEPER-1514. FastLeaderElection - leader ignores the round informa…
Browse files Browse the repository at this point in the history
…tion when joining a quorum (flavio via henryr)

git-svn-id: https://svn.apache.org/repos/asf/zookeeper/trunk@1368737 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information
henryr committed Aug 2, 2012
1 parent 0ee77bb commit 2bd507c
Show file tree
Hide file tree
Showing 5 changed files with 266 additions and 67 deletions.
3 changes: 3 additions & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,9 @@ BUGFIXES:
ZOOKEEPER-1522. intermittent failures in Zab test due to NPE in
recursiveDelete test function (phunt via flavio)

ZOOKEEPER-1514. FastLeaderElection - leader ignores the round
information when joining a quorum (flavio via henryr)

IMPROVEMENTS:

ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -442,7 +442,7 @@ void halt(){
* Returns the current vlue of the logical clock counter
*/
public long getLogicalClock(){
return logicalclock;
return logicalclock;
}

/**
Expand Down Expand Up @@ -621,7 +621,9 @@ private boolean checkLeader(
if(leader != self.getId()){
if(votes.get(leader) == null) predicate = false;
else if(votes.get(leader).getState() != ServerState.LEADING) predicate = false;
}
} else if(logicalclock != electionEpoch) {
predicate = false;
}

return predicate;
}
Expand Down Expand Up @@ -724,7 +726,7 @@ public Vote lookForLeader() throws InterruptedException {

synchronized(this){
logicalclock++;
updateProposal(getInitId(), getInitLastLoggedZxid(), getPeerEpoch());
updateProposal(getInitId(), getInitLastLoggedZxid(), getPeerEpoch());
}

LOG.info("New election. My id = " + self.getId() +
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
/* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.zookeeper.test;

import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.HashMap;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.server.quorum.QuorumCnxManager;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

import org.apache.zookeeper.test.FLETestUtils.LEThread;

public class FLEBackwardElectionRoundTest extends ZKTestCase {
protected static final Logger LOG = LoggerFactory.getLogger(FLELostMessageTest.class);

int count;
HashMap<Long,QuorumServer> peers;
File tmpdir[];
int port[];

QuorumCnxManager cnxManagers[];

@Before
public void setUp() throws Exception {
count = 3;

peers = new HashMap<Long,QuorumServer>(count);
tmpdir = new File[count];
port = new int[count];
cnxManagers = new QuorumCnxManager[count - 1];
}

@After
public void tearDown() throws Exception {
for(int i = 0; i < (count - 1); i++){
if(cnxManagers[i] != null){
cnxManagers[i].halt();
}
}
}

/**
* This test is checking the following case. A server S is
* currently LOOKING and it receives notifications from
* a quorum indicating they are following S. The election
* round E of S is higher than the election round E' in the
* notification messages, so S becomes the leader and sets
* its epoch back to E'. In the meanwhile, one or more
* followers turn to LOOKING and elect S in election round E.
* Having leader and followers with different election rounds
* might prevent other servers from electing a leader because
* they can't get a consistent set of notifications from a
* quorum.
*
* {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1514}
*
*
* @throws Exception
*/

@Test
public void testBackwardElectionRound() throws Exception {
LOG.info("TestLE: " + getTestName()+ ", " + count);
for(int i = 0; i < count; i++) {
int clientport = PortAssignment.unique();
peers.put(Long.valueOf(i),
new QuorumServer(i,
new InetSocketAddress(clientport),
new InetSocketAddress(PortAssignment.unique())));
tmpdir[i] = ClientBase.createTmpDir();
port[i] = clientport;
}

/*
* Start server 0
*/

QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000, 2, 2);
peer.startLeaderElection();
FLETestUtils.LEThread thread = new FLETestUtils.LEThread(peer, 0);
thread.start();


/*
* Start mock server 1
*/
QuorumPeer mockPeer = new QuorumPeer(peers, tmpdir[1], tmpdir[1], port[1], 3, 1, 1000, 2, 2);
cnxManagers[0] = new QuorumCnxManager(mockPeer);
QuorumCnxManager.Listener listener = cnxManagers[0].listener;
listener.start();

cnxManagers[0].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), 0, 0, 1));

/*
* Start mock server 2
*/
mockPeer = new QuorumPeer(peers, tmpdir[2], tmpdir[2], port[2], 3, 2, 1000, 2, 2);
cnxManagers[1] = new QuorumCnxManager(mockPeer);
listener = cnxManagers[1].listener;
listener.start();

cnxManagers[1].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), 0, 0, 1));

/*
* Run another instance of leader election.
*/
thread.join(5000);
thread = new FLETestUtils.LEThread(peer, 0);
thread.start();

/*
* Send the same messages, this time should not make 0 the leader.
*/
cnxManagers[0].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), 0, 0, 1));
cnxManagers[1].toSend(0l, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), 0, 0, 1));


thread.join(5000);

if (!thread.isAlive()) {
Assert.fail("Should not have joined");
}

}
}
70 changes: 6 additions & 64 deletions src/java/test/org/apache/zookeeper/test/FLELostMessageTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -62,45 +62,7 @@ public void setUp() throws Exception {
public void tearDown() throws Exception {
cnxManager.halt();
}


static class LEThread extends Thread {
private int i;
private QuorumPeer peer;

LEThread(QuorumPeer peer, int i) {
this.i = i;
this.peer = peer;
LOG.info("Constructor: " + getName());

}

public void run(){
try{
Vote v = null;
peer.setPeerState(ServerState.LOOKING);
LOG.info("Going to call leader election: " + i);
v = peer.getElectionAlg().lookForLeader();

if (v == null){
Assert.fail("Thread " + i + " got a null vote");
}

/*
* A real zookeeper would take care of setting the current vote. Here
* we do it manually.
*/
peer.setCurrentVote(v);

LOG.info("Finished election: " + i + ", " + v.getId());

Assert.assertTrue("State is not leading.", peer.getPeerState() == ServerState.LEADING);
} catch (Exception e) {
e.printStackTrace();
}
LOG.info("Joining");
}
}

@Test
public void testLostMessage() throws Exception {

Expand All @@ -121,7 +83,7 @@ public void testLostMessage() throws Exception {

QuorumPeer peer = new QuorumPeer(peers, tmpdir[1], tmpdir[1], port[1], 3, 1, 1000, 2, 2);
peer.startLeaderElection();
LEThread thread = new LEThread(peer, 1);
FLETestUtils.LEThread thread = new FLETestUtils.LEThread(peer, 1);
thread.start();

/*
Expand All @@ -134,38 +96,18 @@ public void testLostMessage() throws Exception {
}
}

ByteBuffer createMsg(int state, long leader, long zxid, long epoch){
byte requestBytes[] = new byte[28];
ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);

/*
* Building notification packet to send
*/

requestBuffer.clear();
requestBuffer.putInt(state);
requestBuffer.putLong(leader);
requestBuffer.putLong(zxid);
requestBuffer.putLong(epoch);

return requestBuffer;
}

void mockServer() throws InterruptedException, IOException {
/*
* Create an instance of the connection manager
*/
QuorumPeer peer = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000, 2, 2);
cnxManager = new QuorumCnxManager(peer);
QuorumCnxManager.Listener listener = cnxManager.listener;
if(listener != null){
listener.start();
} else {
LOG.error("Null listener when initializing cnx manager");
}
listener.start();


cnxManager.toSend(1l, createMsg(ServerState.LOOKING.ordinal(), 0, 0, 1));
cnxManager.toSend(1l, FLETestUtils.createMsg(ServerState.LOOKING.ordinal(), 0, 0, 1));
cnxManager.recvQueue.take();
cnxManager.toSend(1L, createMsg(ServerState.FOLLOWING.ordinal(), 1, 0, 1));
cnxManager.toSend(1L, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), 1, 0, 1));
}
}
98 changes: 98 additions & 0 deletions src/java/test/org/apache/zookeeper/test/FLETestUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.zookeeper.test;

import java.nio.ByteBuffer;

import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.junit.Assert;

import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;

public class FLETestUtils {
protected static final Logger LOG = LoggerFactory.getLogger(FLETestUtils.class);


/*
* Thread to run an instance of leader election for
* a given quorum peer.
*/
static class LEThread extends Thread {
private int i;
private QuorumPeer peer;

LEThread(QuorumPeer peer, int i) {
this.i = i;
this.peer = peer;
LOG.info("Constructor: " + getName());

}

public void run(){
try{
Vote v = null;
peer.setPeerState(ServerState.LOOKING);
LOG.info("Going to call leader election: " + i);
v = peer.getElectionAlg().lookForLeader();

if (v == null){
Assert.fail("Thread " + i + " got a null vote");
}

/*
* A real zookeeper would take care of setting the current vote. Here
* we do it manually.
*/
peer.setCurrentVote(v);

LOG.info("Finished election: " + i + ", " + v.getId());

Assert.assertTrue("State is not leading.", peer.getPeerState() == ServerState.LEADING);
} catch (Exception e) {
e.printStackTrace();
}
LOG.info("Joining");
}
}

/*
* Creates a leader election notification message.
*/

static ByteBuffer createMsg(int state, long leader, long zxid, long epoch){
byte requestBytes[] = new byte[28];
ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes);

/*
* Building notification packet to send
*/

requestBuffer.clear();
requestBuffer.putInt(state);
requestBuffer.putLong(leader);
requestBuffer.putLong(zxid);
requestBuffer.putLong(epoch);

return requestBuffer;
}

}

0 comments on commit 2bd507c

Please sign in to comment.