Adding the wrongly removed module back in pom.xml
[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.internal.messages.SendInstallSnapshot;
26 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
27 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
28 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
29 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
30 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
31 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
32 import scala.concurrent.duration.FiniteDuration;
33
34 import java.util.ArrayList;
35 import java.util.Collections;
36 import java.util.HashMap;
37 import java.util.List;
38 import java.util.Map;
39 import java.util.concurrent.TimeUnit;
40 import java.util.concurrent.atomic.AtomicLong;
41
42 /**
43  * The behavior of a RaftActor when it is in the Leader state
44  * <p/>
45  * Leaders:
46  * <ul>
47  * <li> Upon election: send initial empty AppendEntries RPCs
48  * (heartbeat) to each server; repeat during idle periods to
49  * prevent election timeouts (§5.2)
50  * <li> If command received from client: append entry to local log,
51  * respond after entry applied to state machine (§5.3)
52  * <li> If last log index ≥ nextIndex for a follower: send
53  * AppendEntries RPC with log entries starting at nextIndex
54  * <ul>
55  * <li> If successful: update nextIndex and matchIndex for
56  * follower (§5.3)
57  * <li> If AppendEntries fails because of log inconsistency:
58  * decrement nextIndex and retry (§5.3)
59  * </ul>
60  * <li> If there exists an N such that N > commitIndex, a majority
61  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
62  * set commitIndex = N (§5.3, §5.4).
63  */
64 public class Leader extends AbstractRaftActorBehavior {
65
66
67     private final Map<String, FollowerLogInformation> followerToLog =
68         new HashMap();
69
70     private final Map<String, ActorSelection> followerToActor = new HashMap<>();
71
72     private Cancellable heartbeatSchedule = null;
73     private Cancellable appendEntriesSchedule = null;
74     private Cancellable installSnapshotSchedule = null;
75
76     private List<ClientRequestTracker> trackerList = new ArrayList<>();
77
78     private final int minReplicationCount;
79
80     public Leader(RaftActorContext context) {
81         super(context);
82
83         if (lastIndex() >= 0) {
84             context.setCommitIndex(lastIndex());
85         }
86
87         for (String followerId : context.getPeerAddresses().keySet()) {
88             FollowerLogInformation followerLogInformation =
89                 new FollowerLogInformationImpl(followerId,
90                     new AtomicLong(lastIndex()),
91                     new AtomicLong(-1));
92
93             followerToActor.put(followerId,
94                 context.actorSelection(context.getPeerAddress(followerId)));
95
96             followerToLog.put(followerId, followerLogInformation);
97         }
98
99         context.getLogger().debug("Election:Leader has following peers:"+followerToActor.keySet());
100
101         if (followerToActor.size() > 0) {
102             minReplicationCount = (followerToActor.size() + 1) / 2 + 1;
103         } else {
104             minReplicationCount = 0;
105         }
106
107
108         // Immediately schedule a heartbeat
109         // Upon election: send initial empty AppendEntries RPCs
110         // (heartbeat) to each server; repeat during idle periods to
111         // prevent election timeouts (§5.2)
112         scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
113
114         scheduleInstallSnapshotCheck(
115             new FiniteDuration(HEART_BEAT_INTERVAL.length() * 1000,
116                 HEART_BEAT_INTERVAL.unit())
117         );
118
119     }
120
121     @Override protected RaftState handleAppendEntries(ActorRef sender,
122         AppendEntries appendEntries) {
123
124         return state();
125     }
126
127     @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
128         AppendEntriesReply appendEntriesReply) {
129
130         // Update the FollowerLogInformation
131         String followerId = appendEntriesReply.getFollowerId();
132         FollowerLogInformation followerLogInformation =
133             followerToLog.get(followerId);
134         if (appendEntriesReply.isSuccess()) {
135             followerLogInformation
136                 .setMatchIndex(appendEntriesReply.getLogLastIndex());
137             followerLogInformation
138                 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
139         } else {
140
141             // TODO: When we find that the follower is out of sync with the
142             // Leader we simply decrement that followers next index by 1.
143             // Would it be possible to do better than this? The RAFT spec
144             // does not explicitly deal with it but may be something for us to
145             // think about
146
147             followerLogInformation.decrNextIndex();
148         }
149
150         // Now figure out if this reply warrants a change in the commitIndex
151         // If there exists an N such that N > commitIndex, a majority
152         // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
153         // set commitIndex = N (§5.3, §5.4).
154         for (long N = context.getCommitIndex() + 1; ; N++) {
155             int replicatedCount = 1;
156
157             for (FollowerLogInformation info : followerToLog.values()) {
158                 if (info.getMatchIndex().get() >= N) {
159                     replicatedCount++;
160                 }
161             }
162
163             if (replicatedCount >= minReplicationCount) {
164                 ReplicatedLogEntry replicatedLogEntry =
165                     context.getReplicatedLog().get(N);
166                 if (replicatedLogEntry != null
167                     && replicatedLogEntry.getTerm()
168                     == currentTerm()) {
169                     context.setCommitIndex(N);
170                 }
171             } else {
172                 break;
173             }
174         }
175
176         // Apply the change to the state machine
177         if (context.getCommitIndex() > context.getLastApplied()) {
178             applyLogToStateMachine(context.getCommitIndex());
179         }
180
181         return state();
182     }
183
184     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
185         for (ClientRequestTracker tracker : trackerList) {
186             if (tracker.getIndex() == logIndex) {
187                 return tracker;
188             }
189         }
190
191         return null;
192     }
193
194     @Override protected RaftState handleRequestVoteReply(ActorRef sender,
195         RequestVoteReply requestVoteReply) {
196         return state();
197     }
198
199     @Override public RaftState state() {
200         return RaftState.Leader;
201     }
202
203     @Override public RaftState handleMessage(ActorRef sender, Object originalMessage) {
204         Preconditions.checkNotNull(sender, "sender should not be null");
205
206         Object message = fromSerializableMessage(originalMessage);
207
208         if (message instanceof RaftRPC) {
209             RaftRPC rpc = (RaftRPC) message;
210             // If RPC request or response contains term T > currentTerm:
211             // set currentTerm = T, convert to follower (§5.1)
212             // This applies to all RPC messages and responses
213             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
214                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
215                 return RaftState.Follower;
216             }
217         }
218
219         try {
220             if (message instanceof SendHeartBeat) {
221                 return sendHeartBeat();
222             } else if(message instanceof SendInstallSnapshot) {
223                 installSnapshotIfNeeded();
224             } else if (message instanceof Replicate) {
225                 replicate((Replicate) message);
226             } else if (message instanceof InstallSnapshotReply){
227                 handleInstallSnapshotReply(
228                     (InstallSnapshotReply) message);
229             }
230         } finally {
231             scheduleHeartBeat(HEART_BEAT_INTERVAL);
232         }
233
234         return super.handleMessage(sender, message);
235     }
236
237     private void handleInstallSnapshotReply(InstallSnapshotReply message) {
238         InstallSnapshotReply reply = message;
239         String followerId = reply.getFollowerId();
240         FollowerLogInformation followerLogInformation =
241             followerToLog.get(followerId);
242
243         followerLogInformation
244             .setMatchIndex(context.getReplicatedLog().getSnapshotIndex());
245         followerLogInformation
246             .setNextIndex(context.getReplicatedLog().getSnapshotIndex() + 1);
247     }
248
249     private void replicate(Replicate replicate) {
250         long logIndex = replicate.getReplicatedLogEntry().getIndex();
251
252         context.getLogger().debug("Replicate message " + logIndex);
253
254         if (followerToActor.size() == 0) {
255             context.setCommitIndex(
256                 replicate.getReplicatedLogEntry().getIndex());
257
258             context.getActor()
259                 .tell(new ApplyState(replicate.getClientActor(),
260                         replicate.getIdentifier(),
261                         replicate.getReplicatedLogEntry()),
262                     context.getActor()
263                 );
264         } else {
265
266             // Create a tracker entry we will use this later to notify the
267             // client actor
268             trackerList.add(
269                 new ClientRequestTrackerImpl(replicate.getClientActor(),
270                     replicate.getIdentifier(),
271                     logIndex)
272             );
273
274             sendAppendEntries();
275         }
276     }
277
278     private void sendAppendEntries() {
279         // Send an AppendEntries to all followers
280         for (String followerId : followerToActor.keySet()) {
281             ActorSelection followerActor =
282                 followerToActor.get(followerId);
283
284             FollowerLogInformation followerLogInformation =
285                 followerToLog.get(followerId);
286
287             long nextIndex = followerLogInformation.getNextIndex().get();
288
289             List<ReplicatedLogEntry> entries = Collections.emptyList();
290
291             if(context.getReplicatedLog().isPresent(nextIndex)){
292                 // TODO: Instead of sending all entries from nextIndex
293                 // only send a fixed number of entries to each follower
294                 // This is to avoid the situation where there are a lot of
295                 // entries to install for a fresh follower or to a follower
296                 // that has fallen too far behind with the log but yet is not
297                 // eligible to receive a snapshot
298                 entries =
299                     context.getReplicatedLog().getFrom(nextIndex);
300             }
301
302             followerActor.tell(
303                 new AppendEntries(currentTerm(), context.getId(), prevLogIndex(nextIndex),
304                     prevLogTerm(nextIndex), entries, context.getCommitIndex()).toSerializable(),
305                 actor());
306         }
307     }
308
309     /**
310      * An installSnapshot is scheduled at a interval that is a multiple of
311      * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
312      * snapshots at every heartbeat.
313      */
314     private void installSnapshotIfNeeded(){
315         for (String followerId : followerToActor.keySet()) {
316             ActorSelection followerActor =
317                 followerToActor.get(followerId);
318
319             FollowerLogInformation followerLogInformation =
320                 followerToLog.get(followerId);
321
322             long nextIndex = followerLogInformation.getNextIndex().get();
323
324             if(!context.getReplicatedLog().isPresent(nextIndex) && context.getReplicatedLog().isInSnapshot(nextIndex)){
325                 followerActor.tell(
326                     new InstallSnapshot(currentTerm(), context.getId(),
327                         context.getReplicatedLog().getSnapshotIndex(),
328                         context.getReplicatedLog().getSnapshotTerm(),
329                         context.getReplicatedLog().getSnapshot()
330                     ),
331                     actor()
332                 );
333             }
334         }
335     }
336
337     private RaftState sendHeartBeat() {
338         if (followerToActor.size() > 0) {
339             sendAppendEntries();
340         }
341         return state();
342     }
343
344     private void stopHeartBeat() {
345         if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
346             heartbeatSchedule.cancel();
347         }
348     }
349
350     private void stopInstallSnapshotSchedule() {
351         if (installSnapshotSchedule != null && !installSnapshotSchedule.isCancelled()) {
352             installSnapshotSchedule.cancel();
353         }
354     }
355
356     private void scheduleHeartBeat(FiniteDuration interval) {
357         if(followerToActor.keySet().size() == 0){
358             // Optimization - do not bother scheduling a heartbeat as there are
359             // no followers
360             return;
361         }
362
363         stopHeartBeat();
364
365         // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
366         // message is sent to itself.
367         // Scheduling the heartbeat only once here because heartbeats do not
368         // need to be sent if there are other messages being sent to the remote
369         // actor.
370         heartbeatSchedule =
371             context.getActorSystem().scheduler().scheduleOnce(
372                 interval,
373                 context.getActor(), new SendHeartBeat(),
374                 context.getActorSystem().dispatcher(), context.getActor());
375     }
376
377
378     private void scheduleInstallSnapshotCheck(FiniteDuration interval) {
379         if(followerToActor.keySet().size() == 0){
380             // Optimization - do not bother scheduling a heartbeat as there are
381             // no followers
382             return;
383         }
384
385         stopInstallSnapshotSchedule();
386
387         // Schedule a message to send append entries to followers that can
388         // accept an append entries with some data in it
389         installSnapshotSchedule =
390             context.getActorSystem().scheduler().scheduleOnce(
391                 interval,
392                 context.getActor(), new SendInstallSnapshot(),
393                 context.getActorSystem().dispatcher(), context.getActor());
394     }
395
396
397
398     @Override public void close() throws Exception {
399         stopHeartBeat();
400     }
401
402     @Override public String getLeaderId() {
403         return context.getId();
404     }
405
406 }