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