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