Merge "BUG 1839 - HTTP delete of non existing data"
[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.base.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.Set;
23
24 /**
25  * The behavior of a RaftActor when it is in the CandidateState
26  * <p/>
27  * Candidates (§5.2):
28  * <ul>
29  * <li> On conversion to candidate, start election:
30  * <ul>
31  * <li> Increment currentTerm
32  * <li> Vote for self
33  * <li> Reset election timer
34  * <li> Send RequestVote RPCs to all other servers
35  * </ul>
36  * <li> If votes received from majority of servers: become leader
37  * <li> If AppendEntries RPC received from new leader: convert to
38  * follower
39  * <li> If election timeout elapses: start new election
40  * </ul>
41  */
42 public class Candidate extends AbstractRaftActorBehavior {
43
44     private int voteCount;
45
46     private final int votesRequired;
47
48     private final Set<String> peers;
49
50     public Candidate(RaftActorContext context) {
51         super(context);
52
53         peers = context.getPeerAddresses().keySet();
54
55         if(LOG.isDebugEnabled()) {
56             LOG.debug("Election:Candidate has following peers: {}", peers);
57         }
58
59         if(peers.size() > 0) {
60             // Votes are required from a majority of the peers including self.
61             // The votesRequired field therefore stores a calculated value
62             // of the number of votes required for this candidate to win an
63             // election based on it's known peers.
64             // If a peer was added during normal operation and raft replicas
65             // came to know about them then the new peer would also need to be
66             // taken into consideration when calculating this value.
67             // Here are some examples for what the votesRequired would be for n
68             // peers
69             // 0 peers = 1 votesRequired (0 + 1) / 2 + 1 = 1
70             // 2 peers = 2 votesRequired (2 + 1) / 2 + 1 = 2
71             // 4 peers = 3 votesRequired (4 + 1) / 2 + 1 = 3
72             int noOfPeers = peers.size();
73             int self = 1;
74             votesRequired = (noOfPeers + self) / 2 + 1;
75         } else {
76             votesRequired = 0;
77         }
78
79         startNewTerm();
80         scheduleElection(electionDuration());
81     }
82
83     @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
84         AppendEntries appendEntries) {
85
86         if(LOG.isDebugEnabled()) {
87             LOG.debug(appendEntries.toString());
88         }
89
90         return this;
91     }
92
93     @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
94         AppendEntriesReply appendEntriesReply) {
95
96         return this;
97     }
98
99     @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
100         RequestVoteReply requestVoteReply) {
101
102         if (requestVoteReply.isVoteGranted()) {
103             voteCount++;
104         }
105
106         if (voteCount >= votesRequired) {
107             return switchBehavior(new Leader(context));
108         }
109
110         return this;
111     }
112
113     @Override public RaftState state() {
114         return RaftState.Candidate;
115     }
116
117     @Override
118     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
119
120         Object message = fromSerializableMessage(originalMessage);
121
122         if (message instanceof RaftRPC) {
123
124             RaftRPC rpc = (RaftRPC) message;
125
126             if(LOG.isDebugEnabled()) {
127                 LOG.debug("RaftRPC message received {} my term is {}", rpc, context.getTermInformation().getCurrentTerm());
128             }
129
130             // If RPC request or response contains term T > currentTerm:
131             // set currentTerm = T, convert to follower (§5.1)
132             // This applies to all RPC messages and responses
133             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
134                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
135
136                 return switchBehavior(new Follower(context));
137             }
138         }
139
140         if (message instanceof ElectionTimeout) {
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 switchBehavior(new Leader(context));
149             }
150             startNewTerm();
151             scheduleElection(electionDuration());
152             return this;
153         }
154
155         return super.handleMessage(sender, message);
156     }
157
158
159     private void startNewTerm() {
160
161
162         // set voteCount back to 1 (that is voting for self)
163         voteCount = 1;
164
165         // Increment the election term and vote for self
166         long currentTerm = context.getTermInformation().getCurrentTerm();
167         context.getTermInformation().updateAndPersist(currentTerm + 1,
168             context.getId());
169
170         if(LOG.isDebugEnabled()) {
171             LOG.debug("Starting new term {}", (currentTerm + 1));
172         }
173
174         // Request for a vote
175         // TODO: Retry request for vote if replies do not arrive in a reasonable
176         // amount of time TBD
177         for (String peerId : peers) {
178             ActorSelection peerActor = context.getPeerActorSelection(peerId);
179             if(peerActor != null) {
180                 peerActor.tell(new RequestVote(
181                         context.getTermInformation().getCurrentTerm(),
182                         context.getId(),
183                         context.getReplicatedLog().lastIndex(),
184                         context.getReplicatedLog().lastTerm()),
185                     context.getActor()
186                 );
187             }
188         }
189
190
191     }
192
193     @Override public void close() throws Exception {
194         stopElection();
195     }
196 }