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