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=a8762ec76edb11e755731379960a7014f215a598;hp=4a3e2c5d664406844edaddee6308abf112b0f79c;hb=HEAD;hpb=f14033146e051aca1b51c791373f6e867af340b0 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 4a3e2c5d66..77f7a06c49 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 @@ -5,25 +5,28 @@ * terms of the Eclipse Public License v1.0 which accompanies this distribution, * and is available at http://www.eclipse.org/legal/epl-v10.html */ - package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import akka.actor.ActorSelection; +import com.google.common.collect.ImmutableList; +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.ReplicatedLogEntry; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; 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.Set; +import scala.concurrent.duration.FiniteDuration; /** - * The behavior of a RaftActor when it is in the CandidateState - *

+ * The behavior of a RaftActor when it is in the Candidate raft state. + * + *

* Candidates (§5.2): *

*/ -public class Candidate extends AbstractRaftActorBehavior { +public final class Candidate extends AbstractRaftActorBehavior { + private final ImmutableList votingPeers; + private final int votesRequired; private int voteCount; - private final int votesRequired; + public Candidate(final RaftActorContext context) { + super(context, RaftState.Candidate); - private final Set peers; + votingPeers = context.getPeers().stream() + .filter(PeerInfo::isVoting) + .map(PeerInfo::getId) + .collect(ImmutableList.toImmutableList()); - public Candidate(RaftActorContext context) { - super(context); + log.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers); - peers = context.getPeerAddresses().keySet(); + votesRequired = getMajorityVoteCount(votingPeers.size()); - if(LOG.isDebugEnabled()) { - LOG.debug("Election:Candidate has following peers: {}", peers); - } + startNewTerm(); - if(peers.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 = peers.size(); - int self = 1; - votesRequired = (noOfPeers + self) / 2 + 1; + if (votingPeers.isEmpty()) { + actor().tell(ElectionTimeout.INSTANCE, actor()); } else { - votesRequired = 0; + scheduleElection(electionDuration()); } + } - startNewTerm(); - scheduleElection(electionDuration()); + @Override + public String getLeaderId() { + return null; + } + + @Override + public short getLeaderPayloadVersion() { + return -1; } - @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, - AppendEntries appendEntries) { + @Override + protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) { + + log.debug("{}: handleAppendEntries: {}", logName(), appendEntries); - if(LOG.isDebugEnabled()) { - LOG.debug(appendEntries.toString()); + // Some other candidate for the same term became a leader and sent us an append entry + if (currentTerm() == appendEntries.getTerm()) { + log.info("{}: New Leader {} sent an AppendEntries to Candidate for term {} - will switch to Follower", + logName(), appendEntries.getLeaderId(), currentTerm()); + + return switchBehavior(new Follower(context)); } return this; } - @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply) { - + @Override + protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender, + final AppendEntriesReply appendEntriesReply) { return this; } - @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + @Override + protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, final RequestVoteReply requestVoteReply) { + log.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount); if (requestVoteReply.isVoteGranted()) { voteCount++; } if (voteCount >= votesRequired) { - return switchBehavior(new Leader(context)); + if (context.getLastApplied() < context.getReplicatedLog().lastIndex()) { + log.info("{}: LastApplied index {} is behind last index {} - switching to PreLeader", + logName(), context.getLastApplied(), 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) { - - Object message = fromSerializableMessage(originalMessage); - - if (message instanceof RaftRPC) { - - RaftRPC rpc = (RaftRPC) message; + protected FiniteDuration electionDuration() { + return super.electionDuration().$div(context.getConfigParams().getCandidateElectionTimeoutDivisor()); + } - if(LOG.isDebugEnabled()) { - LOG.debug("RaftRPC message received {} my term is {}", 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); - - return switchBehavior(new Follower(context)); - } - } + @Override + ApplyState getApplyStateFor(final ReplicatedLogEntry entry) { + throw new IllegalStateException("A candidate should never attempt to apply " + entry); + } + @Override + public RaftActorBehavior handleMessage(final ActorRef sender, final 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 @@ -145,52 +145,73 @@ public class Candidate extends AbstractRaftActorBehavior { // who we do not know about (as a peer) // to send a message to the candidate - return switchBehavior(new Leader(context)); + return internalSwitchBehavior(RaftState.Leader); } + startNewTerm(); scheduleElection(electionDuration()); return this; } - return super.handleMessage(sender, message); - } + if (message instanceof RaftRPC rpc) { + log.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc, + context.getTermInformation().getCurrentTerm()); - private void startNewTerm() { + // 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()) { + log.info("{}: Term {} in \"{}\" message is greater than Candidate's term {} - switching to Follower", + logName(), rpc.getTerm(), rpc, 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); + } + private void startNewTerm() { // set voteCount back to 1 (that is voting for self) voteCount = 1; // 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 {}", (currentTerm + 1)); - } + log.info("{}: Starting new election 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( + if (peerActor != null) { + 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(); } }