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