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=569f6b3d245fe5a6c1021f0537f08b3e1ec36f6e;hb=86e8e4a06b682aa772c834a2cef56d0596540e1b;hp=176704f3d377323a962d5a171cfe53c84beb494e;hpb=8119659681a6814d257314178e759a6ef1b49766;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 176704f3d3..569f6b3d24 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,7 +5,6 @@ * 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; @@ -15,16 +14,20 @@ 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.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 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 int voteCount; @@ -48,24 +51,22 @@ public class Candidate extends AbstractRaftActorBehavior { private final Collection votingPeers = new ArrayList<>(); - public Candidate(RaftActorContext context) { + public Candidate(final RaftActorContext context) { super(context, RaftState.Candidate); - for(PeerInfo peer: context.getPeers()) { - if(peer.isVoting()) { + for (PeerInfo peer: context.getPeers()) { + if (peer.isVoting()) { votingPeers.add(peer.getId()); } } - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers); - } + log.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers); votesRequired = getMajorityVoteCount(votingPeers.size()); startNewTerm(); - if(votingPeers.isEmpty()){ + if (votingPeers.isEmpty()) { actor().tell(ElectionTimeout.INSTANCE, actor()); } else { scheduleElection(electionDuration()); @@ -73,27 +74,24 @@ public class Candidate extends AbstractRaftActorBehavior { } @Override - public final String getLeaderId() { + public String getLeaderId() { return null; } @Override - public final short getLeaderPayloadVersion() { + public short getLeaderPayloadVersion() { return -1; } @Override - protected RaftActorBehavior handleAppendEntries(ActorRef sender, - AppendEntries appendEntries) { + protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: handleAppendEntries: {}", logName(), 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()); + 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)); } @@ -102,22 +100,23 @@ public class Candidate extends AbstractRaftActorBehavior { } @Override - protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { + protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender, + final AppendEntriesReply appendEntriesReply) { return this; } @Override - protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { - LOG.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount); + protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, final RequestVoteReply requestVoteReply) { + log.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount); if (requestVoteReply.isVoteGranted()) { voteCount++; } if (voteCount >= votesRequired) { - if(context.getLastApplied() < context.getReplicatedLog().lastIndex()) { - LOG.debug("{}: LastApplied index {} is behind last index {}", logName(), context.getLastApplied(), - context.getReplicatedLog().lastIndex()); + 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); @@ -128,9 +127,20 @@ public class Candidate extends AbstractRaftActorBehavior { } @Override - public RaftActorBehavior handleMessage(ActorRef sender, Object message) { + protected FiniteDuration electionDuration() { + return super.electionDuration().$div(context.getConfigParams().getCandidateElectionTimeoutDivisor()); + } + + + @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()); + log.debug("{}: Received ElectionTimeout", logName()); if (votesRequired == 0) { // If there are no peers then we should be a Leader @@ -151,15 +161,16 @@ public class Candidate extends AbstractRaftActorBehavior { RaftRPC rpc = (RaftRPC) message; - if(LOG.isDebugEnabled()) { - LOG.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc, + 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()) { + 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 @@ -175,10 +186,7 @@ public class Candidate extends AbstractRaftActorBehavior { return super.handleMessage(sender, message); } - private void startNewTerm() { - - // set voteCount back to 1 (that is voting for self) voteCount = 1; @@ -187,21 +195,21 @@ public class Candidate extends AbstractRaftActorBehavior { long newTerm = currentTerm + 1; context.getTermInformation().updateAndPersist(newTerm, context.getId()); - LOG.debug("{}: Starting new term {}", logName(), newTerm); + 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 : votingPeers) { ActorSelection peerActor = context.getPeerActorSelection(peerId); - if(peerActor != null) { + 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); + log.debug("{}: Sending {} to peer {}", logName(), requestVote, peerId); peerActor.tell(requestVote, context.getActor()); }