c06ee9bd2b836c784c4adb75e7f318875ebf22c3
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Leader.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 com.google.common.base.Preconditions;
15 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
16 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
17 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
18 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
19 import org.opendaylight.controller.cluster.raft.RaftActorContext;
20 import org.opendaylight.controller.cluster.raft.RaftState;
21 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
22 import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
23 import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
24 import org.opendaylight.controller.cluster.raft.internal.messages.SendHeartBeat;
25 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
26 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
27 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
28 import scala.concurrent.duration.FiniteDuration;
29
30 import java.util.ArrayList;
31 import java.util.HashMap;
32 import java.util.List;
33 import java.util.Map;
34 import java.util.concurrent.TimeUnit;
35 import java.util.concurrent.atomic.AtomicLong;
36
37 /**
38  * The behavior of a RaftActor when it is in the Leader state
39  * <p/>
40  * Leaders:
41  * <ul>
42  * <li> Upon election: send initial empty AppendEntries RPCs
43  * (heartbeat) to each server; repeat during idle periods to
44  * prevent election timeouts (§5.2)
45  * <li> If command received from client: append entry to local log,
46  * respond after entry applied to state machine (§5.3)
47  * <li> If last log index ≥ nextIndex for a follower: send
48  * AppendEntries RPC with log entries starting at nextIndex
49  * <ul>
50  * <li> If successful: update nextIndex and matchIndex for
51  * follower (§5.3)
52  * <li> If AppendEntries fails because of log inconsistency:
53  * decrement nextIndex and retry (§5.3)
54  * </ul>
55  * <li> If there exists an N such that N > commitIndex, a majority
56  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
57  * set commitIndex = N (§5.3, §5.4).
58  */
59 public class Leader extends AbstractRaftActorBehavior {
60
61
62     private final Map<String, FollowerLogInformation> followerToLog =
63         new HashMap();
64
65     private final Map<String, ActorSelection> followerToActor = new HashMap<>();
66
67     private Cancellable heartbeatCancel = null;
68
69     private List<ClientRequestTracker> trackerList = new ArrayList<>();
70
71     private final int minReplicationCount;
72
73     public Leader(RaftActorContext context) {
74         super(context);
75
76         if(lastIndex() >= 0) {
77             context.setCommitIndex(lastIndex());
78         }
79
80         for (String followerId : context.getPeerAddresses().keySet()) {
81             FollowerLogInformation followerLogInformation =
82                 new FollowerLogInformationImpl(followerId,
83                     new AtomicLong(lastIndex()),
84                     new AtomicLong(-1));
85
86             followerToActor.put(followerId,
87                 context.actorSelection(context.getPeerAddress(followerId)));
88
89             followerToLog.put(followerId, followerLogInformation);
90
91         }
92
93         if (followerToActor.size() > 0) {
94             minReplicationCount = (followerToActor.size() + 1) / 2 + 1;
95         } else {
96             minReplicationCount = 0;
97         }
98
99
100         // Immediately schedule a heartbeat
101         // Upon election: send initial empty AppendEntries RPCs
102         // (heartbeat) to each server; repeat during idle periods to
103         // prevent election timeouts (§5.2)
104         scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
105
106
107     }
108
109     @Override protected RaftState handleAppendEntries(ActorRef sender,
110         AppendEntries appendEntries, RaftState suggestedState) {
111
112         context.getLogger()
113             .error("An unexpected AppendEntries received in state " + state());
114
115         return suggestedState;
116     }
117
118     @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
119         AppendEntriesReply appendEntriesReply, RaftState suggestedState) {
120
121         // Do not take any other action since a behavior change is coming
122         if (suggestedState != state())
123             return suggestedState;
124
125         // Update the FollowerLogInformation
126         String followerId = appendEntriesReply.getFollowerId();
127         FollowerLogInformation followerLogInformation =
128             followerToLog.get(followerId);
129         if (appendEntriesReply.isSuccess()) {
130             followerLogInformation
131                 .setMatchIndex(appendEntriesReply.getLogLastIndex());
132             followerLogInformation
133                 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
134         } else {
135             followerLogInformation.decrNextIndex();
136         }
137
138         // Now figure out if this reply warrants a change in the commitIndex
139         // If there exists an N such that N > commitIndex, a majority
140         // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
141         // set commitIndex = N (§5.3, §5.4).
142         for (long N = context.getCommitIndex() + 1; ; N++) {
143             int replicatedCount = 1;
144
145             for (FollowerLogInformation info : followerToLog.values()) {
146                 if (info.getMatchIndex().get() >= N) {
147                     replicatedCount++;
148                 }
149             }
150
151             if (replicatedCount >= minReplicationCount){
152                 ReplicatedLogEntry replicatedLogEntry =
153                     context.getReplicatedLog().get(N);
154                 if (replicatedLogEntry != null
155                     && replicatedLogEntry.getTerm()
156                     == currentTerm()) {
157                     context.setCommitIndex(N);
158                 }
159             } else {
160                 break;
161             }
162         }
163
164         if(context.getCommitIndex() > context.getLastApplied()){
165             applyLogToStateMachine(context.getCommitIndex());
166         }
167
168         return suggestedState;
169     }
170
171     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
172         for (ClientRequestTracker tracker : trackerList) {
173             if (tracker.getIndex() == logIndex) {
174                 return tracker;
175             }
176         }
177
178         return null;
179     }
180
181     @Override protected RaftState handleRequestVoteReply(ActorRef sender,
182         RequestVoteReply requestVoteReply, RaftState suggestedState) {
183         return suggestedState;
184     }
185
186     @Override public RaftState state() {
187         return RaftState.Leader;
188     }
189
190     @Override public RaftState handleMessage(ActorRef sender, Object message) {
191         Preconditions.checkNotNull(sender, "sender should not be null");
192
193         try {
194             if (message instanceof SendHeartBeat) {
195                 return sendHeartBeat();
196             } else if (message instanceof Replicate) {
197
198                 Replicate replicate = (Replicate) message;
199                 long logIndex = replicate.getReplicatedLogEntry().getIndex();
200
201                 context.getLogger().debug("Replicate message " + logIndex);
202
203                 if (followerToActor.size() == 0) {
204                     context.setCommitIndex(
205                         replicate.getReplicatedLogEntry().getIndex());
206
207                     context.getActor()
208                         .tell(new ApplyState(replicate.getClientActor(),
209                                 replicate.getIdentifier(),
210                                 replicate.getReplicatedLogEntry()),
211                             context.getActor()
212                         );
213                 } else {
214
215                     trackerList.add(
216                         new ClientRequestTrackerImpl(replicate.getClientActor(),
217                             replicate.getIdentifier(),
218                             logIndex)
219                     );
220
221                     ReplicatedLogEntry prevEntry =
222                         context.getReplicatedLog().get(lastIndex() - 1);
223                     long prevLogIndex = -1;
224                     long prevLogTerm = -1;
225                     if (prevEntry != null) {
226                         prevLogIndex = prevEntry.getIndex();
227                         prevLogTerm = prevEntry.getTerm();
228                     }
229                     // Send an AppendEntries to all followers
230                     for (String followerId : followerToActor.keySet()) {
231                         ActorSelection followerActor =
232                             followerToActor.get(followerId);
233                         FollowerLogInformation followerLogInformation =
234                             followerToLog.get(followerId);
235                         followerActor.tell(
236                             new AppendEntries(currentTerm(), context.getId(),
237                                 prevLogIndex, prevLogTerm,
238                                 context.getReplicatedLog().getFrom(
239                                     followerLogInformation.getNextIndex()
240                                         .get()
241                                 ), context.getCommitIndex()
242                             ),
243                             actor()
244                         );
245                     }
246                 }
247             }
248         } finally {
249             scheduleHeartBeat(HEART_BEAT_INTERVAL);
250         }
251
252         return super.handleMessage(sender, message);
253     }
254
255     private RaftState sendHeartBeat() {
256         if (followerToActor.size() > 0) {
257             for (String follower : followerToActor.keySet()) {
258
259                 FollowerLogInformation followerLogInformation =
260                     followerToLog.get(follower);
261
262                 AtomicLong nextIndex =
263                     followerLogInformation.getNextIndex();
264
265                 List<ReplicatedLogEntry> entries =
266                     context.getReplicatedLog().getFrom(nextIndex.get());
267
268                 followerToActor.get(follower).tell(new AppendEntries(
269                         context.getTermInformation().getCurrentTerm(),
270                         context.getId(),
271                         context.getReplicatedLog().lastIndex(),
272                         context.getReplicatedLog().lastTerm(),
273                         entries, context.getCommitIndex()),
274                     context.getActor()
275                 );
276             }
277         }
278         return state();
279     }
280
281     private void stopHeartBeat() {
282         if (heartbeatCancel != null && !heartbeatCancel.isCancelled()) {
283             heartbeatCancel.cancel();
284         }
285     }
286
287     private void scheduleHeartBeat(FiniteDuration interval) {
288         stopHeartBeat();
289
290         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
291         // message is sent to itself.
292         // Scheduling the heartbeat only once here because heartbeats do not
293         // need to be sent if there are other messages being sent to the remote
294         // actor.
295         heartbeatCancel =
296             context.getActorSystem().scheduler().scheduleOnce(
297                 interval,
298                 context.getActor(), new SendHeartBeat(),
299                 context.getActorSystem().dispatcher(), context.getActor());
300     }
301
302     @Override public void close() throws Exception {
303         stopHeartBeat();
304     }
305
306     @Override public String getLeaderId() {
307         return context.getId();
308     }
309
310 }