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