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