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