Bug 4564: Implement datastore restore from backup file
[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.serialization.Serialization;
25 import akka.util.Timeout;
26 import com.google.common.annotations.VisibleForTesting;
27 import com.google.common.base.Objects;
28 import com.google.common.base.Optional;
29 import com.google.common.base.Preconditions;
30 import com.google.common.base.Strings;
31 import com.google.common.base.Supplier;
32 import com.google.common.collect.Sets;
33 import java.io.Serializable;
34 import java.util.ArrayList;
35 import java.util.Collection;
36 import java.util.HashMap;
37 import java.util.Iterator;
38 import java.util.List;
39 import java.util.Map;
40 import java.util.Set;
41 import java.util.concurrent.CountDownLatch;
42 import java.util.concurrent.TimeUnit;
43 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
44 import org.opendaylight.controller.cluster.datastore.config.Configuration;
45 import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration;
46 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
47 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
48 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
49 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
50 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfo;
51 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfoMBean;
52 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
53 import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica;
54 import org.opendaylight.controller.cluster.datastore.messages.CreateShard;
55 import org.opendaylight.controller.cluster.datastore.messages.CreateShardReply;
56 import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot;
57 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
58 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
59 import org.opendaylight.controller.cluster.datastore.messages.LocalPrimaryShardFound;
60 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
61 import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound;
62 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
63 import org.opendaylight.controller.cluster.datastore.messages.PeerDown;
64 import org.opendaylight.controller.cluster.datastore.messages.PeerUp;
65 import org.opendaylight.controller.cluster.datastore.messages.RemoteFindPrimary;
66 import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound;
67 import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica;
68 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
69 import org.opendaylight.controller.cluster.datastore.messages.SwitchShardBehavior;
70 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
71 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
72 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
73 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
74 import org.opendaylight.controller.cluster.notifications.RoleChangeNotification;
75 import org.opendaylight.controller.cluster.raft.RaftState;
76 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
77 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
78 import org.opendaylight.controller.cluster.raft.client.messages.GetSnapshot;
79 import org.opendaylight.controller.cluster.raft.messages.AddServer;
80 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
81 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
82 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
83 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
84 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
85 import org.slf4j.Logger;
86 import org.slf4j.LoggerFactory;
87 import scala.concurrent.Future;
88 import scala.concurrent.duration.Duration;
89 import scala.concurrent.duration.FiniteDuration;
90
91 /**
92  * The ShardManager has the following jobs,
93  * <ul>
94  * <li> Create all the local shard replicas that belong on this cluster member
95  * <li> Find the address of the local shard
96  * <li> Find the primary replica for any given shard
97  * <li> Monitor the cluster members and store their addresses
98  * <ul>
99  */
100 public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
101
102     private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class);
103
104     // Stores a mapping between a shard name and it's corresponding information
105     // Shard names look like inventory, topology etc and are as specified in
106     // configuration
107     private final Map<String, ShardInformation> localShards = new HashMap<>();
108
109     // The type of a ShardManager reflects the type of the datastore itself
110     // A data store could be of type config/operational
111     private final String type;
112
113     private final ClusterWrapper cluster;
114
115     private final Configuration configuration;
116
117     private final String shardDispatcherPath;
118
119     private ShardManagerInfo mBean;
120
121     private DatastoreContextFactory datastoreContextFactory;
122
123     private final CountDownLatch waitTillReadyCountdownLatch;
124
125     private final PrimaryShardInfoFutureCache primaryShardInfoCache;
126
127     private final ShardPeerAddressResolver peerAddressResolver;
128
129     private SchemaContext schemaContext;
130
131     private DatastoreSnapshot restoreFromSnapshot;
132
133     /**
134      */
135     protected ShardManager(Builder builder) {
136
137         this.cluster = builder.cluster;
138         this.configuration = builder.configuration;
139         this.datastoreContextFactory = builder.datastoreContextFactory;
140         this.type = builder.datastoreContextFactory.getBaseDatastoreContext().getDataStoreType();
141         this.shardDispatcherPath =
142                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
143         this.waitTillReadyCountdownLatch = builder.waitTillReadyCountdownLatch;
144         this.primaryShardInfoCache = builder.primaryShardInfoCache;
145         this.restoreFromSnapshot = builder.restoreFromSnapshot;
146
147         peerAddressResolver = new ShardPeerAddressResolver(type, cluster.getCurrentMemberName());
148
149         // Subscribe this actor to cluster member events
150         cluster.subscribeToMemberEvents(getSelf());
151
152         createLocalShards();
153     }
154
155     @Override
156     public void postStop() {
157         LOG.info("Stopping ShardManager");
158
159         mBean.unregisterMBean();
160     }
161
162     @Override
163     public void handleCommand(Object message) throws Exception {
164         if (message  instanceof FindPrimary) {
165             findPrimary((FindPrimary)message);
166         } else if(message instanceof FindLocalShard){
167             findLocalShard((FindLocalShard) message);
168         } else if (message instanceof UpdateSchemaContext) {
169             updateSchemaContext(message);
170         } else if(message instanceof ActorInitialized) {
171             onActorInitialized(message);
172         } else if (message instanceof ClusterEvent.MemberUp){
173             memberUp((ClusterEvent.MemberUp) message);
174         } else if (message instanceof ClusterEvent.MemberExited){
175             memberExited((ClusterEvent.MemberExited) message);
176         } else if(message instanceof ClusterEvent.MemberRemoved) {
177             memberRemoved((ClusterEvent.MemberRemoved) message);
178         } else if(message instanceof ClusterEvent.UnreachableMember) {
179             memberUnreachable((ClusterEvent.UnreachableMember)message);
180         } else if(message instanceof ClusterEvent.ReachableMember) {
181             memberReachable((ClusterEvent.ReachableMember) message);
182         } else if(message instanceof DatastoreContextFactory) {
183             onDatastoreContextFactory((DatastoreContextFactory)message);
184         } else if(message instanceof RoleChangeNotification) {
185             onRoleChangeNotification((RoleChangeNotification) message);
186         } else if(message instanceof FollowerInitialSyncUpStatus){
187             onFollowerInitialSyncStatus((FollowerInitialSyncUpStatus) message);
188         } else if(message instanceof ShardNotInitializedTimeout) {
189             onShardNotInitializedTimeout((ShardNotInitializedTimeout)message);
190         } else if(message instanceof ShardLeaderStateChanged) {
191             onLeaderStateChanged((ShardLeaderStateChanged) message);
192         } else if(message instanceof SwitchShardBehavior){
193             onSwitchShardBehavior((SwitchShardBehavior) message);
194         } else if(message instanceof CreateShard) {
195             onCreateShard((CreateShard)message);
196         } else if(message instanceof AddShardReplica){
197             onAddShardReplica((AddShardReplica)message);
198         } else if(message instanceof RemoveShardReplica){
199             onRemoveShardReplica((RemoveShardReplica)message);
200         } else if(message instanceof GetSnapshot) {
201             onGetSnapshot();
202         } else {
203             unknownMessage(message);
204         }
205
206     }
207
208     private void onGetSnapshot() {
209         LOG.debug("{}: onGetSnapshot", persistenceId());
210
211         List<String> notInitialized = null;
212         for(ShardInformation shardInfo: localShards.values()) {
213             if(!shardInfo.isShardInitialized()) {
214                 if(notInitialized == null) {
215                     notInitialized = new ArrayList<>();
216                 }
217
218                 notInitialized.add(shardInfo.getShardName());
219             }
220         }
221
222         if(notInitialized != null) {
223             getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(String.format(
224                     "%d shard(s) %s are not initialized", notInitialized.size(), notInitialized))), getSelf());
225             return;
226         }
227
228         byte[] shardManagerSnapshot = null;
229         ActorRef replyActor = getContext().actorOf(ShardManagerGetSnapshotReplyActor.props(
230                 new ArrayList<>(localShards.keySet()), type, shardManagerSnapshot , getSender(), persistenceId(),
231                 datastoreContextFactory.getBaseDatastoreContext().getShardInitializationTimeout().duration()));
232
233         for(ShardInformation shardInfo: localShards.values()) {
234             shardInfo.getActor().tell(GetSnapshot.INSTANCE, replyActor);
235         }
236     }
237
238     private void onCreateShard(CreateShard createShard) {
239         Object reply;
240         try {
241             ModuleShardConfiguration moduleShardConfig = createShard.getModuleShardConfig();
242             if(localShards.containsKey(moduleShardConfig.getShardName())) {
243                 throw new IllegalStateException(String.format("Shard with name %s already exists",
244                         moduleShardConfig.getShardName()));
245             }
246
247             configuration.addModuleShardConfiguration(moduleShardConfig);
248
249             ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), moduleShardConfig.getShardName());
250             Map<String, String> peerAddresses = getPeerAddresses(moduleShardConfig.getShardName()/*,
251                     moduleShardConfig.getShardMemberNames()*/);
252
253             LOG.debug("onCreateShard: shardId: {}, memberNames: {}. peerAddresses: {}", shardId,
254                     moduleShardConfig.getShardMemberNames(), peerAddresses);
255
256             DatastoreContext shardDatastoreContext = createShard.getDatastoreContext();
257             if(shardDatastoreContext == null) {
258                 shardDatastoreContext = newShardDatastoreContext(moduleShardConfig.getShardName());
259             } else {
260                 shardDatastoreContext = DatastoreContext.newBuilderFrom(shardDatastoreContext).shardPeerAddressResolver(
261                         peerAddressResolver).build();
262             }
263
264             ShardInformation info = new ShardInformation(moduleShardConfig.getShardName(), shardId, peerAddresses,
265                     shardDatastoreContext, createShard.getShardBuilder(), peerAddressResolver);
266             localShards.put(info.getShardName(), info);
267
268             mBean.addLocalShard(shardId.toString());
269
270             if(schemaContext != null) {
271                 info.setActor(newShardActor(schemaContext, info));
272             }
273
274             reply = new CreateShardReply();
275         } catch (Exception e) {
276             LOG.error("onCreateShard failed", e);
277             reply = new akka.actor.Status.Failure(e);
278         }
279
280         if(getSender() != null && !getContext().system().deadLetters().equals(getSender())) {
281             getSender().tell(reply, getSelf());
282         }
283     }
284
285     private DatastoreContext.Builder newShardDatastoreContextBuilder(String shardName) {
286         return DatastoreContext.newBuilderFrom(datastoreContextFactory.getShardDatastoreContext(shardName)).
287                 shardPeerAddressResolver(peerAddressResolver);
288     }
289
290     private DatastoreContext newShardDatastoreContext(String shardName) {
291         return newShardDatastoreContextBuilder(shardName).build();
292     }
293
294     private void checkReady(){
295         if (isReadyWithLeaderId()) {
296             LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
297                     persistenceId(), type, waitTillReadyCountdownLatch.getCount());
298
299             waitTillReadyCountdownLatch.countDown();
300         }
301     }
302
303     private void onLeaderStateChanged(ShardLeaderStateChanged leaderStateChanged) {
304         LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged);
305
306         ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId());
307         if(shardInformation != null) {
308             shardInformation.setLocalDataTree(leaderStateChanged.getLocalShardDataTree());
309             shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion());
310             if(shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) {
311                 primaryShardInfoCache.remove(shardInformation.getShardName());
312             }
313
314             checkReady();
315         } else {
316             LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
317         }
318     }
319
320     private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) {
321         ShardInformation shardInfo = message.getShardInfo();
322
323         LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
324                 shardInfo.getShardName());
325
326         shardInfo.removeOnShardInitialized(message.getOnShardInitialized());
327
328         if(!shardInfo.isShardInitialized()) {
329             LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(), shardInfo.getShardName());
330             message.getSender().tell(createNotInitializedException(shardInfo.shardId), getSelf());
331         } else {
332             LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
333             message.getSender().tell(createNoShardLeaderException(shardInfo.shardId), getSelf());
334         }
335     }
336
337     private void onFollowerInitialSyncStatus(FollowerInitialSyncUpStatus status) {
338         LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(),
339                 status.getName(), status.isInitialSyncDone());
340
341         ShardInformation shardInformation = findShardInformation(status.getName());
342
343         if(shardInformation != null) {
344             shardInformation.setFollowerSyncStatus(status.isInitialSyncDone());
345
346             mBean.setSyncStatus(isInSync());
347         }
348
349     }
350
351     private void onRoleChangeNotification(RoleChangeNotification roleChanged) {
352         LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(),
353                 roleChanged.getOldRole(), roleChanged.getNewRole());
354
355         ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
356         if(shardInformation != null) {
357             shardInformation.setRole(roleChanged.getNewRole());
358             checkReady();
359             mBean.setSyncStatus(isInSync());
360         }
361     }
362
363
364     private ShardInformation findShardInformation(String memberId) {
365         for(ShardInformation info : localShards.values()){
366             if(info.getShardId().toString().equals(memberId)){
367                 return info;
368             }
369         }
370
371         return null;
372     }
373
374     private boolean isReadyWithLeaderId() {
375         boolean isReady = true;
376         for (ShardInformation info : localShards.values()) {
377             if(!info.isShardReadyWithLeaderId()){
378                 isReady = false;
379                 break;
380             }
381         }
382         return isReady;
383     }
384
385     private boolean isInSync(){
386         for (ShardInformation info : localShards.values()) {
387             if(!info.isInSync()){
388                 return false;
389             }
390         }
391         return true;
392     }
393
394     private void onActorInitialized(Object message) {
395         final ActorRef sender = getSender();
396
397         if (sender == null) {
398             return; //why is a non-actor sending this message? Just ignore.
399         }
400
401         String actorName = sender.path().name();
402         //find shard name from actor name; actor name is stringified shardId
403         ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(actorName).build();
404
405         if (shardId.getShardName() == null) {
406             return;
407         }
408
409         markShardAsInitialized(shardId.getShardName());
410     }
411
412     private void markShardAsInitialized(String shardName) {
413         LOG.debug("{}: Initializing shard [{}]", persistenceId(), shardName);
414
415         ShardInformation shardInformation = localShards.get(shardName);
416         if (shardInformation != null) {
417             shardInformation.setActorInitialized();
418
419             shardInformation.getActor().tell(new RegisterRoleChangeListener(), self());
420         }
421     }
422
423     @Override
424     protected void handleRecover(Object message) throws Exception {
425         if (message instanceof RecoveryCompleted) {
426             LOG.info("Recovery complete : {}", persistenceId());
427
428             // We no longer persist SchemaContext modules so delete all the prior messages from the akka
429             // journal on upgrade from Helium.
430             deleteMessages(lastSequenceNr());
431         }
432     }
433
434     private void findLocalShard(FindLocalShard message) {
435         final ShardInformation shardInformation = localShards.get(message.getShardName());
436
437         if(shardInformation == null){
438             getSender().tell(new LocalShardNotFound(message.getShardName()), getSelf());
439             return;
440         }
441
442         sendResponse(shardInformation, message.isWaitUntilInitialized(), false, new Supplier<Object>() {
443             @Override
444             public Object get() {
445                 return new LocalShardFound(shardInformation.getActor());
446             }
447         });
448     }
449
450     private void sendResponse(ShardInformation shardInformation, boolean doWait,
451             boolean wantShardReady, final Supplier<Object> messageSupplier) {
452         if (!shardInformation.isShardInitialized() || (wantShardReady && !shardInformation.isShardReadyWithLeaderId())) {
453             if(doWait) {
454                 final ActorRef sender = getSender();
455                 final ActorRef self = self();
456
457                 Runnable replyRunnable = new Runnable() {
458                     @Override
459                     public void run() {
460                         sender.tell(messageSupplier.get(), self);
461                     }
462                 };
463
464                 OnShardInitialized onShardInitialized = wantShardReady ? new OnShardReady(replyRunnable) :
465                     new OnShardInitialized(replyRunnable);
466
467                 shardInformation.addOnShardInitialized(onShardInitialized);
468
469                 FiniteDuration timeout = shardInformation.getDatastoreContext().getShardInitializationTimeout().duration();
470                 if(shardInformation.isShardInitialized()) {
471                     // If the shard is already initialized then we'll wait enough time for the shard to
472                     // elect a leader, ie 2 times the election timeout.
473                     timeout = FiniteDuration.create(shardInformation.getDatastoreContext().getShardRaftConfig()
474                             .getElectionTimeOutInterval().toMillis() * 2, TimeUnit.MILLISECONDS);
475                 }
476
477                 LOG.debug("{}: Scheduling {} ms timer to wait for shard {}", persistenceId(), timeout.toMillis(),
478                         shardInformation.getShardName());
479
480                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
481                         timeout, getSelf(),
482                         new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
483                         getContext().dispatcher(), getSelf());
484
485                 onShardInitialized.setTimeoutSchedule(timeoutSchedule);
486
487             } else if (!shardInformation.isShardInitialized()) {
488                 LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(),
489                         shardInformation.getShardName());
490                 getSender().tell(createNotInitializedException(shardInformation.shardId), getSelf());
491             } else {
492                 LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
493                         shardInformation.getShardName());
494                 getSender().tell(createNoShardLeaderException(shardInformation.shardId), getSelf());
495             }
496
497             return;
498         }
499
500         getSender().tell(messageSupplier.get(), getSelf());
501     }
502
503     private static NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) {
504         return new NoShardLeaderException(null, shardId.toString());
505     }
506
507     private static NotInitializedException createNotInitializedException(ShardIdentifier shardId) {
508         return new NotInitializedException(String.format(
509                 "Found primary shard %s but it's not initialized yet. Please try again later", shardId));
510     }
511
512     private void memberRemoved(ClusterEvent.MemberRemoved message) {
513         String memberName = message.member().roles().head();
514
515         LOG.debug("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName,
516                 message.member().address());
517
518         peerAddressResolver.removePeerAddress(memberName);
519
520         for(ShardInformation info : localShards.values()){
521             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
522         }
523     }
524
525     private void memberExited(ClusterEvent.MemberExited message) {
526         String memberName = message.member().roles().head();
527
528         LOG.debug("{}: Received MemberExited: memberName: {}, address: {}", persistenceId(), memberName,
529                 message.member().address());
530
531         peerAddressResolver.removePeerAddress(memberName);
532
533         for(ShardInformation info : localShards.values()){
534             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
535         }
536     }
537
538     private void memberUp(ClusterEvent.MemberUp message) {
539         String memberName = message.member().roles().head();
540
541         LOG.debug("{}: Received MemberUp: memberName: {}, address: {}", persistenceId(), memberName,
542                 message.member().address());
543
544         addPeerAddress(memberName, message.member().address());
545
546         checkReady();
547     }
548
549     private void addPeerAddress(String memberName, Address address) {
550         peerAddressResolver.addPeerAddress(memberName, address);
551
552         for(ShardInformation info : localShards.values()){
553             String shardName = info.getShardName();
554             String peerId = getShardIdentifier(memberName, shardName).toString();
555             info.updatePeerAddress(peerId, peerAddressResolver.getShardActorAddress(shardName, memberName), getSelf());
556
557             info.peerUp(memberName, peerId, getSelf());
558         }
559     }
560
561     private void memberReachable(ClusterEvent.ReachableMember message) {
562         String memberName = message.member().roles().head();
563         LOG.debug("Received ReachableMember: memberName {}, address: {}", memberName, message.member().address());
564
565         addPeerAddress(memberName, message.member().address());
566
567         markMemberAvailable(memberName);
568     }
569
570     private void memberUnreachable(ClusterEvent.UnreachableMember message) {
571         String memberName = message.member().roles().head();
572         LOG.debug("Received UnreachableMember: memberName {}, address: {}", memberName, message.member().address());
573
574         markMemberUnavailable(memberName);
575     }
576
577     private void markMemberUnavailable(final String memberName) {
578         for(ShardInformation info : localShards.values()){
579             String leaderId = info.getLeaderId();
580             if(leaderId != null && leaderId.contains(memberName)) {
581                 LOG.debug("Marking Leader {} as unavailable.", leaderId);
582                 info.setLeaderAvailable(false);
583
584                 primaryShardInfoCache.remove(info.getShardName());
585             }
586
587             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
588         }
589     }
590
591     private void markMemberAvailable(final String memberName) {
592         for(ShardInformation info : localShards.values()){
593             String leaderId = info.getLeaderId();
594             if(leaderId != null && leaderId.contains(memberName)) {
595                 LOG.debug("Marking Leader {} as available.", leaderId);
596                 info.setLeaderAvailable(true);
597             }
598
599             info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
600         }
601     }
602
603     private void onDatastoreContextFactory(DatastoreContextFactory factory) {
604         datastoreContextFactory = factory;
605         for (ShardInformation info : localShards.values()) {
606             info.setDatastoreContext(newShardDatastoreContext(info.getShardName()), getSelf());
607         }
608     }
609
610     private void onSwitchShardBehavior(SwitchShardBehavior message) {
611         ShardIdentifier identifier = ShardIdentifier.builder().fromShardIdString(message.getShardName()).build();
612
613         ShardInformation shardInformation = localShards.get(identifier.getShardName());
614
615         if(shardInformation != null && shardInformation.getActor() != null) {
616             shardInformation.getActor().tell(
617                     new SwitchBehavior(RaftState.valueOf(message.getNewState()), message.getTerm()), getSelf());
618         } else {
619             LOG.warn("Could not switch the behavior of shard {} to {} - shard is not yet available",
620                     message.getShardName(), message.getNewState());
621         }
622     }
623
624     /**
625      * Notifies all the local shards of a change in the schema context
626      *
627      * @param message
628      */
629     private void updateSchemaContext(final Object message) {
630         schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
631
632         LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getAllModuleIdentifiers().size());
633
634         for (ShardInformation info : localShards.values()) {
635             if (info.getActor() == null) {
636                 LOG.debug("Creating Shard {}", info.getShardId());
637                 info.setActor(newShardActor(schemaContext, info));
638             } else {
639                 info.getActor().tell(message, getSelf());
640             }
641         }
642     }
643
644     @VisibleForTesting
645     protected ClusterWrapper getCluster() {
646         return cluster;
647     }
648
649     @VisibleForTesting
650     protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
651         return getContext().actorOf(info.newProps(schemaContext)
652                 .withDispatcher(shardDispatcherPath), info.getShardId().toString());
653     }
654
655     private void findPrimary(FindPrimary message) {
656         LOG.debug("{}: In findPrimary: {}", persistenceId(), message);
657
658         final String shardName = message.getShardName();
659         final boolean canReturnLocalShardState = !(message instanceof RemoteFindPrimary);
660
661         // First see if the there is a local replica for the shard
662         final ShardInformation info = localShards.get(shardName);
663         if (info != null) {
664             sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
665                 @Override
666                 public Object get() {
667                     String primaryPath = info.getSerializedLeaderActor();
668                     Object found = canReturnLocalShardState && info.isLeader() ?
669                             new LocalPrimaryShardFound(primaryPath, info.getLocalShardDataTree().get()) :
670                                 new RemotePrimaryShardFound(primaryPath, info.getLeaderVersion());
671
672                             if(LOG.isDebugEnabled()) {
673                                 LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
674                             }
675
676                             return found;
677                 }
678             });
679
680             return;
681         }
682
683         for(String address: peerAddressResolver.getShardManagerPeerActorAddresses()) {
684             LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}", persistenceId(),
685                     shardName, address);
686
687             getContext().actorSelection(address).forward(new RemoteFindPrimary(shardName,
688                     message.isWaitUntilReady()), getContext());
689             return;
690         }
691
692         LOG.debug("{}: No shard found for {}", persistenceId(), shardName);
693
694         getSender().tell(new PrimaryNotFoundException(
695                 String.format("No primary shard found for %s.", shardName)), getSelf());
696     }
697
698     /**
699      * Construct the name of the shard actor given the name of the member on
700      * which the shard resides and the name of the shard
701      *
702      * @param memberName
703      * @param shardName
704      * @return
705      */
706     private ShardIdentifier getShardIdentifier(String memberName, String shardName){
707         return peerAddressResolver.getShardIdentifier(memberName, shardName);
708     }
709
710     /**
711      * Create shards that are local to the member on which the ShardManager
712      * runs
713      *
714      */
715     private void createLocalShards() {
716         String memberName = this.cluster.getCurrentMemberName();
717         Collection<String> memberShardNames = this.configuration.getMemberShardNames(memberName);
718
719         Map<String, DatastoreSnapshot.ShardSnapshot> shardSnapshots = new HashMap<>();
720         if(restoreFromSnapshot != null)
721         {
722             for(DatastoreSnapshot.ShardSnapshot snapshot: restoreFromSnapshot.getShardSnapshots()) {
723                 shardSnapshots.put(snapshot.getName(), snapshot);
724             }
725         }
726
727         restoreFromSnapshot = null; // null out to GC
728
729         List<String> localShardActorNames = new ArrayList<>();
730         for(String shardName : memberShardNames){
731             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
732             Map<String, String> peerAddresses = getPeerAddresses(shardName);
733             localShardActorNames.add(shardId.toString());
734             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses,
735                     newShardDatastoreContext(shardName), Shard.builder().restoreFromSnapshot(
736                         shardSnapshots.get(shardName)), peerAddressResolver));
737         }
738
739         mBean = ShardManagerInfo.createShardManagerMBean(memberName, "shard-manager-" + this.type,
740                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType(), localShardActorNames);
741
742         mBean.setShardManager(this);
743     }
744
745     /**
746      * Given the name of the shard find the addresses of all it's peers
747      *
748      * @param shardName
749      */
750     private Map<String, String> getPeerAddresses(String shardName) {
751         Collection<String> members = configuration.getMembersFromShardName(shardName);
752         Map<String, String> peerAddresses = new HashMap<>();
753
754         String currentMemberName = this.cluster.getCurrentMemberName();
755
756         for(String memberName : members) {
757             if(!currentMemberName.equals(memberName)) {
758                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
759                 String address = peerAddressResolver.getShardActorAddress(shardName, memberName);
760                 peerAddresses.put(shardId.toString(), address);
761             }
762         }
763         return peerAddresses;
764     }
765
766     @Override
767     public SupervisorStrategy supervisorStrategy() {
768
769         return new OneForOneStrategy(10, Duration.create("1 minute"),
770                 new Function<Throwable, SupervisorStrategy.Directive>() {
771             @Override
772             public SupervisorStrategy.Directive apply(Throwable t) {
773                 LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
774                 return SupervisorStrategy.resume();
775             }
776         }
777                 );
778
779     }
780
781     @Override
782     public String persistenceId() {
783         return "shard-manager-" + type;
784     }
785
786     @VisibleForTesting
787     ShardManagerInfoMBean getMBean(){
788         return mBean;
789     }
790
791     private void checkLocalShardExists(final String shardName, final ActorRef sender) {
792         if (localShards.containsKey(shardName)) {
793             String msg = String.format("Local shard %s already exists", shardName);
794             LOG.debug ("{}: {}", persistenceId(), msg);
795             sender.tell(new akka.actor.Status.Failure(new IllegalArgumentException(msg)), getSelf());
796         }
797     }
798
799     private void onAddShardReplica (AddShardReplica shardReplicaMsg) {
800         final String shardName = shardReplicaMsg.getShardName();
801
802         // verify the local shard replica is already available in the controller node
803         LOG.debug ("onAddShardReplica: {}", shardReplicaMsg);
804
805         checkLocalShardExists(shardName, getSender());
806
807         // verify the shard with the specified name is present in the cluster configuration
808         if (!(this.configuration.isShardConfigured(shardName))) {
809             String msg = String.format("No module configuration exists for shard %s", shardName);
810             LOG.debug ("{}: {}", persistenceId(), msg);
811             getSender().tell(new akka.actor.Status.Failure(new IllegalArgumentException(msg)), getSelf());
812             return;
813         }
814
815         // Create the localShard
816         if (schemaContext == null) {
817             String msg = String.format(
818                   "No SchemaContext is available in order to create a local shard instance for %s", shardName);
819             LOG.debug ("{}: {}", persistenceId(), msg);
820             getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(msg)), getSelf());
821             return;
822         }
823
824         Map<String, String> peerAddresses = getPeerAddresses(shardName);
825         if (peerAddresses.isEmpty()) {
826             String msg = String.format("Cannot add replica for shard %s because no peer is available", shardName);
827             LOG.debug ("{}: {}", persistenceId(), msg);
828             getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(msg)), getSelf());
829             return;
830         }
831
832         Timeout findPrimaryTimeout = new Timeout(datastoreContextFactory.getBaseDatastoreContext().
833                 getShardInitializationTimeout().duration().$times(2));
834
835         final ActorRef sender = getSender();
836         Future<Object> futureObj = ask(getSelf(), new RemoteFindPrimary(shardName, true), findPrimaryTimeout);
837         futureObj.onComplete(new OnComplete<Object>() {
838             @Override
839             public void onComplete(Throwable failure, Object response) {
840                 if (failure != null) {
841                     LOG.debug ("{}: Received failure from FindPrimary for shard {}", persistenceId(), shardName, failure);
842                     sender.tell(new akka.actor.Status.Failure(new RuntimeException(
843                         String.format("Failed to find leader for shard %s", shardName), failure)),
844                         getSelf());
845                 } else {
846                     if (!(response instanceof RemotePrimaryShardFound)) {
847                         String msg = String.format("Failed to find leader for shard %s: received response: %s",
848                                 shardName, response);
849                         LOG.debug ("{}: {}", persistenceId(), msg);
850                         sender.tell(new akka.actor.Status.Failure(new RuntimeException(msg)), getSelf());
851                         return;
852                     }
853
854                     RemotePrimaryShardFound message = (RemotePrimaryShardFound)response;
855                     addShard (shardName, message, sender);
856                 }
857             }
858         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
859     }
860
861     private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
862         checkLocalShardExists(shardName, sender);
863
864         ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
865         String localShardAddress = peerAddressResolver.getShardActorAddress(shardName, cluster.getCurrentMemberName());
866
867         DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName).customRaftPolicyImplementation(
868                 DisableElectionsRaftPolicy.class.getName()).build();
869
870         final ShardInformation shardInfo = new ShardInformation(shardName, shardId,
871                           getPeerAddresses(shardName), datastoreContext,
872                           Shard.builder(), peerAddressResolver);
873         localShards.put(shardName, shardInfo);
874         shardInfo.setActor(newShardActor(schemaContext, shardInfo));
875
876         //inform ShardLeader to add this shard as a replica by sending an AddServer message
877         LOG.debug ("{}: Sending AddServer message to peer {} for shard {}", persistenceId(),
878                 response.getPrimaryPath(), shardId);
879
880         Timeout addServerTimeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration().$times(4));
881         Future<Object> futureObj = ask(getContext().actorSelection(response.getPrimaryPath()),
882             new AddServer(shardId.toString(), localShardAddress, true), addServerTimeout);
883
884         futureObj.onComplete(new OnComplete<Object>() {
885             @Override
886             public void onComplete(Throwable failure, Object addServerResponse) {
887                 if (failure != null) {
888                     LOG.debug ("{}: AddServer request to {} for {} failed", persistenceId(),
889                             response.getPrimaryPath(), shardName, failure);
890
891                     // Remove the shard
892                     localShards.remove(shardName);
893                     if (shardInfo.getActor() != null) {
894                         shardInfo.getActor().tell(PoisonPill.getInstance(), getSelf());
895                     }
896
897                     sender.tell(new akka.actor.Status.Failure(new RuntimeException(
898                         String.format("AddServer request to leader %s for shard %s failed",
899                             response.getPrimaryPath(), shardName), failure)), getSelf());
900                 } else {
901                     AddServerReply reply = (AddServerReply)addServerResponse;
902                     onAddServerReply(shardName, shardInfo, reply, sender, response.getPrimaryPath());
903                 }
904             }
905         }, new Dispatchers(context().system().dispatchers()).
906             getDispatcher(Dispatchers.DispatcherType.Client));
907         return;
908     }
909
910     private void onAddServerReply (String shardName, ShardInformation shardInfo,
911                                    AddServerReply replyMsg, ActorRef sender, String leaderPath) {
912         LOG.debug ("{}: Received {} for shard {} from leader {}", persistenceId(), replyMsg, shardName, leaderPath);
913
914         if (replyMsg.getStatus() == ServerChangeStatus.OK) {
915             LOG.debug ("{}: Leader shard successfully added the replica shard {}", persistenceId(), shardName);
916
917             // Make the local shard voting capable
918             shardInfo.setDatastoreContext(newShardDatastoreContext(shardName), getSelf());
919
920             mBean.addLocalShard(shardInfo.getShardId().toString());
921             sender.tell(new akka.actor.Status.Success(true), getSelf());
922         } else {
923             LOG.warn ("{}: Leader failed to add shard replica {} with status {} - removing the local shard",
924                     persistenceId(), shardName, replyMsg.getStatus());
925
926             //remove the local replica created
927             localShards.remove(shardName);
928             if (shardInfo.getActor() != null) {
929                 shardInfo.getActor().tell(PoisonPill.getInstance(), getSelf());
930             }
931             switch (replyMsg.getStatus()) {
932                 case TIMEOUT:
933                     sender.tell(new akka.actor.Status.Failure(new RuntimeException(
934                         String.format("The shard leader %s timed out trying to replicate the initial data to the new shard %s. Possible causes - there was a problem replicating the data or shard leadership changed while replicating the shard data",
935                             leaderPath, shardName))), getSelf());
936                     break;
937                 case NO_LEADER:
938                     sender.tell(new akka.actor.Status.Failure(new RuntimeException(String.format(
939                         "There is no shard leader available for shard %s", shardName))), getSelf());
940                     break;
941                 default :
942                     sender.tell(new akka.actor.Status.Failure(new RuntimeException(String.format(
943                         "AddServer request to leader %s for shard %s failed with status %s",
944                         leaderPath, shardName, replyMsg.getStatus()))), getSelf());
945             }
946         }
947     }
948
949     private void onRemoveShardReplica (RemoveShardReplica shardReplicaMsg) {
950         String shardName = shardReplicaMsg.getShardName();
951
952         // verify the local shard replica is available in the controller node
953         if (!localShards.containsKey(shardName)) {
954             String msg = String.format("Local shard %s does not", shardName);
955             LOG.debug ("{}: {}", persistenceId(), msg);
956             getSender().tell(new akka.actor.Status.Failure(new IllegalArgumentException(msg)), getSelf());
957             return;
958         }
959         // call RemoveShard for the shardName
960         getSender().tell(new akka.actor.Status.Success(true), getSelf());
961         return;
962     }
963
964     @VisibleForTesting
965     protected static class ShardInformation {
966         private final ShardIdentifier shardId;
967         private final String shardName;
968         private ActorRef actor;
969         private ActorPath actorPath;
970         private final Map<String, String> initialPeerAddresses;
971         private Optional<DataTree> localShardDataTree;
972         private boolean leaderAvailable = false;
973
974         // flag that determines if the actor is ready for business
975         private boolean actorInitialized = false;
976
977         private boolean followerSyncStatus = false;
978
979         private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
980         private String role ;
981         private String leaderId;
982         private short leaderVersion;
983
984         private DatastoreContext datastoreContext;
985         private Shard.AbstractBuilder<?, ?> builder;
986         private final ShardPeerAddressResolver addressResolver;
987
988         private ShardInformation(String shardName, ShardIdentifier shardId,
989                 Map<String, String> initialPeerAddresses, DatastoreContext datastoreContext,
990                 Shard.AbstractBuilder<?, ?> builder, ShardPeerAddressResolver addressResolver) {
991             this.shardName = shardName;
992             this.shardId = shardId;
993             this.initialPeerAddresses = initialPeerAddresses;
994             this.datastoreContext = datastoreContext;
995             this.builder = builder;
996             this.addressResolver = addressResolver;
997         }
998
999         Props newProps(SchemaContext schemaContext) {
1000             Preconditions.checkNotNull(builder);
1001             Props props = builder.id(shardId).peerAddresses(initialPeerAddresses).datastoreContext(datastoreContext).
1002                     schemaContext(schemaContext).props();
1003             builder = null;
1004             return props;
1005         }
1006
1007         String getShardName() {
1008             return shardName;
1009         }
1010
1011         ActorRef getActor(){
1012             return actor;
1013         }
1014
1015         ActorPath getActorPath() {
1016             return actorPath;
1017         }
1018
1019         void setActor(ActorRef actor) {
1020             this.actor = actor;
1021             this.actorPath = actor.path();
1022         }
1023
1024         ShardIdentifier getShardId() {
1025             return shardId;
1026         }
1027
1028         void setLocalDataTree(Optional<DataTree> localShardDataTree) {
1029             this.localShardDataTree = localShardDataTree;
1030         }
1031
1032         Optional<DataTree> getLocalShardDataTree() {
1033             return localShardDataTree;
1034         }
1035
1036         DatastoreContext getDatastoreContext() {
1037             return datastoreContext;
1038         }
1039
1040         void setDatastoreContext(DatastoreContext datastoreContext, ActorRef sender) {
1041             this.datastoreContext = datastoreContext;
1042             if (actor != null) {
1043                 LOG.debug ("Sending new DatastoreContext to {}", shardId);
1044                 actor.tell(this.datastoreContext, sender);
1045             }
1046         }
1047
1048         void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
1049             LOG.info("updatePeerAddress for peer {} with address {}", peerId, peerAddress);
1050
1051             if(actor != null) {
1052                 if(LOG.isDebugEnabled()) {
1053                     LOG.debug("Sending PeerAddressResolved for peer {} with address {} to {}",
1054                             peerId, peerAddress, actor.path());
1055                 }
1056
1057                 actor.tell(new PeerAddressResolved(peerId, peerAddress), sender);
1058             }
1059
1060             notifyOnShardInitializedCallbacks();
1061         }
1062
1063         void peerDown(String memberName, String peerId, ActorRef sender) {
1064             if(actor != null) {
1065                 actor.tell(new PeerDown(memberName, peerId), sender);
1066             }
1067         }
1068
1069         void peerUp(String memberName, String peerId, ActorRef sender) {
1070             if(actor != null) {
1071                 actor.tell(new PeerUp(memberName, peerId), sender);
1072             }
1073         }
1074
1075         boolean isShardReady() {
1076             return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
1077         }
1078
1079         boolean isShardReadyWithLeaderId() {
1080             return leaderAvailable && isShardReady() && !RaftState.IsolatedLeader.name().equals(role) &&
1081                     (isLeader() || addressResolver.resolve(leaderId) != null);
1082         }
1083
1084         boolean isShardInitialized() {
1085             return getActor() != null && actorInitialized;
1086         }
1087
1088         boolean isLeader() {
1089             return Objects.equal(leaderId, shardId.toString());
1090         }
1091
1092         String getSerializedLeaderActor() {
1093             if(isLeader()) {
1094                 return Serialization.serializedActorPath(getActor());
1095             } else {
1096                 return addressResolver.resolve(leaderId);
1097             }
1098         }
1099
1100         void setActorInitialized() {
1101             LOG.debug("Shard {} is initialized", shardId);
1102
1103             this.actorInitialized = true;
1104
1105             notifyOnShardInitializedCallbacks();
1106         }
1107
1108         private void notifyOnShardInitializedCallbacks() {
1109             if(onShardInitializedSet.isEmpty()) {
1110                 return;
1111             }
1112
1113             boolean ready = isShardReadyWithLeaderId();
1114
1115             if(LOG.isDebugEnabled()) {
1116                 LOG.debug("Shard {} is {} - notifying {} OnShardInitialized callbacks", shardId,
1117                         ready ? "ready" : "initialized", onShardInitializedSet.size());
1118             }
1119
1120             Iterator<OnShardInitialized> iter = onShardInitializedSet.iterator();
1121             while(iter.hasNext()) {
1122                 OnShardInitialized onShardInitialized = iter.next();
1123                 if(!(onShardInitialized instanceof OnShardReady) || ready) {
1124                     iter.remove();
1125                     onShardInitialized.getTimeoutSchedule().cancel();
1126                     onShardInitialized.getReplyRunnable().run();
1127                 }
1128             }
1129         }
1130
1131         void addOnShardInitialized(OnShardInitialized onShardInitialized) {
1132             onShardInitializedSet.add(onShardInitialized);
1133         }
1134
1135         void removeOnShardInitialized(OnShardInitialized onShardInitialized) {
1136             onShardInitializedSet.remove(onShardInitialized);
1137         }
1138
1139         void setRole(String newRole) {
1140             this.role = newRole;
1141
1142             notifyOnShardInitializedCallbacks();
1143         }
1144
1145         void setFollowerSyncStatus(boolean syncStatus){
1146             this.followerSyncStatus = syncStatus;
1147         }
1148
1149         boolean isInSync(){
1150             if(RaftState.Follower.name().equals(this.role)){
1151                 return followerSyncStatus;
1152             } else if(RaftState.Leader.name().equals(this.role)){
1153                 return true;
1154             }
1155
1156             return false;
1157         }
1158
1159         boolean setLeaderId(String leaderId) {
1160             boolean changed = !Objects.equal(this.leaderId, leaderId);
1161             this.leaderId = leaderId;
1162             if(leaderId != null) {
1163                 this.leaderAvailable = true;
1164             }
1165             notifyOnShardInitializedCallbacks();
1166
1167             return changed;
1168         }
1169
1170         String getLeaderId() {
1171             return leaderId;
1172         }
1173
1174         void setLeaderAvailable(boolean leaderAvailable) {
1175             this.leaderAvailable = leaderAvailable;
1176         }
1177
1178         short getLeaderVersion() {
1179             return leaderVersion;
1180         }
1181
1182         void setLeaderVersion(short leaderVersion) {
1183             this.leaderVersion = leaderVersion;
1184         }
1185     }
1186
1187     private static class OnShardInitialized {
1188         private final Runnable replyRunnable;
1189         private Cancellable timeoutSchedule;
1190
1191         OnShardInitialized(Runnable replyRunnable) {
1192             this.replyRunnable = replyRunnable;
1193         }
1194
1195         Runnable getReplyRunnable() {
1196             return replyRunnable;
1197         }
1198
1199         Cancellable getTimeoutSchedule() {
1200             return timeoutSchedule;
1201         }
1202
1203         void setTimeoutSchedule(Cancellable timeoutSchedule) {
1204             this.timeoutSchedule = timeoutSchedule;
1205         }
1206     }
1207
1208     private static class OnShardReady extends OnShardInitialized {
1209         OnShardReady(Runnable replyRunnable) {
1210             super(replyRunnable);
1211         }
1212     }
1213
1214     private static class ShardNotInitializedTimeout {
1215         private final ActorRef sender;
1216         private final ShardInformation shardInfo;
1217         private final OnShardInitialized onShardInitialized;
1218
1219         ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
1220             this.sender = sender;
1221             this.shardInfo = shardInfo;
1222             this.onShardInitialized = onShardInitialized;
1223         }
1224
1225         ActorRef getSender() {
1226             return sender;
1227         }
1228
1229         ShardInformation getShardInfo() {
1230             return shardInfo;
1231         }
1232
1233         OnShardInitialized getOnShardInitialized() {
1234             return onShardInitialized;
1235         }
1236     }
1237
1238     /**
1239      * We no longer persist SchemaContextModules but keep this class around for now for backwards
1240      * compatibility so we don't get de-serialization failures on upgrade from Helium.
1241      */
1242     @Deprecated
1243     static class SchemaContextModules implements Serializable {
1244         private static final long serialVersionUID = -8884620101025936590L;
1245
1246         private final Set<String> modules;
1247
1248         SchemaContextModules(Set<String> modules){
1249             this.modules = modules;
1250         }
1251
1252         public Set<String> getModules() {
1253             return modules;
1254         }
1255     }
1256
1257     public static Builder builder() {
1258         return new Builder();
1259     }
1260
1261     public static class Builder {
1262         private ClusterWrapper cluster;
1263         private Configuration configuration;
1264         private DatastoreContextFactory datastoreContextFactory;
1265         private CountDownLatch waitTillReadyCountdownLatch;
1266         private PrimaryShardInfoFutureCache primaryShardInfoCache;
1267         private DatastoreSnapshot restoreFromSnapshot;
1268         private volatile boolean sealed;
1269
1270         protected void checkSealed() {
1271             Preconditions.checkState(!sealed, "Builder is already sealed - further modifications are not allowed");
1272         }
1273
1274         public Builder cluster(ClusterWrapper cluster) {
1275             checkSealed();
1276             this.cluster = cluster;
1277             return this;
1278         }
1279
1280         public Builder configuration(Configuration configuration) {
1281             checkSealed();
1282             this.configuration = configuration;
1283             return this;
1284         }
1285
1286         public Builder datastoreContextFactory(DatastoreContextFactory datastoreContextFactory) {
1287             checkSealed();
1288             this.datastoreContextFactory = datastoreContextFactory;
1289             return this;
1290         }
1291
1292         public Builder waitTillReadyCountdownLatch(CountDownLatch waitTillReadyCountdownLatch) {
1293             checkSealed();
1294             this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
1295             return this;
1296         }
1297
1298         public Builder primaryShardInfoCache(PrimaryShardInfoFutureCache primaryShardInfoCache) {
1299             checkSealed();
1300             this.primaryShardInfoCache = primaryShardInfoCache;
1301             return this;
1302         }
1303
1304         public Builder restoreFromSnapshot(DatastoreSnapshot restoreFromSnapshot) {
1305             checkSealed();
1306             this.restoreFromSnapshot = restoreFromSnapshot;
1307             return this;
1308         }
1309
1310         public Props props() {
1311             sealed = true;
1312             Preconditions.checkNotNull(cluster, "cluster should not be null");
1313             Preconditions.checkNotNull(configuration, "configuration should not be null");
1314             Preconditions.checkNotNull(datastoreContextFactory, "datastoreContextFactory should not be null");
1315             Preconditions.checkNotNull(waitTillReadyCountdownLatch, "waitTillReadyCountdownLatch should not be null");
1316             Preconditions.checkNotNull(primaryShardInfoCache, "primaryShardInfoCache should not be null");
1317             return Props.create(ShardManager.class, this);
1318         }
1319     }
1320 }
1321
1322
1323