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