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