Bug 7521: Convert install snapshot chunking to use streams
[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 java.util.ArrayList;
14 import java.util.Collection;
15 import org.opendaylight.controller.cluster.raft.PeerInfo;
16 import org.opendaylight.controller.cluster.raft.RaftActorContext;
17 import org.opendaylight.controller.cluster.raft.RaftState;
18 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
19 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
20 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
21 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
22 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
23 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
24
25 /**
26  * The behavior of a RaftActor when it is in the Candidate raft state.
27  *
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 int voteCount;
47
48     private final int votesRequired;
49
50     private final Collection<String> votingPeers = new ArrayList<>();
51
52     public Candidate(RaftActorContext context) {
53         super(context, RaftState.Candidate);
54
55         for (PeerInfo peer: context.getPeers()) {
56             if (peer.isVoting()) {
57                 votingPeers.add(peer.getId());
58             }
59         }
60
61         log.debug("{}: Election: Candidate has following voting peers: {}", logName(), votingPeers);
62
63         votesRequired = getMajorityVoteCount(votingPeers.size());
64
65         startNewTerm();
66
67         if (votingPeers.isEmpty()) {
68             actor().tell(ElectionTimeout.INSTANCE, actor());
69         } else {
70             scheduleElection(electionDuration());
71         }
72     }
73
74     @Override
75     public final String getLeaderId() {
76         return null;
77     }
78
79     @Override
80     public final short getLeaderPayloadVersion() {
81         return -1;
82     }
83
84     @Override
85     protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) {
86
87         log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
88
89         // Some other candidate for the same term became a leader and sent us an append entry
90         if (currentTerm() == appendEntries.getTerm()) {
91             log.debug("{}: New Leader sent an append entry to Candidate for term {} will switch to Follower",
92                     logName(), currentTerm());
93
94             return switchBehavior(new Follower(context));
95         }
96
97         return this;
98     }
99
100     @Override
101     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
102         return this;
103     }
104
105     @Override
106     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) {
107         log.debug("{}: handleRequestVoteReply: {}, current voteCount: {}", logName(), requestVoteReply, voteCount);
108
109         if (requestVoteReply.isVoteGranted()) {
110             voteCount++;
111         }
112
113         if (voteCount >= votesRequired) {
114             if (context.getLastApplied() < context.getReplicatedLog().lastIndex()) {
115                 log.debug("{}: LastApplied index {} is behind last index {}", logName(), context.getLastApplied(),
116                         context.getReplicatedLog().lastIndex());
117                 return internalSwitchBehavior(RaftState.PreLeader);
118             } else {
119                 return internalSwitchBehavior(RaftState.Leader);
120             }
121         }
122
123         return this;
124     }
125
126     @Override
127     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
128         if (message instanceof ElectionTimeout) {
129             log.debug("{}: Received ElectionTimeout", logName());
130
131             if (votesRequired == 0) {
132                 // If there are no peers then we should be a Leader
133                 // We wait for the election timeout to occur before declare
134                 // ourselves the leader. This gives enough time for a leader
135                 // who we do not know about (as a peer)
136                 // to send a message to the candidate
137
138                 return internalSwitchBehavior(RaftState.Leader);
139             }
140
141             startNewTerm();
142             scheduleElection(electionDuration());
143             return this;
144         }
145
146         if (message instanceof RaftRPC) {
147
148             RaftRPC rpc = (RaftRPC) message;
149
150             log.debug("{}: RaftRPC message received {}, my term is {}", logName(), rpc,
151                         context.getTermInformation().getCurrentTerm());
152
153             // If RPC request or response contains term T > currentTerm:
154             // set currentTerm = T, convert to follower (§5.1)
155             // This applies to all RPC messages and responses
156             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
157                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
158
159                 // The raft paper does not say whether or not a Candidate can/should process a RequestVote in
160                 // this case but doing so gains quicker convergence when the sender's log is more up-to-date.
161                 if (message instanceof RequestVote) {
162                     super.handleMessage(sender, message);
163                 }
164
165                 return internalSwitchBehavior(RaftState.Follower);
166             }
167         }
168
169         return super.handleMessage(sender, message);
170     }
171
172
173     private void startNewTerm() {
174
175
176         // set voteCount back to 1 (that is voting for self)
177         voteCount = 1;
178
179         // Increment the election term and vote for self
180         long currentTerm = context.getTermInformation().getCurrentTerm();
181         long newTerm = currentTerm + 1;
182         context.getTermInformation().updateAndPersist(newTerm, context.getId());
183
184         log.debug("{}: Starting new term {}", logName(), newTerm);
185
186         // Request for a vote
187         // TODO: Retry request for vote if replies do not arrive in a reasonable
188         // amount of time TBD
189         for (String peerId : votingPeers) {
190             ActorSelection peerActor = context.getPeerActorSelection(peerId);
191             if (peerActor != null) {
192                 RequestVote requestVote = new RequestVote(
193                         context.getTermInformation().getCurrentTerm(),
194                         context.getId(),
195                         context.getReplicatedLog().lastIndex(),
196                         context.getReplicatedLog().lastTerm());
197
198                 log.debug("{}: Sending {} to peer {}", logName(), requestVote, peerId);
199
200                 peerActor.tell(requestVote, context.getActor());
201             }
202         }
203     }
204
205     @Override
206     public void close() {
207         stopElection();
208     }
209 }