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