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