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