X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FCandidate.java;h=52ed26758ee26b6f9949fa1202f1a44628b82364;hp=09ffe056c3e94fcd4592f8f44f4e29123967f918;hb=bad1f8b8f3c1780cd37ec8a817ef4b0f23901654;hpb=e36088234ebb1b7df214c806ad7c54b3b819904f 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 09ffe056c3..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,7 +10,9 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import akka.actor.ActorSelection; -import java.util.Set; +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.base.messages.ElectionTimeout; @@ -44,68 +46,113 @@ public class Candidate extends AbstractRaftActorBehavior { private final int votesRequired; - private final Set peers; + private final Collection votingPeers = new ArrayList<>(); public Candidate(RaftActorContext context) { - super(context); + super(context, RaftState.Candidate); - peers = context.getPeerAddresses().keySet(); + for(PeerInfo peer: context.getPeers()) { + if(peer.isVoting()) { + votingPeers.add(peer.getId()); + } + } if(LOG.isDebugEnabled()) { - LOG.debug("{}: Election: Candidate has following peers: {}", context.getId(), peers); + LOG.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers); } - votesRequired = getMajorityVoteCount(peers.size()); + votesRequired = getMajorityVoteCount(votingPeers.size()); startNewTerm(); - scheduleElection(electionDuration()); + + if(votingPeers.isEmpty()){ + actor().tell(ElectionTimeout.INSTANCE, actor()); + } else { + scheduleElection(electionDuration()); + } + } + + @Override + public final String getLeaderId() { + return null; + } + + @Override + public final short getLeaderPayloadVersion() { + return -1; } - @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, + @Override + protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { if(LOG.isDebugEnabled()) { - LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries); + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); + } + + // 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()); + + return switchBehavior(new Follower(context)); } return this; } - @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply) { - + @Override + protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { return this; } - @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + @Override + protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { + LOG.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount); if (requestVoteReply.isVoteGranted()) { voteCount++; } if (voteCount >= votesRequired) { - return switchBehavior(new Leader(context)); + 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 this; } - @Override public RaftState state() { - return RaftState.Candidate; - } - @Override - public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { + public RaftActorBehavior handleMessage(ActorRef sender, Object message) { + if (message instanceof ElectionTimeout) { + LOG.debug("{}: Received ElectionTimeout", logName()); - Object message = fromSerializableMessage(originalMessage); + 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 internalSwitchBehavior(RaftState.Leader); + } + + startNewTerm(); + scheduleElection(electionDuration()); + return this; + } if (message instanceof RaftRPC) { RaftRPC rpc = (RaftRPC) message; if(LOG.isDebugEnabled()) { - LOG.debug("{}: RaftRPC message received {} my term is {}", context.getId(), rpc, + LOG.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc, context.getTermInformation().getCurrentTerm()); } @@ -115,23 +162,14 @@ public class Candidate extends AbstractRaftActorBehavior { if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { context.getTermInformation().updateAndPersist(rpc.getTerm(), null); - return switchBehavior(new Follower(context)); - } - } + // 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); + } - if (message instanceof ElectionTimeout) { - 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 switchBehavior(new Leader(context)); + return internalSwitchBehavior(RaftState.Follower); } - startNewTerm(); - scheduleElection(electionDuration()); - return this; } return super.handleMessage(sender, message); @@ -146,33 +184,32 @@ public class Candidate extends AbstractRaftActorBehavior { // Increment the election term and vote for self long currentTerm = context.getTermInformation().getCurrentTerm(); - context.getTermInformation().updateAndPersist(currentTerm + 1, - context.getId()); + long newTerm = currentTerm + 1; + context.getTermInformation().updateAndPersist(newTerm, context.getId()); - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Starting new term {}", context.getId(), (currentTerm + 1)); - } + LOG.debug("{}: Starting new term {}", logName(), newTerm); // Request for a vote // TODO: Retry request for vote if replies do not arrive in a reasonable // amount of time TBD - for (String peerId : peers) { + for (String peerId : votingPeers) { ActorSelection peerActor = context.getPeerActorSelection(peerId); if(peerActor != null) { - peerActor.tell(new RequestVote( + RequestVote requestVote = new RequestVote( context.getTermInformation().getCurrentTerm(), context.getId(), context.getReplicatedLog().lastIndex(), - context.getReplicatedLog().lastTerm()), - context.getActor() - ); - } - } + 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(); } }