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