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