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