Add Payload.serializedSize()
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Candidate.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.raft.behaviors;
9
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import java.util.ArrayList;
13 import java.util.Collection;
14 import org.opendaylight.controller.cluster.raft.PeerInfo;
15 import org.opendaylight.controller.cluster.raft.RaftActorContext;
16 import org.opendaylight.controller.cluster.raft.RaftState;
17 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
18 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
19 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
20 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
21 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
22 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
23 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
24 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
25 import scala.concurrent.duration.FiniteDuration;
26
27 /**
28  * The behavior of a RaftActor when it is in the Candidate raft state.
29  *
30  * <p>
31  * Candidates (§5.2):
32  * <ul>
33  * <li> On conversion to candidate, start election:
34  * <ul>
35  * <li> Increment currentTerm
36  * <li> Vote for self
37  * <li> Reset election timer
38  * <li> Send RequestVote RPCs to all other servers
39  * </ul>
40  * <li> If votes received from majority of servers: become leader
41  * <li> If AppendEntries RPC received from new leader: convert to
42  * follower
43  * <li> If election timeout elapses: start new election
44  * </ul>
45  */
46 public final class Candidate extends AbstractRaftActorBehavior {
47
48     private int voteCount;
49
50     private final int votesRequired;
51
52     private final Collection<String> votingPeers = new ArrayList<>();
53
54     public Candidate(final RaftActorContext context) {
55         super(context, RaftState.Candidate);
56
57         for (PeerInfo peer: context.getPeers()) {
58             if (peer.isVoting()) {
59                 votingPeers.add(peer.getId());
60             }
61         }
62
63         log.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers);
64
65         votesRequired = getMajorityVoteCount(votingPeers.size());
66
67         startNewTerm();
68
69         if (votingPeers.isEmpty()) {
70             actor().tell(ElectionTimeout.INSTANCE, actor());
71         } else {
72             scheduleElection(electionDuration());
73         }
74     }
75
76     @Override
77     public String getLeaderId() {
78         return null;
79     }
80
81     @Override
82     public short getLeaderPayloadVersion() {
83         return -1;
84     }
85
86     @Override
87     protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) {
88
89         log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
90
91         // Some other candidate for the same term became a leader and sent us an append entry
92         if (currentTerm() == appendEntries.getTerm()) {
93             log.info("{}: New Leader {} sent an AppendEntries to Candidate for term {} - will switch to Follower",
94                     logName(), appendEntries.getLeaderId(), currentTerm());
95
96             return switchBehavior(new Follower(context));
97         }
98
99         return this;
100     }
101
102     @Override
103     protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
104             final AppendEntriesReply appendEntriesReply) {
105         return this;
106     }
107
108     @Override
109     protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, final RequestVoteReply requestVoteReply) {
110         log.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount);
111
112         if (requestVoteReply.isVoteGranted()) {
113             voteCount++;
114         }
115
116         if (voteCount >= votesRequired) {
117             if (context.getLastApplied() < context.getReplicatedLog().lastIndex()) {
118                 log.info("{}: LastApplied index {} is behind last index {} - switching to PreLeader",
119                         logName(), context.getLastApplied(), context.getReplicatedLog().lastIndex());
120                 return internalSwitchBehavior(RaftState.PreLeader);
121             } else {
122                 return internalSwitchBehavior(RaftState.Leader);
123             }
124         }
125
126         return this;
127     }
128
129     @Override
130     protected FiniteDuration electionDuration() {
131         return super.electionDuration().$div(context.getConfigParams().getCandidateElectionTimeoutDivisor());
132     }
133
134
135     @Override
136     ApplyState getApplyStateFor(final ReplicatedLogEntry entry) {
137         throw new IllegalStateException("A candidate should never attempt to apply " + entry);
138     }
139
140     @Override
141     public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
142         if (message instanceof ElectionTimeout) {
143             log.debug("{}: Received ElectionTimeout", logName());
144
145             if (votesRequired == 0) {
146                 // If there are no peers then we should be a Leader
147                 // We wait for the election timeout to occur before declare
148                 // ourselves the leader. This gives enough time for a leader
149                 // who we do not know about (as a peer)
150                 // to send a message to the candidate
151
152                 return internalSwitchBehavior(RaftState.Leader);
153             }
154
155             startNewTerm();
156             scheduleElection(electionDuration());
157             return this;
158         }
159
160         if (message instanceof RaftRPC) {
161
162             RaftRPC rpc = (RaftRPC) message;
163
164             log.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc,
165                         context.getTermInformation().getCurrentTerm());
166
167             // If RPC request or response contains term T > currentTerm:
168             // set currentTerm = T, convert to follower (§5.1)
169             // This applies to all RPC messages and responses
170             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
171                 log.info("{}: Term {} in \"{}\" message is greater than Candidate's term {} - switching to Follower",
172                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
173
174                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
175
176                 // The raft paper does not say whether or not a Candidate can/should process a RequestVote in
177                 // this case but doing so gains quicker convergence when the sender's log is more up-to-date.
178                 if (message instanceof RequestVote) {
179                     super.handleMessage(sender, message);
180                 }
181
182                 return internalSwitchBehavior(RaftState.Follower);
183             }
184         }
185
186         return super.handleMessage(sender, message);
187     }
188
189     private void startNewTerm() {
190         // set voteCount back to 1 (that is voting for self)
191         voteCount = 1;
192
193         // Increment the election term and vote for self
194         long currentTerm = context.getTermInformation().getCurrentTerm();
195         long newTerm = currentTerm + 1;
196         context.getTermInformation().updateAndPersist(newTerm, context.getId());
197
198         log.info("{}: Starting new election term {}", logName(), newTerm);
199
200         // Request for a vote
201         // TODO: Retry request for vote if replies do not arrive in a reasonable
202         // amount of time TBD
203         for (String peerId : votingPeers) {
204             ActorSelection peerActor = context.getPeerActorSelection(peerId);
205             if (peerActor != null) {
206                 RequestVote requestVote = new RequestVote(
207                         context.getTermInformation().getCurrentTerm(),
208                         context.getId(),
209                         context.getReplicatedLog().lastIndex(),
210                         context.getReplicatedLog().lastTerm());
211
212                 log.debug("{}: Sending {} to peer {}", logName(), requestVote, peerId);
213
214                 peerActor.tell(requestVote, context.getActor());
215             }
216         }
217     }
218
219     @Override
220     public void close() {
221         stopElection();
222     }
223 }