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