Fix bug in ReplicatedLogImpl#removeFrom and use akka-persistence for removing entries
[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 org.opendaylight.controller.cluster.raft.RaftActorContext;
14 import org.opendaylight.controller.cluster.raft.RaftState;
15 import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
16 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
17 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
18 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
19 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
20 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
21
22 import java.util.Collection;
23 import java.util.HashMap;
24 import java.util.Map;
25
26 /**
27  * The behavior of a RaftActor when it is in the CandidateState
28  * <p/>
29  * Candidates (§5.2):
30  * <ul>
31  * <li> On conversion to candidate, start election:
32  * <ul>
33  * <li> Increment currentTerm
34  * <li> Vote for self
35  * <li> Reset election timer
36  * <li> Send RequestVote RPCs to all other servers
37  * </ul>
38  * <li> If votes received from majority of servers: become leader
39  * <li> If AppendEntries RPC received from new leader: convert to
40  * follower
41  * <li> If election timeout elapses: start new election
42  * </ul>
43  */
44 public class Candidate extends AbstractRaftActorBehavior {
45
46     private final Map<String, ActorSelection> peerToActor = new HashMap<>();
47
48     private int voteCount;
49
50     private final int votesRequired;
51
52     public Candidate(RaftActorContext context) {
53         super(context);
54
55         Collection<String> peerPaths = context.getPeerAddresses().values();
56
57         for (String peerPath : peerPaths) {
58             peerToActor.put(peerPath,
59                 context.actorSelection(peerPath));
60         }
61
62         context.getLogger().debug("Election:Candidate has following peers:"+peerToActor.keySet());
63         if(peerPaths.size() > 0) {
64             // Votes are required from a majority of the peers including self.
65             // The votesRequired field therefore stores a calculated value
66             // of the number of votes required for this candidate to win an
67             // election based on it's known peers.
68             // If a peer was added during normal operation and raft replicas
69             // came to know about them then the new peer would also need to be
70             // taken into consideration when calculating this value.
71             // Here are some examples for what the votesRequired would be for n
72             // peers
73             // 0 peers = 1 votesRequired (0 + 1) / 2 + 1 = 1
74             // 2 peers = 2 votesRequired (2 + 1) / 2 + 1 = 2
75             // 4 peers = 3 votesRequired (4 + 1) / 2 + 1 = 3
76             int noOfPeers = peerPaths.size();
77             int self = 1;
78             votesRequired = (noOfPeers + self) / 2 + 1;
79         } else {
80             votesRequired = 0;
81         }
82
83         startNewTerm();
84         scheduleElection(electionDuration());
85     }
86
87     @Override protected RaftState handleAppendEntries(ActorRef sender,
88         AppendEntries appendEntries) {
89
90         return state();
91     }
92
93     @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
94         AppendEntriesReply appendEntriesReply) {
95
96         return state();
97     }
98
99     @Override protected RaftState handleRequestVoteReply(ActorRef sender,
100         RequestVoteReply requestVoteReply) {
101
102         if (requestVoteReply.isVoteGranted()) {
103             voteCount++;
104         }
105
106         if (voteCount >= votesRequired) {
107             return RaftState.Leader;
108         }
109
110         return state();
111     }
112
113     @Override public RaftState state() {
114         return RaftState.Candidate;
115     }
116
117     @Override
118     public RaftState handleMessage(ActorRef sender, Object message) {
119
120         if (message instanceof RaftRPC) {
121             RaftRPC rpc = (RaftRPC) message;
122             // If RPC request or response contains term T > currentTerm:
123             // set currentTerm = T, convert to follower (§5.1)
124             // This applies to all RPC messages and responses
125             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
126                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
127                 return RaftState.Follower;
128             }
129         }
130
131         if (message instanceof ElectionTimeout) {
132             if (votesRequired == 0) {
133                 // If there are no peers then we should be a Leader
134                 // We wait for the election timeout to occur before declare
135                 // ourselves the leader. This gives enough time for a leader
136                 // who we do not know about (as a peer)
137                 // to send a message to the candidate
138                 return RaftState.Leader;
139             }
140             startNewTerm();
141             scheduleElection(electionDuration());
142             return state();
143         }
144         return super.handleMessage(sender, message);
145     }
146
147
148     private void startNewTerm() {
149
150
151         // set voteCount back to 1 (that is voting for self)
152         voteCount = 1;
153
154         // Increment the election term and vote for self
155         long currentTerm = context.getTermInformation().getCurrentTerm();
156         context.getTermInformation().updateAndPersist(currentTerm + 1, context.getId());
157
158         context.getLogger().debug("Starting new term " + (currentTerm+1));
159
160         // Request for a vote
161         for (ActorSelection peerActor : peerToActor.values()) {
162             peerActor.tell(new RequestVote(
163                     context.getTermInformation().getCurrentTerm(),
164                     context.getId(),
165                     context.getReplicatedLog().lastIndex(),
166                     context.getReplicatedLog().lastTerm()),
167                 context.getActor()
168             );
169         }
170
171
172     }
173
174     @Override public void close() throws Exception {
175         stopElection();
176     }
177 }