BUG-8618: refactor SyncStatusTracker state
[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.ArrayDeque;
15 import java.util.ArrayList;
16 import java.util.Collection;
17 import java.util.HashSet;
18 import java.util.List;
19 import java.util.Map;
20 import java.util.Queue;
21 import java.util.UUID;
22 import javax.annotation.Nullable;
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.base.messages.TimeoutNow;
26 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
27 import org.opendaylight.controller.cluster.raft.messages.AddServer;
28 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
29 import org.opendaylight.controller.cluster.raft.messages.ChangeServersVotingStatus;
30 import org.opendaylight.controller.cluster.raft.messages.RemoveServer;
31 import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
32 import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply;
33 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
34 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
35 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
36 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
37 import org.opendaylight.controller.cluster.raft.persisted.ServerInfo;
38 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
39 import org.opendaylight.yangtools.concepts.Identifier;
40 import org.opendaylight.yangtools.util.AbstractUUIDIdentifier;
41 import org.slf4j.Logger;
42 import org.slf4j.LoggerFactory;
43 import scala.concurrent.duration.FiniteDuration;
44
45 /**
46  * Handles server configuration related messages for a RaftActor.
47  *
48  * @author Thomas Pantelis
49  */
50 class RaftActorServerConfigurationSupport {
51     private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class);
52
53     @SuppressWarnings("checkstyle:MemberName")
54     private final OperationState IDLE = new Idle();
55
56     private final RaftActor raftActor;
57
58     private final RaftActorContext raftContext;
59
60     private final Queue<ServerOperationContext<?>> pendingOperationsQueue = new ArrayDeque<>();
61
62     private OperationState currentOperationState = IDLE;
63
64     RaftActorServerConfigurationSupport(RaftActor raftActor) {
65         this.raftActor = raftActor;
66         this.raftContext = raftActor.getRaftActorContext();
67     }
68
69     boolean handleMessage(Object message, ActorRef sender) {
70         if (message instanceof AddServer) {
71             onAddServer((AddServer) message, sender);
72             return true;
73         } else if (message instanceof RemoveServer) {
74             onRemoveServer((RemoveServer) message, sender);
75             return true;
76         } else if (message instanceof ChangeServersVotingStatus) {
77             onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender);
78             return true;
79         } else if (message instanceof ServerOperationTimeout) {
80             currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message);
81             return true;
82         } else if (message instanceof UnInitializedFollowerSnapshotReply) {
83             currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message);
84             return true;
85         } else if (message instanceof ApplyState) {
86             return onApplyState((ApplyState) message);
87         } else if (message instanceof SnapshotComplete) {
88             currentOperationState.onSnapshotComplete();
89             return false;
90         } else {
91             return false;
92         }
93     }
94
95     void onNewLeader(String leaderId) {
96         currentOperationState.onNewLeader(leaderId);
97     }
98
99     private void onChangeServersVotingStatus(ChangeServersVotingStatus message, ActorRef sender) {
100         LOG.debug("{}: onChangeServersVotingStatus: {}, state: {}", raftContext.getId(), message,
101                 currentOperationState);
102
103         // The following check is a special case. Normally we fail an operation if there's no leader.
104         // Consider a scenario where one has 2 geographically-separated 3-node clusters, one a primary and
105         // the other a backup such that if the primary cluster is lost, the backup can take over. In this
106         // scenario, we have a logical 6-node cluster where the primary sub-cluster is configured as voting
107         // and the backup sub-cluster as non-voting such that the primary cluster can make progress without
108         // consensus from the backup cluster while still replicating to the backup. On fail-over to the backup,
109         // a request would be sent to a member of the backup cluster to flip the voting states, ie make the
110         // backup sub-cluster voting and the lost primary non-voting. However since the primary majority
111         // cluster is lost, there would be no leader to apply, persist and replicate the server config change.
112         // Therefore, if the local server is currently non-voting and is to be changed to voting and there is
113         // no current leader, we will try to elect a leader using the new server config in order to replicate
114         // the change and progress.
115         boolean localServerChangingToVoting = Boolean.TRUE.equals(message
116                 .getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
117         boolean hasNoLeader = raftActor.getLeaderId() == null;
118         if (localServerChangingToVoting && !raftContext.isVotingMember() && hasNoLeader) {
119             currentOperationState.onNewOperation(new ChangeServersVotingStatusContext(message, sender, true));
120         } else {
121             onNewOperation(new ChangeServersVotingStatusContext(message, sender, false));
122         }
123     }
124
125     private void onRemoveServer(RemoveServer removeServer, ActorRef sender) {
126         LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
127         boolean isSelf = removeServer.getServerId().equals(raftContext.getId());
128         if (isSelf && !raftContext.hasFollowers()) {
129             sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()),
130                     raftActor.getSelf());
131         } else if (!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) {
132             sender.tell(new RemoveServerReply(ServerChangeStatus.DOES_NOT_EXIST, raftActor.getLeaderId()),
133                     raftActor.getSelf());
134         } else {
135             String serverAddress = isSelf ? raftActor.self().path().toString() :
136                 raftContext.getPeerAddress(removeServer.getServerId());
137             onNewOperation(new RemoveServerContext(removeServer, serverAddress, sender));
138         }
139     }
140
141     private boolean onApplyState(ApplyState applyState) {
142         Payload data = applyState.getReplicatedLogEntry().getData();
143         if (data instanceof ServerConfigurationPayload) {
144             currentOperationState.onApplyState(applyState);
145             return true;
146         }
147
148         return false;
149     }
150
151     /**
152      * The algorithm for AddServer is as follows:
153      * <ul>
154      * <li>Add the new server as a peer.</li>
155      * <li>Add the new follower to the leader.</li>
156      * <li>If new server should be voting member</li>
157      * <ul>
158      *     <li>Initialize FollowerState to VOTING_NOT_INITIALIZED.</li>
159      *     <li>Initiate install snapshot to the new follower.</li>
160      *     <li>When install snapshot complete, mark the follower as VOTING and re-calculate majority vote count.</li>
161      * </ul>
162      * <li>Persist and replicate ServerConfigurationPayload with the new server list.</li>
163      * <li>On replication consensus, respond to caller with OK.</li>
164      * </ul>
165      * If the install snapshot times out after a period of 2 * election time out
166      * <ul>
167      *     <li>Remove the new server as a peer.</li>
168      *     <li>Remove the new follower from the leader.</li>
169      *     <li>Respond to caller with TIMEOUT.</li>
170      * </ul>
171      */
172     private void onAddServer(AddServer addServer, ActorRef sender) {
173         LOG.debug("{}: onAddServer: {}, state: {}", raftContext.getId(), addServer, currentOperationState);
174
175         onNewOperation(new AddServerContext(addServer, sender));
176     }
177
178     private void onNewOperation(ServerOperationContext<?> operationContext) {
179         if (raftActor.isLeader()) {
180             currentOperationState.onNewOperation(operationContext);
181         } else {
182             ActorSelection leader = raftActor.getLeader();
183             if (leader != null) {
184                 LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader);
185                 leader.tell(operationContext.getOperation(), operationContext.getClientRequestor());
186             } else {
187                 LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId());
188                 operationContext.getClientRequestor().tell(operationContext.newReply(
189                         ServerChangeStatus.NO_LEADER, null), raftActor.self());
190             }
191         }
192     }
193
194     /**
195      * Interface for the initial state for a server operation.
196      */
197     private interface InitialOperationState {
198         void initiate();
199     }
200
201     /**
202      * Abstract base class for a server operation FSM state. Handles common behavior for all states.
203      */
204     private abstract class OperationState {
205         void onNewOperation(ServerOperationContext<?> operationContext) {
206             // We're currently processing another operation so queue it to be processed later.
207
208             LOG.debug("{}: Server operation already in progress - queueing {}", raftContext.getId(),
209                     operationContext.getOperation());
210
211             pendingOperationsQueue.add(operationContext);
212         }
213
214         void onServerOperationTimeout(ServerOperationTimeout timeout) {
215             LOG.debug("onServerOperationTimeout should not be called in state {}", this);
216         }
217
218         void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
219             LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this);
220         }
221
222         void onApplyState(ApplyState applyState) {
223             LOG.debug("onApplyState was called in state {}", this);
224         }
225
226         void onSnapshotComplete() {
227
228         }
229
230         void onNewLeader(String newLeader) {
231         }
232
233         protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext) {
234             raftContext.setDynamicServerConfigurationInUse();
235
236             ServerConfigurationPayload payload = raftContext.getPeerServerInfo(
237                     operationContext.includeSelfInNewConfiguration(raftActor));
238             LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig());
239
240             raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(),
241                     payload, false);
242
243             currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(
244                     operationContext.getLoggingContext())));
245
246             sendReply(operationContext, ServerChangeStatus.OK);
247         }
248
249         protected void operationComplete(ServerOperationContext<?> operationContext,
250                 @Nullable ServerChangeStatus replyStatus) {
251             if (replyStatus != null) {
252                 sendReply(operationContext, replyStatus);
253             }
254
255             operationContext.operationComplete(raftActor, replyStatus == null || replyStatus == ServerChangeStatus.OK);
256
257             changeToIdleState();
258         }
259
260         protected void changeToIdleState() {
261             currentOperationState = IDLE;
262
263             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
264             if (nextOperation != null) {
265                 RaftActorServerConfigurationSupport.this.onNewOperation(nextOperation);
266             }
267         }
268
269         protected void sendReply(ServerOperationContext<?> operationContext, ServerChangeStatus status) {
270             LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status,
271                     operationContext.getOperation());
272
273             operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()),
274                     raftActor.self());
275         }
276
277         Cancellable newTimer(Object message) {
278             return newTimer(raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), message);
279         }
280
281         Cancellable newTimer(FiniteDuration timeout, Object message) {
282             return raftContext.getActorSystem().scheduler().scheduleOnce(
283                     timeout, raftContext.getActor(), message,
284                             raftContext.getActorSystem().dispatcher(), raftContext.getActor());
285         }
286
287         @Override
288         public String toString() {
289             return getClass().getSimpleName();
290         }
291     }
292
293     /**
294      * The state when no server operation is in progress. It immediately initiates new server operations.
295      */
296     private final class Idle extends OperationState {
297         @Override
298         public void onNewOperation(ServerOperationContext<?> operationContext) {
299             operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate();
300         }
301
302         @Override
303         public void onApplyState(ApplyState applyState) {
304             // Noop - we override b/c ApplyState is called normally for followers in the idle state.
305         }
306     }
307
308     /**
309      * The state when a new server configuration is being persisted and replicated.
310      */
311     private final class Persisting extends OperationState {
312         private final ServerOperationContext<?> operationContext;
313         private final Cancellable timer;
314         private boolean timedOut = false;
315
316         Persisting(ServerOperationContext<?> operationContext, Cancellable timer) {
317             this.operationContext = operationContext;
318             this.timer = timer;
319         }
320
321         @Override
322         public void onApplyState(ApplyState applyState) {
323             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
324             // sure it's meant for us.
325             if (operationContext.getContextId().equals(applyState.getIdentifier())) {
326                 LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(),
327                         applyState.getReplicatedLogEntry().getData());
328
329                 timer.cancel();
330                 operationComplete(operationContext, null);
331             }
332         }
333
334         @Override
335         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
336             LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(),
337                     timeout.getLoggingContext());
338
339             timedOut = true;
340
341             // Fail any pending operations
342             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
343             while (nextOperation != null) {
344                 sendReply(nextOperation, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
345                 nextOperation = pendingOperationsQueue.poll();
346             }
347         }
348
349         @Override
350         public void onNewOperation(ServerOperationContext<?> newOperationContext) {
351             if (timedOut) {
352                 sendReply(newOperationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
353             } else {
354                 super.onNewOperation(newOperationContext);
355             }
356         }
357     }
358
359     /**
360      * Abstract base class for an AddServer operation state.
361      */
362     private abstract class AddServerState extends OperationState {
363         private final AddServerContext addServerContext;
364
365         AddServerState(AddServerContext addServerContext) {
366             this.addServerContext = addServerContext;
367         }
368
369         AddServerContext getAddServerContext() {
370             return addServerContext;
371         }
372
373         Cancellable newInstallSnapshotTimer() {
374             return newTimer(new ServerOperationTimeout(addServerContext.getOperation().getNewServerId()));
375         }
376
377         void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) {
378             String serverId = timeout.getLoggingContext();
379
380             LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId);
381
382             // cleanup
383             raftContext.removePeer(serverId);
384
385             boolean isLeader = raftActor.isLeader();
386             if (isLeader) {
387                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
388                 leader.removeFollower(serverId);
389             }
390
391             operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT
392                     : ServerChangeStatus.NO_LEADER);
393         }
394
395     }
396
397     /**
398      * The initial state for the AddServer operation. It adds the new follower as a peer and initiates
399      * snapshot capture, if necessary.
400      */
401     private final class InitialAddServerState extends AddServerState implements InitialOperationState {
402         InitialAddServerState(AddServerContext addServerContext) {
403             super(addServerContext);
404         }
405
406         @Override
407         public void initiate() {
408             final AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
409             AddServer addServer = getAddServerContext().getOperation();
410
411             LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
412
413             if (raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
414                 operationComplete(getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
415                 return;
416             }
417
418             VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED :
419                     VotingState.NON_VOTING;
420             raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState);
421
422             leader.addFollower(addServer.getNewServerId());
423
424             if (votingState == VotingState.VOTING_NOT_INITIALIZED) {
425                 // schedule the install snapshot timeout timer
426                 Cancellable installSnapshotTimer = newInstallSnapshotTimer();
427                 if (leader.initiateCaptureSnapshot(addServer.getNewServerId())) {
428                     LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
429                             addServer.getNewServerId());
430
431                     currentOperationState = new InstallingSnapshot(getAddServerContext(), installSnapshotTimer);
432                 } else {
433                     LOG.debug("{}: Snapshot already in progress - waiting for completion", raftContext.getId());
434
435                     currentOperationState = new WaitingForPriorSnapshotComplete(getAddServerContext(),
436                             installSnapshotTimer);
437                 }
438             } else {
439                 LOG.debug("{}: New follower is non-voting - directly persisting new server configuration",
440                         raftContext.getId());
441
442                 persistNewServerConfiguration(getAddServerContext());
443             }
444         }
445     }
446
447     /**
448      * The AddServer operation state for when the catch-up snapshot is being installed. It handles successful
449      * reply or timeout.
450      */
451     private final class InstallingSnapshot extends AddServerState {
452         private final Cancellable installSnapshotTimer;
453
454         InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) {
455             super(addServerContext);
456             this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer);
457         }
458
459         @Override
460         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
461             handleInstallSnapshotTimeout(timeout);
462
463             LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
464                     timeout.getLoggingContext());
465         }
466
467         @Override
468         public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
469             LOG.debug("{}: onUnInitializedFollowerSnapshotReply: {}", raftContext.getId(), reply);
470
471             String followerId = reply.getFollowerId();
472
473             // Sanity check to guard against receiving an UnInitializedFollowerSnapshotReply from a prior
474             // add server operation that timed out.
475             if (getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
476                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
477                 raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING);
478                 leader.updateMinReplicaCount();
479
480                 persistNewServerConfiguration(getAddServerContext());
481
482                 installSnapshotTimer.cancel();
483             } else {
484                 LOG.debug("{}: Dropping UnInitializedFollowerSnapshotReply for server {}: {}",
485                         raftContext.getId(), followerId,
486                         !raftActor.isLeader() ? "not leader" : "server Id doesn't match");
487             }
488         }
489     }
490
491     /**
492      * The AddServer operation state for when there is a snapshot already in progress. When the current
493      * snapshot completes, it initiates an install snapshot.
494      */
495     private final class WaitingForPriorSnapshotComplete extends AddServerState {
496         private final Cancellable snapshotTimer;
497
498         WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) {
499             super(addServerContext);
500             this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer);
501         }
502
503         @Override
504         public void onSnapshotComplete() {
505             LOG.debug("{}: onSnapshotComplete", raftContext.getId());
506
507             if (!raftActor.isLeader()) {
508                 LOG.debug("{}: No longer the leader", raftContext.getId());
509                 return;
510             }
511
512             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
513             if (leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) {
514                 LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
515                         getAddServerContext().getOperation().getNewServerId());
516
517                 currentOperationState = new InstallingSnapshot(getAddServerContext(),
518                         newInstallSnapshotTimer());
519
520                 snapshotTimer.cancel();
521             }
522         }
523
524         @Override
525         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
526             handleInstallSnapshotTimeout(timeout);
527
528             LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
529                     raftContext.getId(), timeout.getLoggingContext());
530         }
531     }
532
533     private static final class ServerOperationContextIdentifier
534             extends AbstractUUIDIdentifier<ServerOperationContextIdentifier> {
535         private static final long serialVersionUID = 1L;
536
537         ServerOperationContextIdentifier() {
538             super(UUID.randomUUID());
539         }
540     }
541
542     /**
543      * Stores context information for a server operation.
544      *
545      * @param <T> the operation type
546      */
547     private abstract static class ServerOperationContext<T> {
548         private final T operation;
549         private final ActorRef clientRequestor;
550         private final Identifier contextId;
551
552         ServerOperationContext(T operation, ActorRef clientRequestor) {
553             this.operation = operation;
554             this.clientRequestor = clientRequestor;
555             contextId = new ServerOperationContextIdentifier();
556         }
557
558         Identifier getContextId() {
559             return contextId;
560         }
561
562         T getOperation() {
563             return operation;
564         }
565
566         ActorRef getClientRequestor() {
567             return clientRequestor;
568         }
569
570         void operationComplete(RaftActor raftActor, boolean succeeded) {
571         }
572
573         boolean includeSelfInNewConfiguration(RaftActor raftActor) {
574             return true;
575         }
576
577         abstract Object newReply(ServerChangeStatus status, String leaderId);
578
579         abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support);
580
581         abstract String getLoggingContext();
582     }
583
584     /**
585      * Stores context information for an AddServer operation.
586      */
587     private static class AddServerContext extends ServerOperationContext<AddServer> {
588         AddServerContext(AddServer addServer, ActorRef clientRequestor) {
589             super(addServer, clientRequestor);
590         }
591
592         @Override
593         Object newReply(ServerChangeStatus status, String leaderId) {
594             return new AddServerReply(status, leaderId);
595         }
596
597         @Override
598         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
599             return support.new InitialAddServerState(this);
600         }
601
602         @Override
603         String getLoggingContext() {
604             return getOperation().getNewServerId();
605         }
606     }
607
608     private abstract class RemoveServerState extends OperationState {
609         private final RemoveServerContext removeServerContext;
610
611         protected RemoveServerState(RemoveServerContext removeServerContext) {
612             this.removeServerContext = Preconditions.checkNotNull(removeServerContext);
613
614         }
615
616         public RemoveServerContext getRemoveServerContext() {
617             return removeServerContext;
618         }
619     }
620
621     private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState {
622
623         protected InitialRemoveServerState(RemoveServerContext removeServerContext) {
624             super(removeServerContext);
625         }
626
627         @Override
628         public void initiate() {
629             String serverId = getRemoveServerContext().getOperation().getServerId();
630             raftContext.removePeer(serverId);
631             AbstractLeader leader = (AbstractLeader)raftActor.getCurrentBehavior();
632             leader.removeFollower(serverId);
633             leader.updateMinReplicaCount();
634
635             persistNewServerConfiguration(getRemoveServerContext());
636         }
637     }
638
639     private static class RemoveServerContext extends ServerOperationContext<RemoveServer> {
640         private final String peerAddress;
641
642         RemoveServerContext(RemoveServer operation, String peerAddress, ActorRef clientRequestor) {
643             super(operation, clientRequestor);
644             this.peerAddress = peerAddress;
645         }
646
647         @Override
648         Object newReply(ServerChangeStatus status, String leaderId) {
649             return new RemoveServerReply(status, leaderId);
650         }
651
652         @Override
653         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
654             return support.new InitialRemoveServerState(this);
655         }
656
657         @Override
658         void operationComplete(RaftActor raftActor, boolean succeeded) {
659             if (peerAddress != null) {
660                 raftActor.context().actorSelection(peerAddress).tell(
661                         new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
662             }
663         }
664
665         @Override
666         boolean includeSelfInNewConfiguration(RaftActor raftActor) {
667             return !getOperation().getServerId().equals(raftActor.getId());
668         }
669
670         @Override
671         String getLoggingContext() {
672             return getOperation().getServerId();
673         }
674     }
675
676     private static class ChangeServersVotingStatusContext extends ServerOperationContext<ChangeServersVotingStatus> {
677         private final boolean tryToElectLeader;
678
679         ChangeServersVotingStatusContext(ChangeServersVotingStatus convertMessage, ActorRef clientRequestor,
680                 boolean tryToElectLeader) {
681             super(convertMessage, clientRequestor);
682             this.tryToElectLeader = tryToElectLeader;
683         }
684
685         @Override
686         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
687             return support.new ChangeServersVotingStatusState(this, tryToElectLeader);
688         }
689
690         @Override
691         Object newReply(ServerChangeStatus status, String leaderId) {
692             return new ServerChangeReply(status, leaderId);
693         }
694
695         @Override
696         void operationComplete(final RaftActor raftActor, boolean succeeded) {
697             // If this leader changed to non-voting we need to step down as leader so we'll try to transfer
698             // leadership.
699             boolean localServerChangedToNonVoting = Boolean.FALSE.equals(getOperation()
700                     .getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
701             if (succeeded && localServerChangedToNonVoting) {
702                 LOG.debug("Leader changed to non-voting - trying leadership transfer");
703                 raftActor.becomeNonVoting();
704             }
705         }
706
707         @Override
708         String getLoggingContext() {
709             return getOperation().toString();
710         }
711     }
712
713     private class ChangeServersVotingStatusState extends OperationState implements InitialOperationState {
714         private final ChangeServersVotingStatusContext changeVotingStatusContext;
715         private final boolean tryToElectLeader;
716
717         ChangeServersVotingStatusState(ChangeServersVotingStatusContext changeVotingStatusContext,
718                 boolean tryToElectLeader) {
719             this.changeVotingStatusContext = changeVotingStatusContext;
720             this.tryToElectLeader = tryToElectLeader;
721         }
722
723         @Override
724         public void initiate() {
725             LOG.debug("Initiating ChangeServersVotingStatusState");
726
727             if (tryToElectLeader) {
728                 initiateLocalLeaderElection();
729             } else if (updateLocalPeerInfo()) {
730                 persistNewServerConfiguration(changeVotingStatusContext);
731             }
732         }
733
734         private void initiateLocalLeaderElection() {
735             LOG.debug("{}: Sending local ElectionTimeout to start leader election", raftContext.getId());
736
737             ServerConfigurationPayload previousServerConfig = raftContext.getPeerServerInfo(true);
738             if (!updateLocalPeerInfo()) {
739                 return;
740             }
741
742             raftContext.getActor().tell(TimeoutNow.INSTANCE, raftContext.getActor());
743
744             currentOperationState = new WaitingForLeaderElected(changeVotingStatusContext, previousServerConfig);
745         }
746
747         private boolean updateLocalPeerInfo() {
748             List<ServerInfo> newServerInfoList = newServerInfoList();
749
750             // Check if new voting state would leave us with no voting members.
751             boolean atLeastOneVoting = false;
752             for (ServerInfo info: newServerInfoList) {
753                 if (info.isVoting()) {
754                     atLeastOneVoting = true;
755                     break;
756                 }
757             }
758
759             if (!atLeastOneVoting) {
760                 operationComplete(changeVotingStatusContext, ServerChangeStatus.INVALID_REQUEST);
761                 return false;
762             }
763
764             raftContext.updatePeerIds(new ServerConfigurationPayload(newServerInfoList));
765             if (raftActor.getCurrentBehavior() instanceof AbstractLeader) {
766                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
767                 leader.updateMinReplicaCount();
768             }
769
770             return true;
771         }
772
773         private List<ServerInfo> newServerInfoList() {
774             Map<String, Boolean> serverVotingStatusMap = changeVotingStatusContext.getOperation()
775                     .getServerVotingStatusMap();
776             List<ServerInfo> newServerInfoList = new ArrayList<>();
777             for (String peerId: raftContext.getPeerIds()) {
778                 newServerInfoList.add(new ServerInfo(peerId, serverVotingStatusMap.containsKey(peerId)
779                         ? serverVotingStatusMap.get(peerId) : raftContext.getPeerInfo(peerId).isVoting()));
780             }
781
782             newServerInfoList.add(new ServerInfo(raftContext.getId(), serverVotingStatusMap.containsKey(
783                     raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId())
784                             : raftContext.isVotingMember()));
785
786             return newServerInfoList;
787         }
788     }
789
790     private class WaitingForLeaderElected extends OperationState {
791         private final ServerConfigurationPayload previousServerConfig;
792         private final ChangeServersVotingStatusContext operationContext;
793         private final Cancellable timer;
794
795         WaitingForLeaderElected(ChangeServersVotingStatusContext operationContext,
796                 ServerConfigurationPayload previousServerConfig) {
797             this.operationContext = operationContext;
798             this.previousServerConfig = previousServerConfig;
799
800             timer = newTimer(raftContext.getConfigParams().getElectionTimeOutInterval(),
801                     new ServerOperationTimeout(operationContext.getLoggingContext()));
802         }
803
804         @Override
805         void onNewLeader(String newLeader) {
806             if (newLeader == null) {
807                 return;
808             }
809
810             LOG.debug("{}: New leader {} elected", raftContext.getId(), newLeader);
811
812             timer.cancel();
813
814             if (raftActor.isLeader()) {
815                 persistNewServerConfiguration(operationContext);
816             } else {
817                 // Edge case - some other node became leader so forward the operation.
818                 LOG.debug("{}: Forwarding {} to new leader", raftContext.getId(), operationContext.getOperation());
819
820                 // Revert the local server config change.
821                 raftContext.updatePeerIds(previousServerConfig);
822
823                 changeToIdleState();
824                 RaftActorServerConfigurationSupport.this.onNewOperation(operationContext);
825             }
826         }
827
828         @Override
829         void onServerOperationTimeout(ServerOperationTimeout timeout) {
830             LOG.warn("{}: Leader election timed out - cannot apply operation {}",
831                     raftContext.getId(), timeout.getLoggingContext());
832
833             // Revert the local server config change.
834             raftContext.updatePeerIds(previousServerConfig);
835             raftActor.initializeBehavior();
836
837             tryToForwardOperationToAnotherServer();
838         }
839
840         private void tryToForwardOperationToAnotherServer() {
841             Collection<String> serversVisited = new HashSet<>(operationContext.getOperation().getServersVisited());
842
843             LOG.debug("{}: tryToForwardOperationToAnotherServer - servers already visited {}", raftContext.getId(),
844                     serversVisited);
845
846             serversVisited.add(raftContext.getId());
847
848             // Try to find another whose state is being changed from non-voting to voting and that we haven't
849             // tried yet.
850             Map<String, Boolean> serverVotingStatusMap = operationContext.getOperation().getServerVotingStatusMap();
851             ActorSelection forwardToPeerActor = null;
852             for (Map.Entry<String, Boolean> e: serverVotingStatusMap.entrySet()) {
853                 Boolean isVoting = e.getValue();
854                 String serverId = e.getKey();
855                 PeerInfo peerInfo = raftContext.getPeerInfo(serverId);
856                 if (isVoting && peerInfo != null && !peerInfo.isVoting() && !serversVisited.contains(serverId)) {
857                     ActorSelection actor = raftContext.getPeerActorSelection(serverId);
858                     if (actor != null) {
859                         forwardToPeerActor = actor;
860                         break;
861                     }
862                 }
863             }
864
865             if (forwardToPeerActor != null) {
866                 LOG.debug("{}: Found server {} to forward to", raftContext.getId(), forwardToPeerActor);
867
868                 forwardToPeerActor.tell(new ChangeServersVotingStatus(serverVotingStatusMap, serversVisited),
869                         operationContext.getClientRequestor());
870                 changeToIdleState();
871             } else {
872                 operationComplete(operationContext, ServerChangeStatus.NO_LEADER);
873             }
874         }
875     }
876
877     static class ServerOperationTimeout {
878         private final String loggingContext;
879
880         ServerOperationTimeout(String loggingContext) {
881             this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
882         }
883
884         String getLoggingContext() {
885             return loggingContext;
886         }
887     }
888 }