X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FCandidate.java;h=ecd49012461a7b1ee76f4678c7451a4fa1edbfb1;hb=bad1f8b8f3c1780cd37ec8a817ef4b0f23901654;hp=0d035dbce747ab258da8200b68e0cf7b71d6dd79;hpb=fdab53ef9033fc83c812f7d3d6d3327d3d176f0f;p=controller.git diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java index 0d035dbce7..52ed26758e 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Candidate.java @@ -10,18 +10,18 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import akka.actor.ActorSelection; +import java.util.ArrayList; +import java.util.Collection; +import org.opendaylight.controller.cluster.raft.PeerInfo; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; -import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout; +import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; +import org.opendaylight.controller.cluster.raft.messages.RaftRPC; import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - /** * The behavior of a RaftActor when it is in the CandidateState *

@@ -42,99 +42,136 @@ import java.util.Map; */ public class Candidate extends AbstractRaftActorBehavior { - private final Map peerToActor = new HashMap<>(); - private int voteCount; private final int votesRequired; + private final Collection votingPeers = new ArrayList<>(); + public Candidate(RaftActorContext context) { - super(context); + super(context, RaftState.Candidate); - Collection peerPaths = context.getPeerAddresses().values(); + for(PeerInfo peer: context.getPeers()) { + if(peer.isVoting()) { + votingPeers.add(peer.getId()); + } + } - for (String peerPath : peerPaths) { - peerToActor.put(peerPath, - context.actorSelection(peerPath)); + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers); } - if(peerPaths.size() > 0) { - // Votes are required from a majority of the peers including self. - // The votesRequired field therefore stores a calculated value - // of the number of votes required for this candidate to win an - // election based on it's known peers. - // If a peer was added during normal operation and raft replicas - // came to know about them then the new peer would also need to be - // taken into consideration when calculating this value. - // Here are some examples for what the votesRequired would be for n - // peers - // 0 peers = 1 votesRequired (0 + 1) / 2 + 1 = 1 - // 2 peers = 2 votesRequired (2 + 1) / 2 + 1 = 2 - // 4 peers = 3 votesRequired (4 + 1) / 2 + 1 = 3 - int noOfPeers = peerPaths.size(); - int self = 1; - votesRequired = (noOfPeers + self) / 2 + 1; + votesRequired = getMajorityVoteCount(votingPeers.size()); + + startNewTerm(); + + if(votingPeers.isEmpty()){ + actor().tell(ElectionTimeout.INSTANCE, actor()); } else { - votesRequired = 0; + scheduleElection(electionDuration()); } + } - startNewTerm(); - scheduleElection(electionDuration()); + @Override + public final String getLeaderId() { + return null; } - @Override protected RaftState handleAppendEntries(ActorRef sender, - AppendEntries appendEntries, RaftState suggestedState) { + @Override + public final short getLeaderPayloadVersion() { + return -1; + } - context.getLogger().error("An unexpected AppendEntries received in state " + state()); + @Override + protected RaftActorBehavior handleAppendEntries(ActorRef sender, + AppendEntries appendEntries) { - return suggestedState; - } + if(LOG.isDebugEnabled()) { + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); + } - @Override protected RaftState handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply, RaftState suggestedState) { + // Some other candidate for the same term became a leader and sent us an append entry + if(currentTerm() == appendEntries.getTerm()){ + LOG.debug("{}: New Leader sent an append entry to Candidate for term {} will switch to Follower", + logName(), currentTerm()); - // Some peer thinks I was a leader and sent me a reply + return switchBehavior(new Follower(context)); + } - return suggestedState; + return this; } - @Override protected RaftState handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply, RaftState suggestedState) { - if (suggestedState == RaftState.Follower) { - // If base class thinks I should be follower then I am - return suggestedState; - } + @Override + protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { + return this; + } + + @Override + protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { + LOG.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount); if (requestVoteReply.isVoteGranted()) { voteCount++; } if (voteCount >= votesRequired) { - return RaftState.Leader; + if(context.getCommitIndex() < context.getReplicatedLog().lastIndex()) { + LOG.debug("{}: Connmit index {} is behind last index {}", logName(), context.getCommitIndex(), + context.getReplicatedLog().lastIndex()); + return internalSwitchBehavior(RaftState.PreLeader); + } else { + return internalSwitchBehavior(RaftState.Leader); + } } - return state(); - } - - @Override public RaftState state() { - return RaftState.Candidate; + return this; } @Override - public RaftState handleMessage(ActorRef sender, Object message) { + public RaftActorBehavior handleMessage(ActorRef sender, Object message) { if (message instanceof ElectionTimeout) { + LOG.debug("{}: Received ElectionTimeout", logName()); + if (votesRequired == 0) { // If there are no peers then we should be a Leader // We wait for the election timeout to occur before declare // ourselves the leader. This gives enough time for a leader // who we do not know about (as a peer) // to send a message to the candidate - return RaftState.Leader; + + return internalSwitchBehavior(RaftState.Leader); } + startNewTerm(); scheduleElection(electionDuration()); - return state(); + return this; + } + + if (message instanceof RaftRPC) { + + RaftRPC rpc = (RaftRPC) message; + + if(LOG.isDebugEnabled()) { + LOG.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc, + context.getTermInformation().getCurrentTerm()); + } + + // If RPC request or response contains term T > currentTerm: + // set currentTerm = T, convert to follower (§5.1) + // This applies to all RPC messages and responses + if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); + + // The raft paper does not say whether or not a Candidate can/should process a RequestVote in + // this case but doing so gains quicker convergence when the sender's log is more up-to-date. + if (message instanceof RequestVote) { + super.handleMessage(sender, message); + } + + return internalSwitchBehavior(RaftState.Follower); + } } + return super.handleMessage(sender, message); } @@ -147,25 +184,32 @@ public class Candidate extends AbstractRaftActorBehavior { // Increment the election term and vote for self long currentTerm = context.getTermInformation().getCurrentTerm(); - context.getTermInformation().update(currentTerm + 1, context.getId()); + long newTerm = currentTerm + 1; + context.getTermInformation().updateAndPersist(newTerm, context.getId()); - context.getLogger().debug("Starting new term " + (currentTerm+1)); + LOG.debug("{}: Starting new term {}", logName(), newTerm); // Request for a vote - for (ActorSelection peerActor : peerToActor.values()) { - peerActor.tell(new RequestVote( - context.getTermInformation().getCurrentTerm(), - context.getId(), - context.getReplicatedLog().lastIndex(), - context.getReplicatedLog().lastTerm()), - context.getActor() - ); + // TODO: Retry request for vote if replies do not arrive in a reasonable + // amount of time TBD + for (String peerId : votingPeers) { + ActorSelection peerActor = context.getPeerActorSelection(peerId); + if(peerActor != null) { + RequestVote requestVote = new RequestVote( + context.getTermInformation().getCurrentTerm(), + context.getId(), + context.getReplicatedLog().lastIndex(), + context.getReplicatedLog().lastTerm()); + + LOG.debug("{}: Sending {} to peer {}", logName(), requestVote, peerId); + + peerActor.tell(requestVote, context.getActor()); + } } - - } - @Override public void close() throws Exception { + @Override + public void close() { stopElection(); } }