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=b36c41abf262b8c24d05f354091f7c4ec92a0827;hp=c125bd32b60a5c5d714ea13e7008417f904d88dc;hb=c31509c7a6630e54a9f9749a643fed5e1a1ad380;hpb=07ba9a998f0b3c3045ed8e31afda5c96de141b3b 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 c125bd32b6..b36c41abf2 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,6 +10,7 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import akka.actor.ActorSelection; +import java.util.Set; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; @@ -19,8 +20,6 @@ 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; - /** * The behavior of a RaftActor when it is in the CandidateState *

@@ -48,70 +47,55 @@ public class Candidate extends AbstractRaftActorBehavior { private final Set peers; public Candidate(RaftActorContext context) { - super(context); + super(context, RaftState.Candidate); peers = context.getPeerAddresses().keySet(); - context.getLogger().debug("Election:Candidate has following peers:"+ peers); - - 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; - } else { - votesRequired = 0; + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Election: Candidate has following peers: {}", logName(), peers); } + votesRequired = getMajorityVoteCount(peers.size()); + startNewTerm(); scheduleElection(electionDuration()); } - @Override protected RaftState handleAppendEntries(ActorRef sender, + @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { - context.getLogger().info("Candidate: Received {}", appendEntries.toString()); + if(LOG.isDebugEnabled()) { + LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); + } - return state(); + return this; } - @Override protected RaftState handleAppendEntriesReply(ActorRef sender, + @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { - return state(); + return this; } - @Override protected RaftState 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 RaftState.Leader; + return switchBehavior(new Leader(context)); } - return state(); - } - - @Override public RaftState state() { - return RaftState.Candidate; + return this; } @Override - public RaftState handleMessage(ActorRef sender, Object originalMessage) { + public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { Object message = fromSerializableMessage(originalMessage); @@ -119,29 +103,36 @@ public class Candidate extends AbstractRaftActorBehavior { RaftRPC rpc = (RaftRPC) message; - context.getLogger().debug("RaftRPC message received {} my term is {}", rpc.toString(), context.getTermInformation().getCurrentTerm()); + 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); - return RaftState.Follower; + + return switchBehavior(new Follower(context)); } } 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 switchBehavior(new Leader(context)); } startNewTerm(); scheduleElection(electionDuration()); - return state(); + return this; } return super.handleMessage(sender, message); @@ -156,10 +147,10 @@ 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()); - context.getLogger().debug("Starting new term " + (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 @@ -167,17 +158,17 @@ public class Candidate extends AbstractRaftActorBehavior { for (String peerId : peers) { 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 {