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