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