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