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