Bug 2187: AddServer: check if already exists
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.java
1 /*
2  * Copyright (c) 2015 Brocade Communications 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;
9
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import akka.actor.Cancellable;
13 import com.google.common.base.Preconditions;
14 import java.util.ArrayList;
15 import java.util.Collection;
16 import java.util.LinkedList;
17 import java.util.List;
18 import java.util.Queue;
19 import java.util.UUID;
20 import java.util.concurrent.TimeUnit;
21 import javax.annotation.Nullable;
22 import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload.ServerInfo;
23 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
24 import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete;
25 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
26 import org.opendaylight.controller.cluster.raft.messages.AddServer;
27 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
28 import org.opendaylight.controller.cluster.raft.messages.FollowerCatchUpTimeout;
29 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
30 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
31 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
32 import org.slf4j.Logger;
33 import org.slf4j.LoggerFactory;
34 import scala.concurrent.duration.FiniteDuration;
35
36 /**
37  * Handles server configuration related messages for a RaftActor.
38  *
39  * @author Thomas Pantelis
40  */
41 class RaftActorServerConfigurationSupport {
42     private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class);
43
44     private final OperationState IDLE = new Idle();
45
46     private final RaftActorContext raftContext;
47
48     private final Queue<ServerOperationContext<?>> pendingOperationsQueue = new LinkedList<>();
49
50     private OperationState currentOperationState = IDLE;
51
52     RaftActorServerConfigurationSupport(RaftActorContext context) {
53         this.raftContext = context;
54     }
55
56     boolean handleMessage(Object message, RaftActor raftActor, ActorRef sender) {
57         if(message instanceof AddServer) {
58             onAddServer((AddServer)message, raftActor, sender);
59             return true;
60         } else if (message instanceof FollowerCatchUpTimeout) {
61             currentOperationState.onFollowerCatchupTimeout(raftActor, (FollowerCatchUpTimeout)message);
62             return true;
63         } else if (message instanceof UnInitializedFollowerSnapshotReply) {
64             currentOperationState.onUnInitializedFollowerSnapshotReply(raftActor,
65                     (UnInitializedFollowerSnapshotReply)message);
66             return true;
67         } else if(message instanceof ApplyState) {
68             return onApplyState((ApplyState) message, raftActor);
69         } else if(message instanceof SnapshotComplete) {
70             currentOperationState.onSnapshotComplete(raftActor);
71             return false;
72         } else {
73             return false;
74         }
75     }
76
77     private boolean onApplyState(ApplyState applyState, RaftActor raftActor) {
78         Payload data = applyState.getReplicatedLogEntry().getData();
79         if(data instanceof ServerConfigurationPayload) {
80             currentOperationState.onApplyState(raftActor, applyState);
81             return true;
82         }
83
84         return false;
85     }
86
87     /**
88      * The algorithm for AddServer is as follows:
89      * <ul>
90      * <li>Add the new server as a peer.</li>
91      * <li>Add the new follower to the leader.</li>
92      * <li>If new server should be voting member</li>
93      * <ul>
94      *     <li>Initialize FollowerState to VOTING_NOT_INITIALIZED.</li>
95      *     <li>Initiate install snapshot to the new follower.</li>
96      *     <li>When install snapshot complete, mark the follower as VOTING and re-calculate majority vote count.</li>
97      * </ul>
98      * <li>Persist and replicate ServerConfigurationPayload with the new server list.</li>
99      * <li>On replication consensus, respond to caller with OK.</li>
100      * </ul>
101      * If the install snapshot times out after a period of 2 * election time out
102      * <ul>
103      *     <li>Remove the new server as a peer.</li>
104      *     <li>Remove the new follower from the leader.</li>
105      *     <li>Respond to caller with TIMEOUT.</li>
106      * </ul>
107      */
108     private void onAddServer(AddServer addServer, RaftActor raftActor, ActorRef sender) {
109         LOG.debug("{}: onAddServer: {}", raftContext.getId(), addServer);
110
111         onNewOperation(raftActor, new AddServerContext(addServer, sender));
112     }
113
114     private void onNewOperation(RaftActor raftActor, ServerOperationContext<?> operationContext) {
115         if (raftActor.isLeader()) {
116             currentOperationState.onNewOperation(raftActor, operationContext);
117         } else {
118             ActorSelection leader = raftActor.getLeader();
119             if (leader != null) {
120                 LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader);
121                 leader.forward(operationContext.getOperation(), raftActor.getContext());
122             } else {
123                 LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId());
124                 operationContext.getClientRequestor().tell(operationContext.newReply(
125                         ServerChangeStatus.NO_LEADER, null), raftActor.self());
126             }
127         }
128     }
129
130     /**
131      * Interface for a server operation FSM state.
132      */
133     private interface OperationState {
134         void onNewOperation(RaftActor raftActor, ServerOperationContext<?> operationContext);
135
136         void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout);
137
138         void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, UnInitializedFollowerSnapshotReply reply);
139
140         void onApplyState(RaftActor raftActor, ApplyState applyState);
141
142         void onSnapshotComplete(RaftActor raftActor);
143     }
144
145     /**
146      * Interface for the initial state for a server operation.
147      */
148     private interface InitialOperationState {
149         void initiate(RaftActor raftActor);
150     }
151
152     /**
153      * Abstract base class for server operation FSM state. Handles common behavior for all states.
154      */
155     private abstract class AbstractOperationState implements OperationState {
156         @Override
157         public void onNewOperation(RaftActor raftActor, ServerOperationContext<?> operationContext) {
158             // We're currently processing another operation so queue it to be processed later.
159
160             LOG.debug("{}: Server operation already in progress - queueing {}", raftContext.getId(),
161                     operationContext.getOperation());
162
163             pendingOperationsQueue.add(operationContext);
164         }
165
166         @Override
167         public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) {
168             LOG.debug("onFollowerCatchupTimeout should not be called in state {}", this);
169         }
170
171         @Override
172         public void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, UnInitializedFollowerSnapshotReply reply) {
173             LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this);
174         }
175
176         @Override
177         public void onApplyState(RaftActor raftActor, ApplyState applyState) {
178             LOG.debug("onApplyState was called in state {}", this);
179         }
180
181         @Override
182         public void onSnapshotComplete(RaftActor raftActor) {
183         }
184
185         protected void persistNewServerConfiguration(RaftActor raftActor, ServerOperationContext<?> operationContext){
186             Collection<PeerInfo> peers = raftContext.getPeers();
187             List<ServerInfo> newConfig = new ArrayList<>(peers.size() + 1);
188             for(PeerInfo peer: peers) {
189                 newConfig.add(new ServerInfo(peer.getId(), peer.isVoting()));
190             }
191
192             newConfig.add(new ServerInfo(raftContext.getId(), true));
193
194             LOG.debug("{}: Persisting new server configuration : {}", raftContext.getId(), newConfig);
195
196             ServerConfigurationPayload payload = new ServerConfigurationPayload(newConfig);
197
198             raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload);
199
200             currentOperationState = new Persisting(operationContext);
201
202             sendReply(raftActor, operationContext, ServerChangeStatus.OK);
203         }
204
205         protected void operationComplete(RaftActor raftActor, ServerOperationContext<?> operationContext,
206                 @Nullable ServerChangeStatus replyStatus) {
207             if(replyStatus != null) {
208                 sendReply(raftActor, operationContext, replyStatus);
209             }
210
211             currentOperationState = IDLE;
212
213             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
214             if(nextOperation != null) {
215                 RaftActorServerConfigurationSupport.this.onNewOperation(raftActor, nextOperation);
216             }
217         }
218
219         private void sendReply(RaftActor raftActor, ServerOperationContext<?> operationContext,
220                 ServerChangeStatus status) {
221             LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation());
222
223             operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()),
224                     raftActor.self());
225         }
226
227         @Override
228         public String toString() {
229             return getClass().getSimpleName();
230         }
231     }
232
233     /**
234      * The state when no server operation is in progress. It immediately initiates new server operations.
235      */
236     private class Idle extends AbstractOperationState {
237         @Override
238         public void onNewOperation(RaftActor raftActor, ServerOperationContext<?> operationContext) {
239             operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate(raftActor);
240         }
241
242         @Override
243         public void onApplyState(RaftActor raftActor, ApplyState applyState) {
244             // Noop - we override b/c ApplyState is called normally for followers in the idle state.
245         }
246     }
247
248     /**
249      * The state when a new server configuration is being persisted and replicated.
250      */
251     private class Persisting extends AbstractOperationState {
252         private final ServerOperationContext<?> operationContext;
253
254         Persisting(ServerOperationContext<?> operationContext) {
255             this.operationContext = operationContext;
256         }
257
258         @Override
259         public void onApplyState(RaftActor raftActor, ApplyState applyState) {
260             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
261             // sure it's meant for us.
262             if(operationContext.getContextId().equals(applyState.getIdentifier())) {
263                 LOG.info("{}: {} has been successfully replicated to a majority of followers",
264                         applyState.getReplicatedLogEntry().getData());
265
266                 operationComplete(raftActor, operationContext, null);
267             }
268         }
269     }
270
271     /**
272      * Abstract base class for an AddServer operation state.
273      */
274     private abstract class AddServerState extends AbstractOperationState {
275         private final AddServerContext addServerContext;
276
277         AddServerState(AddServerContext addServerContext) {
278             this.addServerContext = addServerContext;
279         }
280
281         AddServerContext getAddServerContext() {
282             return addServerContext;
283         }
284
285         Cancellable newInstallSnapshotTimer(RaftActor raftActor) {
286             return raftContext.getActorSystem().scheduler().scheduleOnce(
287                     new FiniteDuration(((raftContext.getConfigParams().getElectionTimeOutInterval().toMillis()) * 2),
288                             TimeUnit.MILLISECONDS), raftContext.getActor(),
289                             new FollowerCatchUpTimeout(addServerContext.getOperation().getNewServerId()),
290                             raftContext.getActorSystem().dispatcher(), raftContext.getActor());
291         }
292
293         void handleOnFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) {
294             String serverId = followerTimeout.getNewServerId();
295
296             LOG.debug("{}: onFollowerCatchupTimeout for new server {}", raftContext.getId(), serverId);
297
298             // cleanup
299             raftContext.removePeer(serverId);
300
301             boolean isLeader = raftActor.isLeader();
302             if(isLeader) {
303                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
304                 leader.removeFollower(serverId);
305             }
306
307             operationComplete(raftActor, getAddServerContext(),
308                     isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER);
309         }
310     }
311
312     /**
313      * The initial state for the AddServer operation. It adds the new follower as a peer and initiates
314      * snapshot capture, if necessary.
315      */
316     private class InitialAddServerState extends AddServerState implements InitialOperationState {
317         InitialAddServerState(AddServerContext addServerContext) {
318             super(addServerContext);
319         }
320
321         @Override
322         public void initiate(RaftActor raftActor) {
323             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
324             AddServer addServer = getAddServerContext().getOperation();
325
326             LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
327
328             if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
329                 operationComplete(raftActor, getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
330                 return;
331             }
332
333             VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED :
334                     VotingState.NON_VOTING;
335             raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState);
336
337             leader.addFollower(addServer.getNewServerId());
338
339             if(votingState == VotingState.VOTING_NOT_INITIALIZED){
340                 // schedule the install snapshot timeout timer
341                 Cancellable installSnapshotTimer = newInstallSnapshotTimer(raftActor);
342                 if(leader.initiateCaptureSnapshot(addServer.getNewServerId())) {
343                     LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
344                             addServer.getNewServerId());
345
346                     currentOperationState = new InstallingSnapshot(getAddServerContext(), installSnapshotTimer);
347                 } else {
348                     LOG.debug("{}: Snapshot already in progress - waiting for completion", raftContext.getId());
349
350                     currentOperationState = new WaitingForPriorSnapshotComplete(getAddServerContext(),
351                             installSnapshotTimer);
352                 }
353             } else {
354                 LOG.debug("{}: New follower is non-voting - directly persisting new server configuration",
355                         raftContext.getId());
356
357                 persistNewServerConfiguration(raftActor, getAddServerContext());
358             }
359         }
360     }
361
362     /**
363      * The AddServer operation state for when the catch-up snapshot is being installed. It handles successful
364      * reply or timeout.
365      */
366     private class InstallingSnapshot extends AddServerState {
367         private final Cancellable installSnapshotTimer;
368
369         InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) {
370             super(addServerContext);
371             this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer);
372         }
373
374         @Override
375         public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) {
376             handleOnFollowerCatchupTimeout(raftActor, followerTimeout);
377
378             LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
379                     followerTimeout.getNewServerId());
380         }
381
382         @Override
383         public void onUnInitializedFollowerSnapshotReply(RaftActor raftActor, UnInitializedFollowerSnapshotReply reply) {
384             LOG.debug("{}: onUnInitializedFollowerSnapshotReply: {}", raftContext.getId(), reply);
385
386             String followerId = reply.getFollowerId();
387
388             // Sanity check to guard against receiving an UnInitializedFollowerSnapshotReply from a prior
389             // add server operation that timed out.
390             if(getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
391                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
392                 raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING);
393                 leader.updateMinReplicaCount();
394
395                 persistNewServerConfiguration(raftActor, getAddServerContext());
396
397                 installSnapshotTimer.cancel();
398             } else {
399                 LOG.debug("{}: Dropping UnInitializedFollowerSnapshotReply for server {}: {}",
400                         raftContext.getId(), followerId,
401                         !raftActor.isLeader() ? "not leader" : "server Id doesn't match");
402             }
403         }
404     }
405
406     /**
407      * The AddServer operation state for when there is a snapshot already in progress. When the current
408      * snapshot completes, it initiates an install snapshot.
409      */
410     private class WaitingForPriorSnapshotComplete extends AddServerState {
411         private final Cancellable snapshotTimer;
412
413         WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) {
414             super(addServerContext);
415             this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer);
416         }
417
418         @Override
419         public void onSnapshotComplete(RaftActor raftActor) {
420             LOG.debug("{}: onSnapshotComplete", raftContext.getId());
421
422             if(!raftActor.isLeader()) {
423                 LOG.debug("{}: No longer the leader", raftContext.getId());
424                 return;
425             }
426
427             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
428             if(leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) {
429                 LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
430                         getAddServerContext().getOperation().getNewServerId());
431
432                 currentOperationState = new InstallingSnapshot(getAddServerContext(),
433                         newInstallSnapshotTimer(raftActor));
434
435                 snapshotTimer.cancel();
436             }
437         }
438
439         @Override
440         public void onFollowerCatchupTimeout(RaftActor raftActor, FollowerCatchUpTimeout followerTimeout) {
441             handleOnFollowerCatchupTimeout(raftActor, followerTimeout);
442
443             LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
444                     raftContext.getId(), followerTimeout.getNewServerId());
445         }
446     }
447
448     /**
449      * Stores context information for a server operation.
450      *
451      * @param <T> the operation type
452      */
453     private static abstract class ServerOperationContext<T> {
454         private final T operation;
455         private final ActorRef clientRequestor;
456         private final String contextId;
457
458         ServerOperationContext(T operation, ActorRef clientRequestor){
459             this.operation = operation;
460             this.clientRequestor = clientRequestor;
461             contextId = UUID.randomUUID().toString();
462         }
463
464         String getContextId() {
465             return contextId;
466         }
467
468         T getOperation() {
469             return operation;
470         }
471
472         ActorRef getClientRequestor() {
473             return clientRequestor;
474         }
475
476         abstract Object newReply(ServerChangeStatus status, String leaderId);
477
478         abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support);
479     }
480
481     /**
482      * Stores context information for an AddServer operation.
483      */
484     private static class AddServerContext extends ServerOperationContext<AddServer> {
485         AddServerContext(AddServer addServer, ActorRef clientRequestor) {
486             super(addServer, clientRequestor);
487         }
488
489         @Override
490         Object newReply(ServerChangeStatus status, String leaderId) {
491             return new AddServerReply(status, leaderId);
492         }
493
494         @Override
495         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
496             return support.new InitialAddServerState(this);
497         }
498     }
499 }