Remove prefix shard leftovers
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / shardmanager / ShardManager.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.datastore.shardmanager;
10
11 import static akka.pattern.Patterns.ask;
12 import static java.util.Objects.requireNonNull;
13
14 import akka.actor.ActorRef;
15 import akka.actor.Address;
16 import akka.actor.Cancellable;
17 import akka.actor.OneForOneStrategy;
18 import akka.actor.PoisonPill;
19 import akka.actor.Status;
20 import akka.actor.SupervisorStrategy;
21 import akka.actor.SupervisorStrategy.Directive;
22 import akka.cluster.ClusterEvent;
23 import akka.cluster.ClusterEvent.MemberWeaklyUp;
24 import akka.cluster.Member;
25 import akka.dispatch.Futures;
26 import akka.dispatch.OnComplete;
27 import akka.japi.Function;
28 import akka.pattern.Patterns;
29 import akka.persistence.DeleteSnapshotsFailure;
30 import akka.persistence.DeleteSnapshotsSuccess;
31 import akka.persistence.RecoveryCompleted;
32 import akka.persistence.SaveSnapshotFailure;
33 import akka.persistence.SaveSnapshotSuccess;
34 import akka.persistence.SnapshotOffer;
35 import akka.persistence.SnapshotSelectionCriteria;
36 import akka.util.Timeout;
37 import com.google.common.annotations.VisibleForTesting;
38 import com.google.common.util.concurrent.SettableFuture;
39 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
40 import java.util.ArrayList;
41 import java.util.Collection;
42 import java.util.Collections;
43 import java.util.HashMap;
44 import java.util.HashSet;
45 import java.util.List;
46 import java.util.Map;
47 import java.util.Map.Entry;
48 import java.util.Set;
49 import java.util.concurrent.TimeUnit;
50 import java.util.concurrent.TimeoutException;
51 import java.util.function.Consumer;
52 import java.util.function.Supplier;
53 import org.opendaylight.controller.cluster.access.concepts.MemberName;
54 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
55 import org.opendaylight.controller.cluster.common.actor.Dispatchers;
56 import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
57 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
58 import org.opendaylight.controller.cluster.datastore.DatastoreContextFactory;
59 import org.opendaylight.controller.cluster.datastore.Shard;
60 import org.opendaylight.controller.cluster.datastore.config.Configuration;
61 import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration;
62 import org.opendaylight.controller.cluster.datastore.exceptions.AlreadyExistsException;
63 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
64 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
65 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
66 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
67 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
68 import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica;
69 import org.opendaylight.controller.cluster.datastore.messages.ChangeShardMembersVotingStatus;
70 import org.opendaylight.controller.cluster.datastore.messages.CreateShard;
71 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
72 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
73 import org.opendaylight.controller.cluster.datastore.messages.FlipShardMembersVotingStatus;
74 import org.opendaylight.controller.cluster.datastore.messages.GetShardRole;
75 import org.opendaylight.controller.cluster.datastore.messages.GetShardRoleReply;
76 import org.opendaylight.controller.cluster.datastore.messages.LocalPrimaryShardFound;
77 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
78 import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound;
79 import org.opendaylight.controller.cluster.datastore.messages.RemoteFindPrimary;
80 import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound;
81 import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica;
82 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
83 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
84 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
85 import org.opendaylight.controller.cluster.datastore.persisted.ShardManagerSnapshot;
86 import org.opendaylight.controller.cluster.datastore.utils.CompositeOnComplete;
87 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
88 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
89 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply;
90 import org.opendaylight.controller.cluster.notifications.RoleChangeNotification;
91 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
92 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
93 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
94 import org.opendaylight.controller.cluster.raft.client.messages.GetSnapshot;
95 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
96 import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
97 import org.opendaylight.controller.cluster.raft.messages.AddServer;
98 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
99 import org.opendaylight.controller.cluster.raft.messages.ChangeServersVotingStatus;
100 import org.opendaylight.controller.cluster.raft.messages.RemoveServer;
101 import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
102 import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply;
103 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
104 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
105 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
106 import org.opendaylight.yangtools.concepts.Registration;
107 import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
108 import org.slf4j.Logger;
109 import org.slf4j.LoggerFactory;
110 import scala.concurrent.ExecutionContext;
111 import scala.concurrent.Future;
112 import scala.concurrent.duration.FiniteDuration;
113
114 /**
115  * Manages the shards for a data store. The ShardManager has the following jobs:
116  * <ul>
117  * <li> Create all the local shard replicas that belong on this cluster member
118  * <li> Find the address of the local shard
119  * <li> Find the primary replica for any given shard
120  * <li> Monitor the cluster members and store their addresses
121  * </ul>
122  */
123 class ShardManager extends AbstractUntypedPersistentActorWithMetering {
124     private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class);
125
126     // Stores a mapping between a shard name and it's corresponding information
127     // Shard names look like inventory, topology etc and are as specified in
128     // configuration
129     @VisibleForTesting
130     final Map<String, ShardInformation> localShards = new HashMap<>();
131
132     // The type of a ShardManager reflects the type of the datastore itself
133     // A data store could be of type config/operational
134     private final String type;
135
136     private final ClusterWrapper cluster;
137
138     private final Configuration configuration;
139
140     @VisibleForTesting
141     final String shardDispatcherPath;
142
143     private final ShardManagerInfo shardManagerMBean;
144
145     private DatastoreContextFactory datastoreContextFactory;
146
147     private final SettableFuture<Void> readinessFuture;
148
149     private final PrimaryShardInfoFutureCache primaryShardInfoCache;
150
151     @VisibleForTesting
152     final ShardPeerAddressResolver peerAddressResolver;
153
154     private EffectiveModelContext schemaContext;
155
156     private DatastoreSnapshot restoreFromSnapshot;
157
158     private ShardManagerSnapshot currentSnapshot;
159
160     private final Set<String> shardReplicaOperationsInProgress = new HashSet<>();
161
162     private final Map<String, CompositeOnComplete<Boolean>> shardActorsStopping = new HashMap<>();
163
164     private final Set<Consumer<String>> shardAvailabilityCallbacks = new HashSet<>();
165
166     private final String persistenceId;
167
168     ShardManager(final AbstractShardManagerCreator<?> builder) {
169         this.cluster = builder.getCluster();
170         this.configuration = builder.getConfiguration();
171         this.datastoreContextFactory = builder.getDatastoreContextFactory();
172         this.type = datastoreContextFactory.getBaseDatastoreContext().getDataStoreName();
173         this.shardDispatcherPath =
174                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
175         this.readinessFuture = builder.getReadinessFuture();
176         this.primaryShardInfoCache = builder.getPrimaryShardInfoCache();
177         this.restoreFromSnapshot = builder.getRestoreFromSnapshot();
178
179         String possiblePersistenceId = datastoreContextFactory.getBaseDatastoreContext().getShardManagerPersistenceId();
180         persistenceId = possiblePersistenceId != null ? possiblePersistenceId : "shard-manager-" + type;
181
182         peerAddressResolver = new ShardPeerAddressResolver(type, cluster.getCurrentMemberName());
183
184         // Subscribe this actor to cluster member events
185         cluster.subscribeToMemberEvents(getSelf());
186
187         shardManagerMBean = new ShardManagerInfo(getSelf(), cluster.getCurrentMemberName(),
188                 "shard-manager-" + this.type,
189                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType());
190         shardManagerMBean.registerMBean();
191     }
192
193     @Override
194     public void preStart() {
195         LOG.info("Starting ShardManager {}", persistenceId);
196     }
197
198     @Override
199     public void postStop() {
200         LOG.info("Stopping ShardManager {}", persistenceId());
201
202         shardManagerMBean.unregisterMBean();
203     }
204
205     @Override
206     public void handleCommand(final Object message) throws Exception {
207         if (message  instanceof FindPrimary) {
208             findPrimary((FindPrimary)message);
209         } else if (message instanceof FindLocalShard) {
210             findLocalShard((FindLocalShard) message);
211         } else if (message instanceof UpdateSchemaContext) {
212             updateSchemaContext(message);
213         } else if (message instanceof ActorInitialized) {
214             onActorInitialized(message);
215         } else if (message instanceof ClusterEvent.MemberUp) {
216             memberUp((ClusterEvent.MemberUp) message);
217         } else if (message instanceof ClusterEvent.MemberWeaklyUp) {
218             memberWeaklyUp((ClusterEvent.MemberWeaklyUp) message);
219         } else if (message instanceof ClusterEvent.MemberExited) {
220             memberExited((ClusterEvent.MemberExited) message);
221         } else if (message instanceof ClusterEvent.MemberRemoved) {
222             memberRemoved((ClusterEvent.MemberRemoved) message);
223         } else if (message instanceof ClusterEvent.UnreachableMember) {
224             memberUnreachable((ClusterEvent.UnreachableMember) message);
225         } else if (message instanceof ClusterEvent.ReachableMember) {
226             memberReachable((ClusterEvent.ReachableMember) message);
227         } else if (message instanceof DatastoreContextFactory) {
228             onDatastoreContextFactory((DatastoreContextFactory) message);
229         } else if (message instanceof RoleChangeNotification) {
230             onRoleChangeNotification((RoleChangeNotification) message);
231         } else if (message instanceof FollowerInitialSyncUpStatus) {
232             onFollowerInitialSyncStatus((FollowerInitialSyncUpStatus) message);
233         } else if (message instanceof ShardNotInitializedTimeout) {
234             onShardNotInitializedTimeout((ShardNotInitializedTimeout) message);
235         } else if (message instanceof ShardLeaderStateChanged) {
236             onLeaderStateChanged((ShardLeaderStateChanged) message);
237         } else if (message instanceof SwitchShardBehavior) {
238             onSwitchShardBehavior((SwitchShardBehavior) message);
239         } else if (message instanceof CreateShard) {
240             onCreateShard((CreateShard)message);
241         } else if (message instanceof AddShardReplica) {
242             onAddShardReplica((AddShardReplica) message);
243         } else if (message instanceof ForwardedAddServerReply) {
244             ForwardedAddServerReply msg = (ForwardedAddServerReply)message;
245             onAddServerReply(msg.shardInfo, msg.addServerReply, getSender(), msg.leaderPath,
246                     msg.removeShardOnFailure);
247         } else if (message instanceof ForwardedAddServerFailure) {
248             ForwardedAddServerFailure msg = (ForwardedAddServerFailure)message;
249             onAddServerFailure(msg.shardName, msg.failureMessage, msg.failure, getSender(), msg.removeShardOnFailure);
250         } else if (message instanceof RemoveShardReplica) {
251             onRemoveShardReplica((RemoveShardReplica) message);
252         } else if (message instanceof WrappedShardResponse) {
253             onWrappedShardResponse((WrappedShardResponse) message);
254         } else if (message instanceof GetSnapshot) {
255             onGetSnapshot((GetSnapshot) message);
256         } else if (message instanceof ServerRemoved) {
257             onShardReplicaRemoved((ServerRemoved) message);
258         } else if (message instanceof ChangeShardMembersVotingStatus) {
259             onChangeShardServersVotingStatus((ChangeShardMembersVotingStatus) message);
260         } else if (message instanceof FlipShardMembersVotingStatus) {
261             onFlipShardMembersVotingStatus((FlipShardMembersVotingStatus) message);
262         } else if (message instanceof SaveSnapshotSuccess) {
263             onSaveSnapshotSuccess((SaveSnapshotSuccess) message);
264         } else if (message instanceof SaveSnapshotFailure) {
265             LOG.error("{}: SaveSnapshotFailure received for saving snapshot of shards", persistenceId(),
266                     ((SaveSnapshotFailure) message).cause());
267         } else if (message instanceof Shutdown) {
268             onShutDown();
269         } else if (message instanceof GetLocalShardIds) {
270             onGetLocalShardIds();
271         } else if (message instanceof GetShardRole) {
272             onGetShardRole((GetShardRole) message);
273         } else if (message instanceof RunnableMessage) {
274             ((RunnableMessage)message).run();
275         } else if (message instanceof RegisterForShardAvailabilityChanges) {
276             onRegisterForShardAvailabilityChanges((RegisterForShardAvailabilityChanges)message);
277         } else if (message instanceof DeleteSnapshotsFailure) {
278             LOG.warn("{}: Failed to delete prior snapshots", persistenceId(),
279                     ((DeleteSnapshotsFailure) message).cause());
280         } else if (message instanceof DeleteSnapshotsSuccess) {
281             LOG.debug("{}: Successfully deleted prior snapshots", persistenceId());
282         } else if (message instanceof RegisterRoleChangeListenerReply) {
283             LOG.trace("{}: Received RegisterRoleChangeListenerReply", persistenceId());
284         } else if (message instanceof ClusterEvent.MemberEvent) {
285             LOG.trace("{}: Received other ClusterEvent.MemberEvent: {}", persistenceId(), message);
286         } else {
287             unknownMessage(message);
288         }
289     }
290
291     private void onRegisterForShardAvailabilityChanges(final RegisterForShardAvailabilityChanges message) {
292         LOG.debug("{}: onRegisterForShardAvailabilityChanges: {}", persistenceId(), message);
293
294         final Consumer<String> callback = message.getCallback();
295         shardAvailabilityCallbacks.add(callback);
296
297         getSender().tell(new Status.Success((Registration)
298             () -> executeInSelf(() -> shardAvailabilityCallbacks.remove(callback))), self());
299     }
300
301     private void onGetShardRole(final GetShardRole message) {
302         LOG.debug("{}: onGetShardRole for shard: {}", persistenceId(), message.getName());
303
304         final String name = message.getName();
305
306         final ShardInformation shardInformation = localShards.get(name);
307
308         if (shardInformation == null) {
309             LOG.info("{}: no shard information for {} found", persistenceId(), name);
310             getSender().tell(new Status.Failure(
311                     new IllegalArgumentException("Shard with name " + name + " not present.")), ActorRef.noSender());
312             return;
313         }
314
315         getSender().tell(new GetShardRoleReply(shardInformation.getRole()), ActorRef.noSender());
316     }
317
318     void onShutDown() {
319         List<Future<Boolean>> stopFutures = new ArrayList<>(localShards.size());
320         for (ShardInformation info : localShards.values()) {
321             if (info.getActor() != null) {
322                 LOG.debug("{}: Issuing gracefulStop to shard {}", persistenceId(), info.getShardId());
323
324                 FiniteDuration duration = info.getDatastoreContext().getShardRaftConfig()
325                         .getElectionTimeOutInterval().$times(2);
326                 stopFutures.add(Patterns.gracefulStop(info.getActor(), duration, Shutdown.INSTANCE));
327             }
328         }
329
330         LOG.info("Shutting down ShardManager {} - waiting on {} shards", persistenceId(), stopFutures.size());
331
332         ExecutionContext dispatcher = new Dispatchers(context().system().dispatchers())
333                 .getDispatcher(Dispatchers.DispatcherType.Client);
334         Future<Iterable<Boolean>> combinedFutures = Futures.sequence(stopFutures, dispatcher);
335
336         combinedFutures.onComplete(new OnComplete<Iterable<Boolean>>() {
337             @Override
338             public void onComplete(final Throwable failure, final Iterable<Boolean> results) {
339                 LOG.debug("{}: All shards shutdown - sending PoisonPill to self", persistenceId());
340
341                 self().tell(PoisonPill.getInstance(), self());
342
343                 if (failure != null) {
344                     LOG.warn("{}: An error occurred attempting to shut down the shards", persistenceId(), failure);
345                 } else {
346                     int nfailed = 0;
347                     for (Boolean result : results) {
348                         if (!result) {
349                             nfailed++;
350                         }
351                     }
352
353                     if (nfailed > 0) {
354                         LOG.warn("{}: {} shards did not shut down gracefully", persistenceId(), nfailed);
355                     }
356                 }
357             }
358         }, dispatcher);
359     }
360
361     private void onWrappedShardResponse(final WrappedShardResponse message) {
362         if (message.getResponse() instanceof RemoveServerReply) {
363             onRemoveServerReply(getSender(), message.getShardId(), (RemoveServerReply) message.getResponse(),
364                     message.getLeaderPath());
365         }
366     }
367
368     private void onRemoveServerReply(final ActorRef originalSender, final ShardIdentifier shardId,
369             final RemoveServerReply replyMsg, final String leaderPath) {
370         shardReplicaOperationsInProgress.remove(shardId.getShardName());
371
372         LOG.debug("{}: Received {} for shard {}", persistenceId(), replyMsg, shardId.getShardName());
373
374         if (replyMsg.getStatus() == ServerChangeStatus.OK) {
375             LOG.debug("{}: Leader shard successfully removed the replica shard {}", persistenceId(),
376                     shardId.getShardName());
377             originalSender.tell(new Status.Success(null), getSelf());
378         } else {
379             LOG.warn("{}: Leader failed to remove shard replica {} with status {}",
380                     persistenceId(), shardId, replyMsg.getStatus());
381
382             Exception failure = getServerChangeException(RemoveServer.class, replyMsg.getStatus(), leaderPath, shardId);
383             originalSender.tell(new Status.Failure(failure), getSelf());
384         }
385     }
386
387     @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
388             justification = "https://github.com/spotbugs/spotbugs/issues/811")
389     private void removeShardReplica(final RemoveShardReplica contextMessage, final String shardName,
390             final String primaryPath, final ActorRef sender) {
391         if (isShardReplicaOperationInProgress(shardName, sender)) {
392             return;
393         }
394
395         shardReplicaOperationsInProgress.add(shardName);
396
397         final ShardIdentifier shardId = getShardIdentifier(contextMessage.getMemberName(), shardName);
398
399         final DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName).build();
400
401         //inform ShardLeader to remove this shard as a replica by sending an RemoveServer message
402         LOG.debug("{}: Sending RemoveServer message to peer {} for shard {}", persistenceId(),
403                 primaryPath, shardId);
404
405         Timeout removeServerTimeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration());
406         Future<Object> futureObj = ask(getContext().actorSelection(primaryPath),
407                 new RemoveServer(shardId.toString()), removeServerTimeout);
408
409         futureObj.onComplete(new OnComplete<>() {
410             @Override
411             public void onComplete(final Throwable failure, final Object response) {
412                 if (failure != null) {
413                     shardReplicaOperationsInProgress.remove(shardName);
414                     LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath,
415                         shardName, failure);
416
417                     // FAILURE
418                     sender.tell(new Status.Failure(new RuntimeException(
419                         String.format("RemoveServer request to leader %s for shard %s failed", primaryPath, shardName),
420                         failure)), self());
421                 } else {
422                     // SUCCESS
423                     self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender);
424                 }
425             }
426         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
427     }
428
429     private void onShardReplicaRemoved(final ServerRemoved message) {
430         removeShard(new ShardIdentifier.Builder().fromShardIdString(message.getServerId()).build());
431     }
432
433     @SuppressWarnings("checkstyle:IllegalCatch")
434     private void removeShard(final ShardIdentifier shardId) {
435         final String shardName = shardId.getShardName();
436         final ShardInformation shardInformation = localShards.remove(shardName);
437         if (shardInformation == null) {
438             LOG.debug("{} : Shard replica {} is not present in list", persistenceId(), shardId.toString());
439             return;
440         }
441
442         final ActorRef shardActor = shardInformation.getActor();
443         if (shardActor != null) {
444             long timeoutInMS = Math.max(shardInformation.getDatastoreContext().getShardRaftConfig()
445                     .getElectionTimeOutInterval().$times(3).toMillis(), 10000);
446
447             LOG.debug("{} : Sending Shutdown to Shard actor {} with {} ms timeout", persistenceId(), shardActor,
448                     timeoutInMS);
449
450             final Future<Boolean> stopFuture = Patterns.gracefulStop(shardActor,
451                     FiniteDuration.apply(timeoutInMS, TimeUnit.MILLISECONDS), Shutdown.INSTANCE);
452
453             final CompositeOnComplete<Boolean> onComplete = new CompositeOnComplete<>() {
454                 @Override
455                 public void onComplete(final Throwable failure, final Boolean result) {
456                     if (failure == null) {
457                         LOG.debug("{} : Successfully shut down Shard actor {}", persistenceId(), shardActor);
458                     } else {
459                         LOG.warn("{}: Failed to shut down Shard actor {}", persistenceId(), shardActor, failure);
460                     }
461
462                     self().tell((RunnableMessage) () -> {
463                         // At any rate, invalidate primaryShardInfo cache
464                         primaryShardInfoCache.remove(shardName);
465
466                         shardActorsStopping.remove(shardName);
467                         notifyOnCompleteTasks(failure, result);
468                     }, ActorRef.noSender());
469                 }
470             };
471
472             shardActorsStopping.put(shardName, onComplete);
473             stopFuture.onComplete(onComplete, new Dispatchers(context().system().dispatchers())
474                     .getDispatcher(Dispatchers.DispatcherType.Client));
475         }
476
477         LOG.debug("{} : Local Shard replica for shard {} has been removed", persistenceId(), shardName);
478         persistShardList();
479     }
480
481     private void onGetSnapshot(final GetSnapshot getSnapshot) {
482         LOG.debug("{}: onGetSnapshot", persistenceId());
483
484         List<String> notInitialized = null;
485         for (ShardInformation shardInfo : localShards.values()) {
486             if (!shardInfo.isShardInitialized()) {
487                 if (notInitialized == null) {
488                     notInitialized = new ArrayList<>();
489                 }
490
491                 notInitialized.add(shardInfo.getShardName());
492             }
493         }
494
495         if (notInitialized != null) {
496             getSender().tell(new Status.Failure(new IllegalStateException(String.format(
497                     "%d shard(s) %s are not initialized", notInitialized.size(), notInitialized))), getSelf());
498             return;
499         }
500
501         ActorRef replyActor = getContext().actorOf(ShardManagerGetSnapshotReplyActor.props(
502                 new ArrayList<>(localShards.keySet()), type, currentSnapshot , getSender(), persistenceId(),
503                 datastoreContextFactory.getBaseDatastoreContext().getShardInitializationTimeout().duration()));
504
505         for (ShardInformation shardInfo: localShards.values()) {
506             shardInfo.getActor().tell(getSnapshot, replyActor);
507         }
508     }
509
510     @SuppressWarnings("checkstyle:IllegalCatch")
511     private void onCreateShard(final CreateShard createShard) {
512         LOG.debug("{}: onCreateShard: {}", persistenceId(), createShard);
513
514         Object reply;
515         try {
516             String shardName = createShard.getModuleShardConfig().getShardName();
517             if (localShards.containsKey(shardName)) {
518                 LOG.debug("{}: Shard {} already exists", persistenceId(), shardName);
519                 reply = new Status.Success(String.format("Shard with name %s already exists", shardName));
520             } else {
521                 doCreateShard(createShard);
522                 reply = new Status.Success(null);
523             }
524         } catch (Exception e) {
525             LOG.error("{}: onCreateShard failed", persistenceId(), e);
526             reply = new Status.Failure(e);
527         }
528
529         if (getSender() != null && !getContext().system().deadLetters().equals(getSender())) {
530             getSender().tell(reply, getSelf());
531         }
532     }
533
534     @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
535         justification = "https://github.com/spotbugs/spotbugs/issues/811")
536     private boolean isPreviousShardActorStopInProgress(final String shardName, final Object messageToDefer) {
537         final CompositeOnComplete<Boolean> stopOnComplete = shardActorsStopping.get(shardName);
538         if (stopOnComplete == null) {
539             return false;
540         }
541
542         LOG.debug("{} : Stop is in progress for shard {} - adding OnComplete callback to defer {}", persistenceId(),
543                 shardName, messageToDefer);
544         final ActorRef sender = getSender();
545         stopOnComplete.addOnComplete(new OnComplete<Boolean>() {
546             @Override
547             public void onComplete(final Throwable failure, final Boolean result) {
548                 LOG.debug("{} : Stop complete for shard {} - re-queing {}", persistenceId(), shardName, messageToDefer);
549                 self().tell(messageToDefer, sender);
550             }
551         });
552
553         return true;
554     }
555
556     private void doCreateShard(final CreateShard createShard) {
557         final ModuleShardConfiguration moduleShardConfig = createShard.getModuleShardConfig();
558         final String shardName = moduleShardConfig.getShardName();
559
560         configuration.addModuleShardConfiguration(moduleShardConfig);
561
562         DatastoreContext shardDatastoreContext = createShard.getDatastoreContext();
563         if (shardDatastoreContext == null) {
564             shardDatastoreContext = newShardDatastoreContext(shardName);
565         } else {
566             shardDatastoreContext = DatastoreContext.newBuilderFrom(shardDatastoreContext).shardPeerAddressResolver(
567                     peerAddressResolver).build();
568         }
569
570         ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
571
572         boolean shardWasInRecoveredSnapshot = currentSnapshot != null
573                 && currentSnapshot.getShardList().contains(shardName);
574
575         Map<String, String> peerAddresses;
576         boolean isActiveMember;
577         if (shardWasInRecoveredSnapshot || configuration.getMembersFromShardName(shardName)
578                 .contains(cluster.getCurrentMemberName())) {
579             peerAddresses = getPeerAddresses(shardName);
580             isActiveMember = true;
581         } else {
582             // The local member is not in the static shard member configuration and the shard did not
583             // previously exist (ie !shardWasInRecoveredSnapshot). In this case we'll create
584             // the shard with no peers and with elections disabled so it stays as follower. A
585             // subsequent AddServer request will be needed to make it an active member.
586             isActiveMember = false;
587             peerAddresses = Collections.emptyMap();
588             shardDatastoreContext = DatastoreContext.newBuilderFrom(shardDatastoreContext)
589                     .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).build();
590         }
591
592         LOG.debug("{} doCreateShard: shardId: {}, memberNames: {}, peerAddresses: {}, isActiveMember: {}",
593                 persistenceId(), shardId, moduleShardConfig.getShardMemberNames(), peerAddresses,
594                 isActiveMember);
595
596         ShardInformation info = new ShardInformation(shardName, shardId, peerAddresses,
597                 shardDatastoreContext, createShard.getShardBuilder(), peerAddressResolver);
598         info.setActiveMember(isActiveMember);
599         localShards.put(info.getShardName(), info);
600
601         if (schemaContext != null) {
602             info.setSchemaContext(schemaContext);
603             info.setActor(newShardActor(info));
604         }
605     }
606
607     private DatastoreContext.Builder newShardDatastoreContextBuilder(final String shardName) {
608         return DatastoreContext.newBuilderFrom(datastoreContextFactory.getShardDatastoreContext(shardName))
609                 .shardPeerAddressResolver(peerAddressResolver);
610     }
611
612     private DatastoreContext newShardDatastoreContext(final String shardName) {
613         return newShardDatastoreContextBuilder(shardName).build();
614     }
615
616     private void checkReady() {
617         if (isReadyWithLeaderId()) {
618             LOG.info("{}: All Shards are ready - data store {} is ready", persistenceId(), type);
619             readinessFuture.set(null);
620         }
621     }
622
623     private void onLeaderStateChanged(final ShardLeaderStateChanged leaderStateChanged) {
624         LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged);
625
626         ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId());
627         if (shardInformation != null) {
628             shardInformation.setLocalDataTree(leaderStateChanged.getLocalShardDataTree());
629             shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion());
630             if (shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) {
631                 primaryShardInfoCache.remove(shardInformation.getShardName());
632
633                 notifyShardAvailabilityCallbacks(shardInformation);
634             }
635
636             checkReady();
637         } else {
638             LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
639         }
640     }
641
642     private void notifyShardAvailabilityCallbacks(final ShardInformation shardInformation) {
643         shardAvailabilityCallbacks.forEach(callback -> callback.accept(shardInformation.getShardName()));
644     }
645
646     private void onShardNotInitializedTimeout(final ShardNotInitializedTimeout message) {
647         ShardInformation shardInfo = message.getShardInfo();
648
649         LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
650                 shardInfo.getShardName());
651
652         shardInfo.removeOnShardInitialized(message.getOnShardInitialized());
653
654         if (!shardInfo.isShardInitialized()) {
655             LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(), shardInfo.getShardName());
656             message.getSender().tell(createNotInitializedException(shardInfo.getShardId()), getSelf());
657         } else {
658             LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
659             message.getSender().tell(new NoShardLeaderException(shardInfo.getShardId()), getSelf());
660         }
661     }
662
663     private void onFollowerInitialSyncStatus(final FollowerInitialSyncUpStatus status) {
664         LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(),
665                 status.getName(), status.isInitialSyncDone());
666
667         ShardInformation shardInformation = findShardInformation(status.getName());
668
669         if (shardInformation != null) {
670             shardInformation.setFollowerSyncStatus(status.isInitialSyncDone());
671
672             shardManagerMBean.setSyncStatus(isInSync());
673         }
674
675     }
676
677     private void onRoleChangeNotification(final RoleChangeNotification roleChanged) {
678         LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(),
679                 roleChanged.getOldRole(), roleChanged.getNewRole());
680
681         ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
682         if (shardInformation != null) {
683             shardInformation.setRole(roleChanged.getNewRole());
684             checkReady();
685             shardManagerMBean.setSyncStatus(isInSync());
686         }
687     }
688
689
690     private ShardInformation findShardInformation(final String memberId) {
691         for (ShardInformation info : localShards.values()) {
692             if (info.getShardId().toString().equals(memberId)) {
693                 return info;
694             }
695         }
696
697         return null;
698     }
699
700     private boolean isReadyWithLeaderId() {
701         boolean isReady = true;
702         for (ShardInformation info : localShards.values()) {
703             if (!info.isShardReadyWithLeaderId()) {
704                 isReady = false;
705                 break;
706             }
707         }
708         return isReady;
709     }
710
711     private boolean isInSync() {
712         for (ShardInformation info : localShards.values()) {
713             if (!info.isInSync()) {
714                 return false;
715             }
716         }
717         return true;
718     }
719
720     private void onActorInitialized(final Object message) {
721         final ActorRef sender = getSender();
722
723         if (sender == null) {
724             // why is a non-actor sending this message? Just ignore.
725             return;
726         }
727
728         String actorName = sender.path().name();
729         //find shard name from actor name; actor name is stringified shardId
730
731         final ShardIdentifier shardId;
732         try {
733             shardId = ShardIdentifier.fromShardIdString(actorName);
734         } catch (IllegalArgumentException e) {
735             LOG.debug("{}: ignoring actor {}", persistenceId, actorName, e);
736             return;
737         }
738
739         markShardAsInitialized(shardId.getShardName());
740     }
741
742     private void markShardAsInitialized(final String shardName) {
743         LOG.debug("{}: Initializing shard [{}]", persistenceId(), shardName);
744
745         ShardInformation shardInformation = localShards.get(shardName);
746         if (shardInformation != null) {
747             shardInformation.setActorInitialized();
748
749             shardInformation.getActor().tell(new RegisterRoleChangeListener(), self());
750         }
751     }
752
753     @Override
754     protected void handleRecover(final Object message) throws Exception {
755         if (message instanceof RecoveryCompleted) {
756             onRecoveryCompleted();
757         } else if (message instanceof SnapshotOffer) {
758             applyShardManagerSnapshot((ShardManagerSnapshot)((SnapshotOffer) message).snapshot());
759         }
760     }
761
762     @SuppressWarnings("checkstyle:IllegalCatch")
763     private void onRecoveryCompleted() {
764         LOG.info("Recovery complete : {}", persistenceId());
765
766         if (currentSnapshot == null && restoreFromSnapshot != null
767                 && restoreFromSnapshot.getShardManagerSnapshot() != null) {
768             ShardManagerSnapshot snapshot = restoreFromSnapshot.getShardManagerSnapshot();
769
770             LOG.debug("{}: Restoring from ShardManagerSnapshot: {}", persistenceId(), snapshot);
771
772             applyShardManagerSnapshot(snapshot);
773         }
774
775         createLocalShards();
776     }
777
778     private void sendResponse(final ShardInformation shardInformation, final boolean doWait,
779             final boolean wantShardReady, final Supplier<Object> messageSupplier) {
780         if (!shardInformation.isShardInitialized() || wantShardReady && !shardInformation.isShardReadyWithLeaderId()) {
781             if (doWait) {
782                 final ActorRef sender = getSender();
783                 final ActorRef self = self();
784
785                 Runnable replyRunnable = () -> sender.tell(messageSupplier.get(), self);
786
787                 OnShardInitialized onShardInitialized = wantShardReady ? new OnShardReady(replyRunnable) :
788                     new OnShardInitialized(replyRunnable);
789
790                 shardInformation.addOnShardInitialized(onShardInitialized);
791
792                 FiniteDuration timeout = shardInformation.getDatastoreContext()
793                         .getShardInitializationTimeout().duration();
794                 if (shardInformation.isShardInitialized()) {
795                     // If the shard is already initialized then we'll wait enough time for the shard to
796                     // elect a leader, ie 2 times the election timeout.
797                     timeout = FiniteDuration.create(shardInformation.getDatastoreContext().getShardRaftConfig()
798                             .getElectionTimeOutInterval().toMillis() * 2, TimeUnit.MILLISECONDS);
799                 }
800
801                 LOG.debug("{}: Scheduling {} ms timer to wait for shard {}", persistenceId(), timeout.toMillis(),
802                         shardInformation);
803
804                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
805                         timeout, getSelf(),
806                         new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
807                         getContext().dispatcher(), getSelf());
808
809                 onShardInitialized.setTimeoutSchedule(timeoutSchedule);
810
811             } else if (!shardInformation.isShardInitialized()) {
812                 LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(),
813                         shardInformation.getShardName());
814                 getSender().tell(createNotInitializedException(shardInformation.getShardId()), getSelf());
815             } else {
816                 LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
817                         shardInformation.getShardName());
818                 getSender().tell(new NoShardLeaderException(shardInformation.getShardId()), getSelf());
819             }
820
821             return;
822         }
823
824         getSender().tell(messageSupplier.get(), getSelf());
825     }
826
827     private static NotInitializedException createNotInitializedException(final ShardIdentifier shardId) {
828         return new NotInitializedException(String.format(
829                 "Found primary shard %s but it's not initialized yet. Please try again later", shardId));
830     }
831
832     @VisibleForTesting
833     static MemberName memberToName(final Member member) {
834         return MemberName.forName(member.roles().iterator().next());
835     }
836
837     private void memberRemoved(final ClusterEvent.MemberRemoved message) {
838         MemberName memberName = memberToName(message.member());
839
840         LOG.info("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName,
841                 message.member().address());
842
843         peerAddressResolver.removePeerAddress(memberName);
844
845         for (ShardInformation info : localShards.values()) {
846             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
847         }
848     }
849
850     private void memberExited(final ClusterEvent.MemberExited message) {
851         MemberName memberName = memberToName(message.member());
852
853         LOG.info("{}: Received MemberExited: memberName: {}, address: {}", persistenceId(), memberName,
854                 message.member().address());
855
856         peerAddressResolver.removePeerAddress(memberName);
857
858         for (ShardInformation info : localShards.values()) {
859             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
860         }
861     }
862
863     private void memberUp(final ClusterEvent.MemberUp message) {
864         MemberName memberName = memberToName(message.member());
865
866         LOG.info("{}: Received MemberUp: memberName: {}, address: {}", persistenceId(), memberName,
867                 message.member().address());
868
869         memberUp(memberName, message.member().address());
870     }
871
872     private void memberUp(final MemberName memberName, final Address address) {
873         addPeerAddress(memberName, address);
874         checkReady();
875     }
876
877     private void memberWeaklyUp(final MemberWeaklyUp message) {
878         MemberName memberName = memberToName(message.member());
879
880         LOG.info("{}: Received MemberWeaklyUp: memberName: {}, address: {}", persistenceId(), memberName,
881                 message.member().address());
882
883         memberUp(memberName, message.member().address());
884     }
885
886     private void addPeerAddress(final MemberName memberName, final Address address) {
887         peerAddressResolver.addPeerAddress(memberName, address);
888
889         for (ShardInformation info : localShards.values()) {
890             String shardName = info.getShardName();
891             String peerId = getShardIdentifier(memberName, shardName).toString();
892             info.updatePeerAddress(peerId, peerAddressResolver.getShardActorAddress(shardName, memberName), getSelf());
893
894             info.peerUp(memberName, peerId, getSelf());
895         }
896     }
897
898     private void memberReachable(final ClusterEvent.ReachableMember message) {
899         MemberName memberName = memberToName(message.member());
900         LOG.info("Received ReachableMember: memberName {}, address: {}", memberName, message.member().address());
901
902         addPeerAddress(memberName, message.member().address());
903
904         markMemberAvailable(memberName);
905     }
906
907     private void memberUnreachable(final ClusterEvent.UnreachableMember message) {
908         MemberName memberName = memberToName(message.member());
909         LOG.info("Received UnreachableMember: memberName {}, address: {}", memberName, message.member().address());
910
911         markMemberUnavailable(memberName);
912     }
913
914     private void markMemberUnavailable(final MemberName memberName) {
915         for (ShardInformation info : localShards.values()) {
916             String leaderId = info.getLeaderId();
917             if (leaderId != null && ShardIdentifier.fromShardIdString(leaderId).getMemberName().equals(memberName)) {
918                 LOG.debug("Marking Leader {} as unavailable.", leaderId);
919                 info.setLeaderAvailable(false);
920
921                 primaryShardInfoCache.remove(info.getShardName());
922
923                 notifyShardAvailabilityCallbacks(info);
924             }
925
926             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
927         }
928     }
929
930     private void markMemberAvailable(final MemberName memberName) {
931         for (ShardInformation info : localShards.values()) {
932             String leaderId = info.getLeaderId();
933             if (leaderId != null && ShardIdentifier.fromShardIdString(leaderId).getMemberName().equals(memberName)) {
934                 LOG.debug("Marking Leader {} as available.", leaderId);
935                 info.setLeaderAvailable(true);
936             }
937
938             info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
939         }
940     }
941
942     private void onDatastoreContextFactory(final DatastoreContextFactory factory) {
943         datastoreContextFactory = factory;
944         for (ShardInformation info : localShards.values()) {
945             info.setDatastoreContext(newShardDatastoreContext(info.getShardName()), getSelf());
946         }
947     }
948
949     private void onGetLocalShardIds() {
950         final List<String> response = new ArrayList<>(localShards.size());
951
952         for (ShardInformation info : localShards.values()) {
953             response.add(info.getShardId().toString());
954         }
955
956         getSender().tell(new Status.Success(response), getSelf());
957     }
958
959     private void onSwitchShardBehavior(final SwitchShardBehavior message) {
960         final ShardIdentifier identifier = message.getShardId();
961
962         if (identifier != null) {
963             final ShardInformation info = localShards.get(identifier.getShardName());
964             if (info == null) {
965                 getSender().tell(new Status.Failure(
966                     new IllegalArgumentException("Shard " + identifier + " is not local")), getSelf());
967                 return;
968             }
969
970             switchShardBehavior(info, new SwitchBehavior(message.getNewState(), message.getTerm()));
971         } else {
972             for (ShardInformation info : localShards.values()) {
973                 switchShardBehavior(info, new SwitchBehavior(message.getNewState(), message.getTerm()));
974             }
975         }
976
977         getSender().tell(new Status.Success(null), getSelf());
978     }
979
980     private void switchShardBehavior(final ShardInformation info, final SwitchBehavior switchBehavior) {
981         final ActorRef actor = info.getActor();
982         if (actor != null) {
983             actor.tell(switchBehavior, getSelf());
984         } else {
985             LOG.warn("Could not switch the behavior of shard {} to {} - shard is not yet available",
986                 info.getShardName(), switchBehavior.getNewState());
987         }
988     }
989
990     /**
991      * Notifies all the local shards of a change in the schema context.
992      *
993      * @param message the message to send
994      */
995     private void updateSchemaContext(final Object message) {
996         schemaContext = ((UpdateSchemaContext) message).getEffectiveModelContext();
997
998         LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getModules().size());
999
1000         for (ShardInformation info : localShards.values()) {
1001             info.setSchemaContext(schemaContext);
1002
1003             if (info.getActor() == null) {
1004                 LOG.debug("Creating Shard {}", info.getShardId());
1005                 info.setActor(newShardActor(info));
1006                 // Update peer address for every existing peer memeber to avoid missing sending
1007                 // PeerAddressResolved and PeerUp to this shard while UpdateSchemaContext comes after MemberUp.
1008                 String shardName = info.getShardName();
1009                 for (MemberName memberName : peerAddressResolver.getPeerMembers()) {
1010                     String peerId = getShardIdentifier(memberName, shardName).toString() ;
1011                     String peerAddress = peerAddressResolver.getShardActorAddress(shardName, memberName);
1012                     info.updatePeerAddress(peerId, peerAddress, getSelf());
1013                     info.peerUp(memberName, peerId, getSelf());
1014                     LOG.debug("{}: updated peer {} on member {} with address {} on shard {} whose actor address is {}",
1015                             persistenceId(), peerId, memberName, peerAddress, info.getShardId(), info.getActor());
1016                 }
1017             } else {
1018                 info.getActor().tell(message, getSelf());
1019             }
1020         }
1021     }
1022
1023     @VisibleForTesting
1024     protected ClusterWrapper getCluster() {
1025         return cluster;
1026     }
1027
1028     @VisibleForTesting
1029     protected ActorRef newShardActor(final ShardInformation info) {
1030         return getContext().actorOf(info.newProps().withDispatcher(shardDispatcherPath),
1031                 info.getShardId().toString());
1032     }
1033
1034     private void findPrimary(final FindPrimary message) {
1035         LOG.debug("{}: In findPrimary: {}", persistenceId(), message);
1036
1037         final String shardName = message.getShardName();
1038         final boolean canReturnLocalShardState = !(message instanceof RemoteFindPrimary);
1039
1040         // First see if the there is a local replica for the shard
1041         final ShardInformation info = localShards.get(shardName);
1042         if (info != null && info.isActiveMember()) {
1043             sendResponse(info, message.isWaitUntilReady(), true, () -> {
1044                 String primaryPath = info.getSerializedLeaderActor();
1045                 Object found = canReturnLocalShardState && info.isLeader()
1046                         ? new LocalPrimaryShardFound(primaryPath, info.getLocalShardDataTree().get()) :
1047                             new RemotePrimaryShardFound(primaryPath, info.getLeaderVersion());
1048
1049                 LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
1050                 return found;
1051             });
1052
1053             return;
1054         }
1055
1056         final Collection<String> visitedAddresses;
1057         if (message instanceof RemoteFindPrimary) {
1058             visitedAddresses = ((RemoteFindPrimary)message).getVisitedAddresses();
1059         } else {
1060             visitedAddresses = new ArrayList<>(1);
1061         }
1062
1063         visitedAddresses.add(peerAddressResolver.getShardManagerActorPathBuilder(cluster.getSelfAddress()).toString());
1064
1065         for (String address: peerAddressResolver.getShardManagerPeerActorAddresses()) {
1066             if (visitedAddresses.contains(address)) {
1067                 continue;
1068             }
1069
1070             LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}, visitedAddresses: {}",
1071                     persistenceId(), shardName, address, visitedAddresses);
1072
1073             getContext().actorSelection(address).forward(new RemoteFindPrimary(shardName,
1074                     message.isWaitUntilReady(), visitedAddresses), getContext());
1075             return;
1076         }
1077
1078         LOG.debug("{}: No shard found for {}", persistenceId(), shardName);
1079
1080         getSender().tell(new PrimaryNotFoundException(
1081                 String.format("No primary shard found for %s.", shardName)), getSelf());
1082     }
1083
1084     private void findPrimary(final String shardName, final FindPrimaryResponseHandler handler) {
1085         Timeout findPrimaryTimeout = new Timeout(datastoreContextFactory.getBaseDatastoreContext()
1086                 .getShardInitializationTimeout().duration().$times(2));
1087
1088         Future<Object> futureObj = ask(getSelf(), new FindPrimary(shardName, true), findPrimaryTimeout);
1089         futureObj.onComplete(new OnComplete<>() {
1090             @Override
1091             public void onComplete(final Throwable failure, final Object response) {
1092                 if (failure != null) {
1093                     handler.onFailure(failure);
1094                 } else {
1095                     if (response instanceof RemotePrimaryShardFound) {
1096                         handler.onRemotePrimaryShardFound((RemotePrimaryShardFound) response);
1097                     } else if (response instanceof LocalPrimaryShardFound) {
1098                         handler.onLocalPrimaryFound((LocalPrimaryShardFound) response);
1099                     } else {
1100                         handler.onUnknownResponse(response);
1101                     }
1102                 }
1103             }
1104         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
1105     }
1106
1107     /**
1108      * Construct the name of the shard actor given the name of the member on
1109      * which the shard resides and the name of the shard.
1110      *
1111      * @param memberName the member name
1112      * @param shardName the shard name
1113      * @return a b
1114      */
1115     private ShardIdentifier getShardIdentifier(final MemberName memberName, final String shardName) {
1116         return peerAddressResolver.getShardIdentifier(memberName, shardName);
1117     }
1118
1119     /**
1120      * Create shards that are local to the member on which the ShardManager runs.
1121      */
1122     private void createLocalShards() {
1123         MemberName memberName = this.cluster.getCurrentMemberName();
1124         Collection<String> memberShardNames = this.configuration.getMemberShardNames(memberName);
1125
1126         Map<String, DatastoreSnapshot.ShardSnapshot> shardSnapshots = new HashMap<>();
1127         if (restoreFromSnapshot != null) {
1128             for (DatastoreSnapshot.ShardSnapshot snapshot: restoreFromSnapshot.getShardSnapshots()) {
1129                 shardSnapshots.put(snapshot.getName(), snapshot);
1130             }
1131         }
1132
1133         // null out to GC
1134         restoreFromSnapshot = null;
1135
1136         for (String shardName : memberShardNames) {
1137             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
1138
1139             LOG.debug("{}: Creating local shard: {}", persistenceId(), shardId);
1140
1141             Map<String, String> peerAddresses = getPeerAddresses(shardName);
1142             localShards.put(shardName, createShardInfoFor(shardName, shardId, peerAddresses,
1143                     newShardDatastoreContext(shardName), shardSnapshots));
1144         }
1145     }
1146
1147     @VisibleForTesting
1148     ShardInformation createShardInfoFor(final String shardName, final ShardIdentifier shardId,
1149                                         final Map<String, String> peerAddresses,
1150                                         final DatastoreContext datastoreContext,
1151                                         final Map<String, DatastoreSnapshot.ShardSnapshot> shardSnapshots) {
1152         return new ShardInformation(shardName, shardId, peerAddresses,
1153                 datastoreContext, Shard.builder().restoreFromSnapshot(shardSnapshots.get(shardName)),
1154                 peerAddressResolver);
1155     }
1156
1157     /**
1158      * Given the name of the shard find the addresses of all it's peers.
1159      *
1160      * @param shardName the shard name
1161      */
1162     Map<String, String> getPeerAddresses(final String shardName) {
1163         final Collection<MemberName> members = configuration.getMembersFromShardName(shardName);
1164         return getPeerAddresses(shardName, members);
1165     }
1166
1167     private Map<String, String> getPeerAddresses(final String shardName, final Collection<MemberName> members) {
1168         Map<String, String> peerAddresses = new HashMap<>();
1169         MemberName currentMemberName = this.cluster.getCurrentMemberName();
1170
1171         for (MemberName memberName : members) {
1172             if (!currentMemberName.equals(memberName)) {
1173                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
1174                 String address = peerAddressResolver.getShardActorAddress(shardName, memberName);
1175                 peerAddresses.put(shardId.toString(), address);
1176             }
1177         }
1178         return peerAddresses;
1179     }
1180
1181     @Override
1182     public SupervisorStrategy supervisorStrategy() {
1183
1184         return new OneForOneStrategy(10, FiniteDuration.create(1, TimeUnit.MINUTES),
1185                 (Function<Throwable, Directive>) t -> {
1186                     LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
1187                     return SupervisorStrategy.resume();
1188                 });
1189     }
1190
1191     @Override
1192     public String persistenceId() {
1193         return persistenceId;
1194     }
1195
1196     @VisibleForTesting
1197     ShardManagerInfoMBean getMBean() {
1198         return shardManagerMBean;
1199     }
1200
1201     private boolean isShardReplicaOperationInProgress(final String shardName, final ActorRef sender) {
1202         if (shardReplicaOperationsInProgress.contains(shardName)) {
1203             LOG.debug("{}: A shard replica operation for {} is already in progress", persistenceId(), shardName);
1204             sender.tell(new Status.Failure(new IllegalStateException(
1205                 String.format("A shard replica operation for %s is already in progress", shardName))), getSelf());
1206             return true;
1207         }
1208
1209         return false;
1210     }
1211
1212     private void onAddShardReplica(final AddShardReplica shardReplicaMsg) {
1213         final String shardName = shardReplicaMsg.getShardName();
1214
1215         LOG.debug("{}: onAddShardReplica: {}", persistenceId(), shardReplicaMsg);
1216
1217         // verify the shard with the specified name is present in the cluster configuration
1218         if (!this.configuration.isShardConfigured(shardName)) {
1219             LOG.debug("{}: No module configuration exists for shard {}", persistenceId(), shardName);
1220             getSender().tell(new Status.Failure(new IllegalArgumentException(
1221                 "No module configuration exists for shard " + shardName)), getSelf());
1222             return;
1223         }
1224
1225         // Create the localShard
1226         if (schemaContext == null) {
1227             LOG.debug("{}: No SchemaContext is available in order to create a local shard instance for {}",
1228                 persistenceId(), shardName);
1229             getSender().tell(new Status.Failure(new IllegalStateException(
1230                 "No SchemaContext is available in order to create a local shard instance for " + shardName)),
1231                 getSelf());
1232             return;
1233         }
1234
1235         findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), shardName, persistenceId(),
1236                 getSelf()) {
1237             @Override
1238             public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) {
1239                 final RunnableMessage runnable = (RunnableMessage) () ->
1240                     addShard(getShardName(), response, getSender());
1241                 if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) {
1242                     getSelf().tell(runnable, getTargetActor());
1243                 }
1244             }
1245
1246             @Override
1247             public void onLocalPrimaryFound(final LocalPrimaryShardFound message) {
1248                 sendLocalReplicaAlreadyExistsReply(getShardName(), getTargetActor());
1249             }
1250         });
1251     }
1252
1253     @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
1254             justification = "https://github.com/spotbugs/spotbugs/issues/811")
1255     private void sendLocalReplicaAlreadyExistsReply(final String shardName, final ActorRef sender) {
1256         LOG.debug("{}: Local shard {} already exists", persistenceId(), shardName);
1257         sender.tell(new Status.Failure(new AlreadyExistsException(
1258             String.format("Local shard %s already exists", shardName))), getSelf());
1259     }
1260
1261     @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
1262             justification = "https://github.com/spotbugs/spotbugs/issues/811")
1263     private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
1264         if (isShardReplicaOperationInProgress(shardName, sender)) {
1265             return;
1266         }
1267
1268         shardReplicaOperationsInProgress.add(shardName);
1269
1270         final ShardInformation shardInfo;
1271         final boolean removeShardOnFailure;
1272         ShardInformation existingShardInfo = localShards.get(shardName);
1273         if (existingShardInfo == null) {
1274             removeShardOnFailure = true;
1275             ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
1276
1277             DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName)
1278                     .customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).build();
1279
1280             shardInfo = new ShardInformation(shardName, shardId, getPeerAddresses(shardName), datastoreContext,
1281                     Shard.builder(), peerAddressResolver);
1282             shardInfo.setActiveMember(false);
1283             shardInfo.setSchemaContext(schemaContext);
1284             localShards.put(shardName, shardInfo);
1285             shardInfo.setActor(newShardActor(shardInfo));
1286         } else {
1287             removeShardOnFailure = false;
1288             shardInfo = existingShardInfo;
1289         }
1290
1291         execAddShard(shardName, shardInfo, response, removeShardOnFailure, sender);
1292     }
1293
1294     private void execAddShard(final String shardName,
1295                               final ShardInformation shardInfo,
1296                               final RemotePrimaryShardFound response,
1297                               final boolean removeShardOnFailure,
1298                               final ActorRef sender) {
1299
1300         final String localShardAddress =
1301                 peerAddressResolver.getShardActorAddress(shardName, cluster.getCurrentMemberName());
1302
1303         //inform ShardLeader to add this shard as a replica by sending an AddServer message
1304         LOG.debug("{}: Sending AddServer message to peer {} for shard {}", persistenceId(),
1305                 response.getPrimaryPath(), shardInfo.getShardId());
1306
1307         final Timeout addServerTimeout = new Timeout(shardInfo.getDatastoreContext()
1308                 .getShardLeaderElectionTimeout().duration());
1309         final Future<Object> futureObj = ask(getContext().actorSelection(response.getPrimaryPath()),
1310                 new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout);
1311
1312         futureObj.onComplete(new OnComplete<>() {
1313             @Override
1314             public void onComplete(final Throwable failure, final Object addServerResponse) {
1315                 if (failure != null) {
1316                     LOG.debug("{}: AddServer request to {} for {} failed", persistenceId(),
1317                             response.getPrimaryPath(), shardName, failure);
1318
1319                     final String msg = String.format("AddServer request to leader %s for shard %s failed",
1320                             response.getPrimaryPath(), shardName);
1321                     self().tell(new ForwardedAddServerFailure(shardName, msg, failure, removeShardOnFailure), sender);
1322                 } else {
1323                     self().tell(new ForwardedAddServerReply(shardInfo, (AddServerReply)addServerResponse,
1324                             response.getPrimaryPath(), removeShardOnFailure), sender);
1325                 }
1326             }
1327         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
1328     }
1329
1330     private void onAddServerFailure(final String shardName, final String message, final Throwable failure,
1331             final ActorRef sender, final boolean removeShardOnFailure) {
1332         shardReplicaOperationsInProgress.remove(shardName);
1333
1334         if (removeShardOnFailure) {
1335             ShardInformation shardInfo = localShards.remove(shardName);
1336             if (shardInfo.getActor() != null) {
1337                 shardInfo.getActor().tell(PoisonPill.getInstance(), getSelf());
1338             }
1339         }
1340
1341         sender.tell(new Status.Failure(message == null ? failure :
1342             new RuntimeException(message, failure)), getSelf());
1343     }
1344
1345     private void onAddServerReply(final ShardInformation shardInfo, final AddServerReply replyMsg,
1346             final ActorRef sender, final String leaderPath, final boolean removeShardOnFailure) {
1347         String shardName = shardInfo.getShardName();
1348         shardReplicaOperationsInProgress.remove(shardName);
1349
1350         LOG.debug("{}: Received {} for shard {} from leader {}", persistenceId(), replyMsg, shardName, leaderPath);
1351
1352         if (replyMsg.getStatus() == ServerChangeStatus.OK) {
1353             LOG.debug("{}: Leader shard successfully added the replica shard {}", persistenceId(), shardName);
1354
1355             // Make the local shard voting capable
1356             shardInfo.setDatastoreContext(newShardDatastoreContext(shardName), getSelf());
1357             shardInfo.setActiveMember(true);
1358             persistShardList();
1359
1360             sender.tell(new Status.Success(null), getSelf());
1361         } else if (replyMsg.getStatus() == ServerChangeStatus.ALREADY_EXISTS) {
1362             sendLocalReplicaAlreadyExistsReply(shardName, sender);
1363         } else {
1364             LOG.warn("{}: Leader failed to add shard replica {} with status {}",
1365                     persistenceId(), shardName, replyMsg.getStatus());
1366
1367             Exception failure = getServerChangeException(AddServer.class, replyMsg.getStatus(), leaderPath,
1368                     shardInfo.getShardId());
1369
1370             onAddServerFailure(shardName, null, failure, sender, removeShardOnFailure);
1371         }
1372     }
1373
1374     private static Exception getServerChangeException(final Class<?> serverChange,
1375             final ServerChangeStatus serverChangeStatus, final String leaderPath, final ShardIdentifier shardId) {
1376         switch (serverChangeStatus) {
1377             case TIMEOUT:
1378                 return new TimeoutException(String.format(
1379                         "The shard leader %s timed out trying to replicate the initial data to the new shard %s."
1380                         + "Possible causes - there was a problem replicating the data or shard leadership changed "
1381                         + "while replicating the shard data", leaderPath, shardId.getShardName()));
1382             case NO_LEADER:
1383                 return new NoShardLeaderException(shardId);
1384             case NOT_SUPPORTED:
1385                 return new UnsupportedOperationException(String.format("%s request is not supported for shard %s",
1386                         serverChange.getSimpleName(), shardId.getShardName()));
1387             default :
1388                 return new RuntimeException(String.format("%s request to leader %s for shard %s failed with status %s",
1389                         serverChange.getSimpleName(), leaderPath, shardId.getShardName(), serverChangeStatus));
1390         }
1391     }
1392
1393     private void onRemoveShardReplica(final RemoveShardReplica shardReplicaMsg) {
1394         LOG.debug("{}: onRemoveShardReplica: {}", persistenceId(), shardReplicaMsg);
1395
1396         findPrimary(shardReplicaMsg.getShardName(), new AutoFindPrimaryFailureResponseHandler(getSender(),
1397                 shardReplicaMsg.getShardName(), persistenceId(), getSelf()) {
1398             @Override
1399             public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) {
1400                 doRemoveShardReplicaAsync(response.getPrimaryPath());
1401             }
1402
1403             @Override
1404             public void onLocalPrimaryFound(final LocalPrimaryShardFound response) {
1405                 doRemoveShardReplicaAsync(response.getPrimaryPath());
1406             }
1407
1408             private void doRemoveShardReplicaAsync(final String primaryPath) {
1409                 getSelf().tell((RunnableMessage) () -> removeShardReplica(shardReplicaMsg, getShardName(),
1410                         primaryPath, getSender()), getTargetActor());
1411             }
1412         });
1413     }
1414
1415     private void persistShardList() {
1416         List<String> shardList = new ArrayList<>(localShards.keySet());
1417         for (ShardInformation shardInfo : localShards.values()) {
1418             if (!shardInfo.isActiveMember()) {
1419                 shardList.remove(shardInfo.getShardName());
1420             }
1421         }
1422         LOG.debug("{}: persisting the shard list {}", persistenceId(), shardList);
1423         saveSnapshot(updateShardManagerSnapshot(shardList));
1424     }
1425
1426     private ShardManagerSnapshot updateShardManagerSnapshot(final List<String> shardList) {
1427         currentSnapshot = new ShardManagerSnapshot(shardList);
1428         return currentSnapshot;
1429     }
1430
1431     private void applyShardManagerSnapshot(final ShardManagerSnapshot snapshot) {
1432         currentSnapshot = snapshot;
1433
1434         LOG.debug("{}: onSnapshotOffer: {}", persistenceId(), currentSnapshot);
1435
1436         final MemberName currentMember = cluster.getCurrentMemberName();
1437         Set<String> configuredShardList =
1438             new HashSet<>(configuration.getMemberShardNames(currentMember));
1439         for (String shard : currentSnapshot.getShardList()) {
1440             if (!configuredShardList.contains(shard)) {
1441                 // add the current member as a replica for the shard
1442                 LOG.debug("{}: adding shard {}", persistenceId(), shard);
1443                 configuration.addMemberReplicaForShard(shard, currentMember);
1444             } else {
1445                 configuredShardList.remove(shard);
1446             }
1447         }
1448         for (String shard : configuredShardList) {
1449             // remove the member as a replica for the shard
1450             LOG.debug("{}: removing shard {}", persistenceId(), shard);
1451             configuration.removeMemberReplicaForShard(shard, currentMember);
1452         }
1453     }
1454
1455     private void onSaveSnapshotSuccess(final SaveSnapshotSuccess successMessage) {
1456         LOG.debug("{} saved ShardManager snapshot successfully. Deleting the prev snapshot if available",
1457             persistenceId());
1458         deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), successMessage.metadata().timestamp() - 1,
1459             0, 0));
1460     }
1461
1462     private void onChangeShardServersVotingStatus(final ChangeShardMembersVotingStatus changeMembersVotingStatus) {
1463         LOG.debug("{}: onChangeShardServersVotingStatus: {}", persistenceId(), changeMembersVotingStatus);
1464
1465         String shardName = changeMembersVotingStatus.getShardName();
1466         Map<String, Boolean> serverVotingStatusMap = new HashMap<>();
1467         for (Entry<String, Boolean> e: changeMembersVotingStatus.getMeberVotingStatusMap().entrySet()) {
1468             serverVotingStatusMap.put(getShardIdentifier(MemberName.forName(e.getKey()), shardName).toString(),
1469                     e.getValue());
1470         }
1471
1472         ChangeServersVotingStatus changeServersVotingStatus = new ChangeServersVotingStatus(serverVotingStatusMap);
1473
1474         findLocalShard(shardName, getSender(),
1475             localShardFound -> changeShardMembersVotingStatus(changeServersVotingStatus, shardName,
1476             localShardFound.getPath(), getSender()));
1477     }
1478
1479     private void onFlipShardMembersVotingStatus(final FlipShardMembersVotingStatus flipMembersVotingStatus) {
1480         LOG.debug("{}: onFlipShardMembersVotingStatus: {}", persistenceId(), flipMembersVotingStatus);
1481
1482         ActorRef sender = getSender();
1483         final String shardName = flipMembersVotingStatus.getShardName();
1484         findLocalShard(shardName, sender, localShardFound -> {
1485             Future<Object> future = ask(localShardFound.getPath(), GetOnDemandRaftState.INSTANCE,
1486                     Timeout.apply(30, TimeUnit.SECONDS));
1487
1488             future.onComplete(new OnComplete<>() {
1489                 @Override
1490                 public void onComplete(final Throwable failure, final Object response) {
1491                     if (failure != null) {
1492                         sender.tell(new Status.Failure(new RuntimeException(
1493                                 String.format("Failed to access local shard %s", shardName), failure)), self());
1494                         return;
1495                     }
1496
1497                     OnDemandRaftState raftState = (OnDemandRaftState) response;
1498                     Map<String, Boolean> serverVotingStatusMap = new HashMap<>();
1499                     for (Entry<String, Boolean> e: raftState.getPeerVotingStates().entrySet()) {
1500                         serverVotingStatusMap.put(e.getKey(), !e.getValue());
1501                     }
1502
1503                     serverVotingStatusMap.put(getShardIdentifier(cluster.getCurrentMemberName(), shardName)
1504                             .toString(), !raftState.isVoting());
1505
1506                     changeShardMembersVotingStatus(new ChangeServersVotingStatus(serverVotingStatusMap),
1507                             shardName, localShardFound.getPath(), sender);
1508                 }
1509             }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
1510         });
1511
1512     }
1513
1514     private void findLocalShard(final FindLocalShard message) {
1515         LOG.debug("{}: findLocalShard : {}", persistenceId(), message.getShardName());
1516
1517         final ShardInformation shardInformation = localShards.get(message.getShardName());
1518
1519         if (shardInformation == null) {
1520             LOG.debug("{}: Local shard {} not found - shards present: {}",
1521                     persistenceId(), message.getShardName(), localShards.keySet());
1522
1523             getSender().tell(new LocalShardNotFound(message.getShardName()), getSelf());
1524             return;
1525         }
1526
1527         sendResponse(shardInformation, message.isWaitUntilInitialized(), false,
1528             () -> new LocalShardFound(shardInformation.getActor()));
1529     }
1530
1531     private void findLocalShard(final String shardName, final ActorRef sender,
1532             final Consumer<LocalShardFound> onLocalShardFound) {
1533         Timeout findLocalTimeout = new Timeout(datastoreContextFactory.getBaseDatastoreContext()
1534                 .getShardInitializationTimeout().duration().$times(2));
1535
1536         Future<Object> futureObj = ask(getSelf(), new FindLocalShard(shardName, true), findLocalTimeout);
1537         futureObj.onComplete(new OnComplete<>() {
1538             @Override
1539             public void onComplete(final Throwable failure, final Object response) {
1540                 if (failure != null) {
1541                     LOG.debug("{}: Received failure from FindLocalShard for shard {}", persistenceId, shardName,
1542                             failure);
1543                     sender.tell(new Status.Failure(new RuntimeException(
1544                             String.format("Failed to find local shard %s", shardName), failure)), self());
1545                 } else {
1546                     if (response instanceof LocalShardFound) {
1547                         getSelf().tell((RunnableMessage) () -> onLocalShardFound.accept((LocalShardFound) response),
1548                                 sender);
1549                     } else if (response instanceof LocalShardNotFound) {
1550                         LOG.debug("{}: Local shard {} does not exist", persistenceId, shardName);
1551                         sender.tell(new Status.Failure(new IllegalArgumentException(
1552                             String.format("Local shard %s does not exist", shardName))), self());
1553                     } else {
1554                         LOG.debug("{}: Failed to find local shard {}: received response: {}", persistenceId, shardName,
1555                             response);
1556                         sender.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response
1557                                 : new RuntimeException(
1558                                     String.format("Failed to find local shard %s: received response: %s", shardName,
1559                                         response))), self());
1560                     }
1561                 }
1562             }
1563         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
1564     }
1565
1566     private void changeShardMembersVotingStatus(final ChangeServersVotingStatus changeServersVotingStatus,
1567             final String shardName, final ActorRef shardActorRef, final ActorRef sender) {
1568         if (isShardReplicaOperationInProgress(shardName, sender)) {
1569             return;
1570         }
1571
1572         shardReplicaOperationsInProgress.add(shardName);
1573
1574         DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName).build();
1575         final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
1576
1577         LOG.debug("{}: Sending ChangeServersVotingStatus message {} to local shard {}", persistenceId(),
1578                 changeServersVotingStatus, shardActorRef.path());
1579
1580         Timeout timeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration().$times(2));
1581         Future<Object> futureObj = ask(shardActorRef, changeServersVotingStatus, timeout);
1582
1583         futureObj.onComplete(new OnComplete<>() {
1584             @Override
1585             public void onComplete(final Throwable failure, final Object response) {
1586                 shardReplicaOperationsInProgress.remove(shardName);
1587                 if (failure != null) {
1588                     LOG.debug("{}: ChangeServersVotingStatus request to local shard {} failed", persistenceId(),
1589                         shardActorRef.path(), failure);
1590                     sender.tell(new Status.Failure(new RuntimeException(
1591                         String.format("ChangeServersVotingStatus request to local shard %s failed",
1592                             shardActorRef.path()), failure)), self());
1593                 } else {
1594                     LOG.debug("{}: Received {} from local shard {}", persistenceId(), response, shardActorRef.path());
1595
1596                     ServerChangeReply replyMsg = (ServerChangeReply) response;
1597                     if (replyMsg.getStatus() == ServerChangeStatus.OK) {
1598                         LOG.debug("{}: ChangeServersVotingStatus succeeded for shard {}", persistenceId(), shardName);
1599                         sender.tell(new Status.Success(null), getSelf());
1600                     } else if (replyMsg.getStatus() == ServerChangeStatus.INVALID_REQUEST) {
1601                         sender.tell(new Status.Failure(new IllegalArgumentException(String.format(
1602                                 "The requested voting state change for shard %s is invalid. At least one member "
1603                                 + "must be voting", shardId.getShardName()))), getSelf());
1604                     } else {
1605                         LOG.warn("{}: ChangeServersVotingStatus failed for shard {} with status {}",
1606                                 persistenceId(), shardName, replyMsg.getStatus());
1607
1608                         Exception error = getServerChangeException(ChangeServersVotingStatus.class,
1609                                 replyMsg.getStatus(), shardActorRef.path().toString(), shardId);
1610                         sender.tell(new Status.Failure(error), getSelf());
1611                     }
1612                 }
1613             }
1614         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
1615     }
1616
1617     private static final class ForwardedAddServerReply {
1618         ShardInformation shardInfo;
1619         AddServerReply addServerReply;
1620         String leaderPath;
1621         boolean removeShardOnFailure;
1622
1623         ForwardedAddServerReply(final ShardInformation shardInfo, final AddServerReply addServerReply,
1624             final String leaderPath, final boolean removeShardOnFailure) {
1625             this.shardInfo = shardInfo;
1626             this.addServerReply = addServerReply;
1627             this.leaderPath = leaderPath;
1628             this.removeShardOnFailure = removeShardOnFailure;
1629         }
1630     }
1631
1632     private static final class ForwardedAddServerFailure {
1633         String shardName;
1634         String failureMessage;
1635         Throwable failure;
1636         boolean removeShardOnFailure;
1637
1638         ForwardedAddServerFailure(final String shardName, final String failureMessage, final Throwable failure,
1639                 final boolean removeShardOnFailure) {
1640             this.shardName = shardName;
1641             this.failureMessage = failureMessage;
1642             this.failure = failure;
1643             this.removeShardOnFailure = removeShardOnFailure;
1644         }
1645     }
1646
1647     static class OnShardInitialized {
1648         private final Runnable replyRunnable;
1649         private Cancellable timeoutSchedule;
1650
1651         OnShardInitialized(final Runnable replyRunnable) {
1652             this.replyRunnable = replyRunnable;
1653         }
1654
1655         Runnable getReplyRunnable() {
1656             return replyRunnable;
1657         }
1658
1659         Cancellable getTimeoutSchedule() {
1660             return timeoutSchedule;
1661         }
1662
1663         void setTimeoutSchedule(final Cancellable timeoutSchedule) {
1664             this.timeoutSchedule = timeoutSchedule;
1665         }
1666     }
1667
1668     static class OnShardReady extends OnShardInitialized {
1669         OnShardReady(final Runnable replyRunnable) {
1670             super(replyRunnable);
1671         }
1672     }
1673
1674     private interface RunnableMessage extends Runnable {
1675     }
1676
1677     /**
1678      * The FindPrimaryResponseHandler provides specific callback methods which are invoked when a response to the
1679      * a remote or local find primary message is processed.
1680      */
1681     private interface FindPrimaryResponseHandler {
1682         /**
1683          * Invoked when a Failure message is received as a response.
1684          *
1685          * @param failure the failure exception
1686          */
1687         void onFailure(Throwable failure);
1688
1689         /**
1690          * Invoked when a RemotePrimaryShardFound response is received.
1691          *
1692          * @param response the response
1693          */
1694         void onRemotePrimaryShardFound(RemotePrimaryShardFound response);
1695
1696         /**
1697          * Invoked when a LocalPrimaryShardFound response is received.
1698          *
1699          * @param response the response
1700          */
1701         void onLocalPrimaryFound(LocalPrimaryShardFound response);
1702
1703         /**
1704          * Invoked when an unknown response is received. This is another type of failure.
1705          *
1706          * @param response the response
1707          */
1708         void onUnknownResponse(Object response);
1709     }
1710
1711     /**
1712      * The AutoFindPrimaryFailureResponseHandler automatically processes Failure responses when finding a primary
1713      * replica and sends a wrapped Failure response to some targetActor.
1714      */
1715     private abstract static class AutoFindPrimaryFailureResponseHandler implements FindPrimaryResponseHandler {
1716         private final ActorRef targetActor;
1717         private final String shardName;
1718         private final String persistenceId;
1719         private final ActorRef shardManagerActor;
1720
1721         /**
1722          * Constructs an instance.
1723          *
1724          * @param targetActor The actor to whom the Failure response should be sent when a FindPrimary failure occurs
1725          * @param shardName The name of the shard for which the primary replica had to be found
1726          * @param persistenceId The persistenceId for the ShardManager
1727          * @param shardManagerActor The ShardManager actor which triggered the call to FindPrimary
1728          */
1729         protected AutoFindPrimaryFailureResponseHandler(final ActorRef targetActor, final String shardName,
1730                 final String persistenceId, final ActorRef shardManagerActor) {
1731             this.targetActor = requireNonNull(targetActor);
1732             this.shardName = requireNonNull(shardName);
1733             this.persistenceId = requireNonNull(persistenceId);
1734             this.shardManagerActor = requireNonNull(shardManagerActor);
1735         }
1736
1737         public ActorRef getTargetActor() {
1738             return targetActor;
1739         }
1740
1741         public String getShardName() {
1742             return shardName;
1743         }
1744
1745         @Override
1746         public void onFailure(final Throwable failure) {
1747             LOG.debug("{}: Received failure from FindPrimary for shard {}", persistenceId, shardName, failure);
1748             targetActor.tell(new Status.Failure(new RuntimeException(
1749                     String.format("Failed to find leader for shard %s", shardName), failure)), shardManagerActor);
1750         }
1751
1752         @Override
1753         public void onUnknownResponse(final Object response) {
1754             LOG.debug("{}: Failed to find leader for shard {}: received response: {}", persistenceId, shardName,
1755                 response);
1756             targetActor.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response
1757                     : new RuntimeException(String.format("Failed to find leader for shard %s: received response: %s",
1758                         shardName, response))), shardManagerActor);
1759         }
1760     }
1761
1762     /**
1763      * The WrappedShardResponse class wraps a response from a Shard.
1764      */
1765     private static final class WrappedShardResponse {
1766         private final ShardIdentifier shardId;
1767         private final Object response;
1768         private final String leaderPath;
1769
1770         WrappedShardResponse(final ShardIdentifier shardId, final Object response, final String leaderPath) {
1771             this.shardId = shardId;
1772             this.response = response;
1773             this.leaderPath = leaderPath;
1774         }
1775
1776         ShardIdentifier getShardId() {
1777             return shardId;
1778         }
1779
1780         Object getResponse() {
1781             return response;
1782         }
1783
1784         String getLeaderPath() {
1785             return leaderPath;
1786         }
1787     }
1788
1789     private static final class ShardNotInitializedTimeout {
1790         private final ActorRef sender;
1791         private final ShardInformation shardInfo;
1792         private final OnShardInitialized onShardInitialized;
1793
1794         ShardNotInitializedTimeout(final ShardInformation shardInfo, final OnShardInitialized onShardInitialized,
1795             final ActorRef sender) {
1796             this.sender = sender;
1797             this.shardInfo = shardInfo;
1798             this.onShardInitialized = onShardInitialized;
1799         }
1800
1801         ActorRef getSender() {
1802             return sender;
1803         }
1804
1805         ShardInformation getShardInfo() {
1806             return shardInfo;
1807         }
1808
1809         OnShardInitialized getOnShardInitialized() {
1810             return onShardInitialized;
1811         }
1812     }
1813 }
1814
1815
1816