Complete Candidate behavior implementation
[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
9 package org.opendaylight.controller.cluster.raft.behaviors;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.actor.Cancellable;
14 import org.opendaylight.controller.cluster.raft.RaftActorContext;
15 import org.opendaylight.controller.cluster.raft.RaftState;
16 import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
17 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
18 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
19 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
20 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
21 import scala.concurrent.duration.FiniteDuration;
22
23 import java.util.HashMap;
24 import java.util.List;
25 import java.util.Map;
26 import java.util.Random;
27 import java.util.concurrent.TimeUnit;
28 import java.util.concurrent.atomic.AtomicLong;
29
30 /**
31  * The behavior of a RaftActor when it is in the CandidateState
32  * <p>
33  * Candidates (ยง5.2):
34  * <ul>
35  * <li> On conversion to candidate, start election:
36  * <ul>
37  * <li> Increment currentTerm
38  * <li> Vote for self
39  * <li> Reset election timer
40  * <li> Send RequestVote RPCs to all other servers
41  * </ul>
42  * <li> If votes received from majority of servers: become leader
43  * <li> If AppendEntries RPC received from new leader: convert to
44  * follower
45  * <li> If election timeout elapses: start new election
46  * </ul>
47  */
48 public class Candidate extends AbstractRaftActorBehavior {
49
50     /**
51      * The maximum election time variance
52      */
53     private static final int ELECTION_TIME_MAX_VARIANCE = 100;
54
55     /**
56      * The interval in which a new election would get triggered if no leader is found
57      */
58     private static final long ELECTION_TIME_INTERVAL = Leader.HEART_BEAT_INTERVAL.toMillis() * 2;
59
60     /**
61      *
62      */
63     private final Map<String, ActorSelection> peerToActor = new HashMap<>();
64
65     private Cancellable electionCancel = null;
66
67     private int voteCount;
68
69     private final int votesRequired;
70
71     public Candidate(RaftActorContext context, List<String> peerPaths) {
72         super(context);
73
74         for (String peerPath : peerPaths) {
75             peerToActor.put(peerPath,
76                 context.actorSelection(peerPath));
77         }
78
79         if(peerPaths.size() > 0) {
80             // Votes are required from a majority of the peers including self.
81             // The votesRequired field therefore stores a calculated value
82             // of the number of votes required for this candidate to win an
83             // election based on it's known peers.
84             // If a peer was added during normal operation and raft replicas
85             // came to know about them then the new peer would also need to be
86             // taken into consideration when calculating this value.
87             // Here are some examples for what the votesRequired would be for n
88             // peers
89             // 0 peers = 1 votesRequired (0 + 1) / 2 + 1 = 1
90             // 2 peers = 2 votesRequired (2 + 1) / 2 + 1 = 2
91             // 4 peers = 3 votesRequired (4 + 1) / 2 + 1 = 3
92             int noOfPeers = peerPaths.size();
93             int self = 1;
94             votesRequired = (noOfPeers + self) / 2 + 1;
95         } else {
96             votesRequired = 0;
97         }
98
99         scheduleElection(randomizedDuration());
100     }
101
102     @Override protected RaftState handleAppendEntries(ActorRef sender,
103         AppendEntries appendEntries, RaftState suggestedState) {
104
105         // There is some peer who thinks it's a leader but is not
106         // I will not accept this append entries
107         sender.tell(new AppendEntriesReply(
108             context.getTermInformation().getCurrentTerm().get(), false),
109             context.getActor());
110
111         return suggestedState;
112     }
113
114     @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
115         AppendEntriesReply appendEntriesReply, RaftState suggestedState) {
116
117         // Some peer thinks I was a leader and sent me a reply
118
119         return suggestedState;
120     }
121
122     @Override protected RaftState handleRequestVote(ActorRef sender,
123         RequestVote requestVote, RaftState suggestedState) {
124
125         // We got this RequestVote because the term in there is less than
126         // or equal to our current term, so do not grant the vote
127         sender.tell(new RequestVoteReply(
128             context.getTermInformation().getCurrentTerm().get(), false),
129             context.getActor());
130
131         return suggestedState;
132     }
133
134     @Override protected RaftState handleRequestVoteReply(ActorRef sender,
135         RequestVoteReply requestVoteReply, RaftState suggestedState) {
136         if(suggestedState == RaftState.Follower) {
137             // If base class thinks I should be follower then I am
138             return suggestedState;
139         }
140
141         if(requestVoteReply.isVoteGranted()){
142             voteCount++;
143         }
144
145         if(voteCount >= votesRequired){
146             return RaftState.Leader;
147         }
148
149         return state();
150     }
151
152     @Override protected RaftState state() {
153         return RaftState.Candidate;
154     }
155
156     @Override
157     public RaftState handleMessage(ActorRef sender, Object message) {
158         if(message instanceof ElectionTimeout){
159             if(votesRequired == 0){
160                 // If there are no peers then we should be a Leader
161                 // We wait for the election timeout to occur before declare
162                 // ourselves the leader. This gives enough time for a leader
163                 // who we do not know about (as a peer)
164                 // to send a message to the candidate
165                 return RaftState.Leader;
166             }
167             scheduleElection(randomizedDuration());
168             return state();
169         }
170         return super.handleMessage(sender, message);
171     }
172
173     private FiniteDuration randomizedDuration(){
174         long variance = new Random().nextInt(ELECTION_TIME_MAX_VARIANCE);
175         return new FiniteDuration(ELECTION_TIME_INTERVAL + variance, TimeUnit.MILLISECONDS);
176     }
177
178     private void scheduleElection(FiniteDuration interval) {
179
180         // set voteCount back to 1 (that is voting for self)
181         voteCount = 1;
182
183         // Increment the election term and vote for self
184         AtomicLong currentTerm = context.getTermInformation().getCurrentTerm();
185         context.getTermInformation().update(currentTerm.incrementAndGet(), context.getId());
186
187         // Request for a vote
188         for(ActorSelection peerActor : peerToActor.values()){
189             peerActor.tell(new RequestVote(
190                 context.getTermInformation().getCurrentTerm().get(),
191                 context.getId(), context.getReplicatedLog().last().getIndex(),
192                 context.getReplicatedLog().last().getTerm()),
193                 context.getActor());
194         }
195
196         if (electionCancel != null && !electionCancel.isCancelled()) {
197             electionCancel.cancel();
198         }
199
200         // Schedule an election. When the scheduler triggers an ElectionTimeout
201         // message is sent to itself
202         electionCancel =
203             context.getActorSystem().scheduler().scheduleOnce(interval,
204                 context.getActor(), new ElectionTimeout(),
205                 context.getActorSystem().dispatcher(), context.getActor());
206     }
207
208 }