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