Merge "BUG 2852 : Reduce and improve logging in RemoteRpcImplementation"
[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 akka.actor.ActorPath;
12 import akka.actor.ActorRef;
13 import akka.actor.Address;
14 import akka.actor.Cancellable;
15 import akka.actor.OneForOneStrategy;
16 import akka.actor.Props;
17 import akka.actor.SupervisorStrategy;
18 import akka.cluster.ClusterEvent;
19 import akka.japi.Creator;
20 import akka.japi.Function;
21 import akka.japi.Procedure;
22 import akka.persistence.RecoveryCompleted;
23 import akka.persistence.RecoveryFailure;
24 import akka.serialization.Serialization;
25 import com.google.common.annotations.VisibleForTesting;
26 import com.google.common.base.Objects;
27 import com.google.common.base.Preconditions;
28 import com.google.common.base.Strings;
29 import com.google.common.base.Supplier;
30 import com.google.common.collect.ImmutableSet;
31 import com.google.common.collect.Sets;
32 import java.io.Serializable;
33 import java.util.ArrayList;
34 import java.util.Collection;
35 import java.util.Collections;
36 import java.util.HashMap;
37 import java.util.HashSet;
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 org.opendaylight.controller.cluster.DataPersistenceProvider;
44 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
45 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
46 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
47 import org.opendaylight.controller.cluster.datastore.identifiers.ShardManagerIdentifier;
48 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfo;
49 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfoMBean;
50 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
51 import org.opendaylight.controller.cluster.datastore.messages.ActorNotInitialized;
52 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
53 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
54 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
55 import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound;
56 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
57 import org.opendaylight.controller.cluster.datastore.messages.PrimaryFound;
58 import org.opendaylight.controller.cluster.datastore.messages.PrimaryNotFound;
59 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
60 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
61 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
62 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
63 import org.opendaylight.controller.cluster.notifications.RoleChangeNotification;
64 import org.opendaylight.controller.cluster.raft.RaftState;
65 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
66 import org.opendaylight.yangtools.yang.model.api.ModuleIdentifier;
67 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
68 import org.slf4j.Logger;
69 import org.slf4j.LoggerFactory;
70 import scala.concurrent.duration.Duration;
71
72 /**
73  * The ShardManager has the following jobs,
74  * <ul>
75  * <li> Create all the local shard replicas that belong on this cluster member
76  * <li> Find the address of the local shard
77  * <li> Find the primary replica for any given shard
78  * <li> Monitor the cluster members and store their addresses
79  * <ul>
80  */
81 public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
82
83     private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class);
84
85     // Stores a mapping between a member name and the address of the member
86     // Member names look like "member-1", "member-2" etc and are as specified
87     // in configuration
88     private final Map<String, Address> memberNameToAddress = new HashMap<>();
89
90     // Stores a mapping between a shard name and it's corresponding information
91     // Shard names look like inventory, topology etc and are as specified in
92     // configuration
93     private final Map<String, ShardInformation> localShards = new HashMap<>();
94
95     // The type of a ShardManager reflects the type of the datastore itself
96     // A data store could be of type config/operational
97     private final String type;
98
99     private final ClusterWrapper cluster;
100
101     private final Configuration configuration;
102
103     private final String shardDispatcherPath;
104
105     private ShardManagerInfo mBean;
106
107     private DatastoreContext datastoreContext;
108
109     private Collection<String> knownModules = Collections.emptySet();
110
111     private final DataPersistenceProvider dataPersistenceProvider;
112
113     private final CountDownLatch waitTillReadyCountdownLatch;
114
115     /**
116      */
117     protected ShardManager(ClusterWrapper cluster, Configuration configuration,
118             DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
119
120         this.cluster = Preconditions.checkNotNull(cluster, "cluster should not be null");
121         this.configuration = Preconditions.checkNotNull(configuration, "configuration should not be null");
122         this.datastoreContext = datastoreContext;
123         this.dataPersistenceProvider = createDataPersistenceProvider(datastoreContext.isPersistent());
124         this.type = datastoreContext.getDataStoreType();
125         this.shardDispatcherPath =
126                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
127         this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
128
129         // Subscribe this actor to cluster member events
130         cluster.subscribeToMemberEvents(getSelf());
131
132         createLocalShards();
133     }
134
135     protected DataPersistenceProvider createDataPersistenceProvider(boolean persistent) {
136         return (persistent) ? new PersistentDataProvider() : new NonPersistentDataProvider();
137     }
138
139     public static Props props(
140         final ClusterWrapper cluster,
141         final Configuration configuration,
142         final DatastoreContext datastoreContext,
143         final CountDownLatch waitTillReadyCountdownLatch) {
144
145         Preconditions.checkNotNull(cluster, "cluster should not be null");
146         Preconditions.checkNotNull(configuration, "configuration should not be null");
147         Preconditions.checkNotNull(waitTillReadyCountdownLatch, "waitTillReadyCountdownLatch should not be null");
148
149         return Props.create(new ShardManagerCreator(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch));
150     }
151
152     @Override
153     public void postStop() {
154         LOG.info("Stopping ShardManager");
155
156         mBean.unregisterMBean();
157     }
158
159     @Override
160     public void handleCommand(Object message) throws Exception {
161         if (FindPrimary.SERIALIZABLE_CLASS.isInstance(message)) {
162             findPrimary(FindPrimary.fromSerializable(message));
163         } else if(message instanceof FindLocalShard){
164             findLocalShard((FindLocalShard) message);
165         } else if (message instanceof UpdateSchemaContext) {
166             updateSchemaContext(message);
167         } else if(message instanceof ActorInitialized) {
168             onActorInitialized(message);
169         } else if (message instanceof ClusterEvent.MemberUp){
170             memberUp((ClusterEvent.MemberUp) message);
171         } else if(message instanceof ClusterEvent.MemberRemoved) {
172             memberRemoved((ClusterEvent.MemberRemoved) message);
173         } else if(message instanceof ClusterEvent.UnreachableMember) {
174             ignoreMessage(message);
175         } else if(message instanceof DatastoreContext) {
176             onDatastoreContext((DatastoreContext)message);
177         } else if(message instanceof RoleChangeNotification) {
178             onRoleChangeNotification((RoleChangeNotification) message);
179         } else if(message instanceof FollowerInitialSyncUpStatus){
180             onFollowerInitialSyncStatus((FollowerInitialSyncUpStatus) message);
181         } else if(message instanceof ShardNotInitializedTimeout) {
182             onShardNotInitializedTimeout((ShardNotInitializedTimeout)message);
183         } else if(message instanceof LeaderStateChanged) {
184             onLeaderStateChanged((LeaderStateChanged)message);
185         } else {
186             unknownMessage(message);
187         }
188
189     }
190
191     private void onLeaderStateChanged(LeaderStateChanged leaderStateChanged) {
192         LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged);
193
194         ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId());
195         if(shardInformation != null) {
196             shardInformation.setLeaderId(leaderStateChanged.getLeaderId());
197         } else {
198             LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
199         }
200     }
201
202     private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) {
203         ShardInformation shardInfo = message.getShardInfo();
204
205         LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
206                 shardInfo.getShardId());
207
208         shardInfo.removeOnShardInitialized(message.getOnShardInitialized());
209
210         if(!shardInfo.isShardInitialized()) {
211             message.getSender().tell(new ActorNotInitialized(), getSelf());
212         } else {
213             message.getSender().tell(createNoShardLeaderException(shardInfo.shardId), getSelf());
214         }
215     }
216
217     private void onFollowerInitialSyncStatus(FollowerInitialSyncUpStatus status) {
218         LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(),
219                 status.getName(), status.isInitialSyncDone());
220
221         ShardInformation shardInformation = findShardInformation(status.getName());
222
223         if(shardInformation != null) {
224             shardInformation.setFollowerSyncStatus(status.isInitialSyncDone());
225
226             mBean.setSyncStatus(isInSync());
227         }
228
229     }
230
231     private void onRoleChangeNotification(RoleChangeNotification roleChanged) {
232         LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(),
233                 roleChanged.getOldRole(), roleChanged.getNewRole());
234
235         ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
236         if(shardInformation != null) {
237             shardInformation.setRole(roleChanged.getNewRole());
238
239             if (isReady()) {
240                 LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
241                         persistenceId(), type, waitTillReadyCountdownLatch.getCount());
242
243                 waitTillReadyCountdownLatch.countDown();
244             }
245
246             mBean.setSyncStatus(isInSync());
247         }
248     }
249
250
251     private ShardInformation findShardInformation(String memberId) {
252         for(ShardInformation info : localShards.values()){
253             if(info.getShardId().toString().equals(memberId)){
254                 return info;
255             }
256         }
257
258         return null;
259     }
260
261     private boolean isReady() {
262         boolean isReady = true;
263         for (ShardInformation info : localShards.values()) {
264             if(!info.isShardReady()){
265                 isReady = false;
266                 break;
267             }
268         }
269         return isReady;
270     }
271
272     private boolean isInSync(){
273         for (ShardInformation info : localShards.values()) {
274             if(!info.isInSync()){
275                 return false;
276             }
277         }
278         return true;
279     }
280
281     private void onActorInitialized(Object message) {
282         final ActorRef sender = getSender();
283
284         if (sender == null) {
285             return; //why is a non-actor sending this message? Just ignore.
286         }
287
288         String actorName = sender.path().name();
289         //find shard name from actor name; actor name is stringified shardId
290         ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(actorName).build();
291
292         if (shardId.getShardName() == null) {
293             return;
294         }
295
296         markShardAsInitialized(shardId.getShardName());
297     }
298
299     private void markShardAsInitialized(String shardName) {
300         LOG.debug("Initializing shard [{}]", shardName);
301
302         ShardInformation shardInformation = localShards.get(shardName);
303         if (shardInformation != null) {
304             shardInformation.setActorInitialized();
305
306             shardInformation.getActor().tell(new RegisterRoleChangeListener(), self());
307         }
308     }
309
310     @Override
311     protected void handleRecover(Object message) throws Exception {
312         if(dataPersistenceProvider.isRecoveryApplicable()) {
313             if (message instanceof SchemaContextModules) {
314                 SchemaContextModules msg = (SchemaContextModules) message;
315                 knownModules = ImmutableSet.copyOf(msg.getModules());
316             } else if (message instanceof RecoveryFailure) {
317                 RecoveryFailure failure = (RecoveryFailure) message;
318                 LOG.error("Recovery failed", failure.cause());
319             } else if (message instanceof RecoveryCompleted) {
320                 LOG.info("Recovery complete : {}", persistenceId());
321
322                 // Delete all the messages from the akka journal except the last one
323                 deleteMessages(lastSequenceNr() - 1);
324             }
325         } else {
326             if (message instanceof RecoveryCompleted) {
327                 LOG.info("Recovery complete : {}", persistenceId());
328
329                 // Delete all the messages from the akka journal
330                 deleteMessages(lastSequenceNr());
331             }
332         }
333     }
334
335     private void findLocalShard(FindLocalShard message) {
336         final ShardInformation shardInformation = localShards.get(message.getShardName());
337
338         if(shardInformation == null){
339             getSender().tell(new LocalShardNotFound(message.getShardName()), getSelf());
340             return;
341         }
342
343         sendResponse(shardInformation, message.isWaitUntilInitialized(), false, new Supplier<Object>() {
344             @Override
345             public Object get() {
346                 return new LocalShardFound(shardInformation.getActor());
347             }
348         });
349     }
350
351     private void sendResponse(ShardInformation shardInformation, boolean doWait,
352             boolean wantShardReady, final Supplier<Object> messageSupplier) {
353         if (!shardInformation.isShardInitialized() || (wantShardReady && !shardInformation.isShardReadyWithLeaderId())) {
354             if(doWait) {
355                 final ActorRef sender = getSender();
356                 final ActorRef self = self();
357
358                 Runnable replyRunnable = new Runnable() {
359                     @Override
360                     public void run() {
361                         sender.tell(messageSupplier.get(), self);
362                     }
363                 };
364
365                 OnShardInitialized onShardInitialized = wantShardReady ? new OnShardReady(replyRunnable) :
366                     new OnShardInitialized(replyRunnable);
367
368                 shardInformation.addOnShardInitialized(onShardInitialized);
369
370                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
371                         datastoreContext.getShardInitializationTimeout().duration(), getSelf(),
372                         new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
373                         getContext().dispatcher(), getSelf());
374
375                 onShardInitialized.setTimeoutSchedule(timeoutSchedule);
376
377             } else if (!shardInformation.isShardInitialized()) {
378                 getSender().tell(new ActorNotInitialized(), getSelf());
379             } else {
380                 getSender().tell(createNoShardLeaderException(shardInformation.shardId), getSelf());
381             }
382
383             return;
384         }
385
386         getSender().tell(messageSupplier.get(), getSelf());
387     }
388
389     private NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) {
390         return new NoShardLeaderException(String.format(
391                 "Could not find a leader for shard %s. This typically happens when the system is coming up or " +
392                 "recovering and a leader is being elected. Try again later.", shardId));
393     }
394
395     private void memberRemoved(ClusterEvent.MemberRemoved message) {
396         memberNameToAddress.remove(message.member().roles().head());
397     }
398
399     private void memberUp(ClusterEvent.MemberUp message) {
400         String memberName = message.member().roles().head();
401
402         memberNameToAddress.put(memberName, message.member().address());
403
404         for(ShardInformation info : localShards.values()){
405             String shardName = info.getShardName();
406             info.updatePeerAddress(getShardIdentifier(memberName, shardName).toString(),
407                 getShardActorPath(shardName, memberName), getSelf());
408         }
409     }
410
411     private void onDatastoreContext(DatastoreContext context) {
412         datastoreContext = context;
413         for (ShardInformation info : localShards.values()) {
414             if (info.getActor() != null) {
415                 info.getActor().tell(datastoreContext, getSelf());
416             }
417         }
418     }
419
420     /**
421      * Notifies all the local shards of a change in the schema context
422      *
423      * @param message
424      */
425     private void updateSchemaContext(final Object message) {
426         final SchemaContext schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
427
428         Set<ModuleIdentifier> allModuleIdentifiers = schemaContext.getAllModuleIdentifiers();
429         Set<String> newModules = new HashSet<>(128);
430
431         for(ModuleIdentifier moduleIdentifier : allModuleIdentifiers){
432             String s = moduleIdentifier.getNamespace().toString();
433             newModules.add(s);
434         }
435
436         if(newModules.containsAll(knownModules)) {
437
438             LOG.debug("New SchemaContext has a super set of current knownModules - persisting info");
439
440             knownModules = ImmutableSet.copyOf(newModules);
441
442             dataPersistenceProvider.persist(new SchemaContextModules(newModules), new Procedure<SchemaContextModules>() {
443
444                 @Override
445                 public void apply(SchemaContextModules param) throws Exception {
446                     LOG.debug("Sending new SchemaContext to Shards");
447                     for (ShardInformation info : localShards.values()) {
448                         if (info.getActor() == null) {
449                             info.setActor(newShardActor(schemaContext, info));
450                         } else {
451                             info.getActor().tell(message, getSelf());
452                         }
453                     }
454                 }
455
456             });
457         } else {
458             LOG.debug("Rejecting schema context update - not a super set of previously known modules:\nUPDATE: {}\nKNOWN: {}",
459                     newModules, knownModules);
460         }
461
462     }
463
464     @VisibleForTesting
465     protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
466         return getContext().actorOf(Shard.props(info.getShardId(),
467                 info.getPeerAddresses(), datastoreContext, schemaContext)
468                         .withDispatcher(shardDispatcherPath), info.getShardId().toString());
469     }
470
471     private void findPrimary(FindPrimary message) {
472         final String shardName = message.getShardName();
473
474         // First see if the there is a local replica for the shard
475         final ShardInformation info = localShards.get(shardName);
476         if (info != null) {
477             sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
478                 @Override
479                 public Object get() {
480                     Object found = new PrimaryFound(info.getSerializedLeaderActor()).toSerializable();
481
482                     if(LOG.isDebugEnabled()) {
483                         LOG.debug("{}: Found primary for {}: {}", shardName, found);
484                     }
485
486                     return found;
487                 }
488             });
489
490             return;
491         }
492
493         List<String> members = configuration.getMembersFromShardName(shardName);
494
495         if(cluster.getCurrentMemberName() != null) {
496             members.remove(cluster.getCurrentMemberName());
497         }
498
499         /**
500          * FIXME: Instead of sending remote shard actor path back to sender,
501          * forward FindPrimary message to remote shard manager
502          */
503         // There is no way for us to figure out the primary (for now) so assume
504         // that one of the remote nodes is a primary
505         for(String memberName : members) {
506             Address address = memberNameToAddress.get(memberName);
507             if(address != null){
508                 String path =
509                     getShardActorPath(shardName, memberName);
510                 getSender().tell(new PrimaryFound(path).toSerializable(), getSelf());
511                 return;
512             }
513         }
514         getSender().tell(new PrimaryNotFound(shardName).toSerializable(), getSelf());
515     }
516
517     private String getShardActorPath(String shardName, String memberName) {
518         Address address = memberNameToAddress.get(memberName);
519         if(address != null) {
520             StringBuilder builder = new StringBuilder();
521             builder.append(address.toString())
522                 .append("/user/")
523                 .append(ShardManagerIdentifier.builder().type(type).build().toString())
524                 .append("/")
525                 .append(getShardIdentifier(memberName, shardName));
526             return builder.toString();
527         }
528         return null;
529     }
530
531     /**
532      * Construct the name of the shard actor given the name of the member on
533      * which the shard resides and the name of the shard
534      *
535      * @param memberName
536      * @param shardName
537      * @return
538      */
539     private ShardIdentifier getShardIdentifier(String memberName, String shardName){
540         return ShardIdentifier.builder().memberName(memberName).shardName(shardName).type(type).build();
541     }
542
543     /**
544      * Create shards that are local to the member on which the ShardManager
545      * runs
546      *
547      */
548     private void createLocalShards() {
549         String memberName = this.cluster.getCurrentMemberName();
550         List<String> memberShardNames =
551             this.configuration.getMemberShardNames(memberName);
552
553         List<String> localShardActorNames = new ArrayList<>();
554         for(String shardName : memberShardNames){
555             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
556             Map<String, String> peerAddresses = getPeerAddresses(shardName);
557             localShardActorNames.add(shardId.toString());
558             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses));
559         }
560
561         mBean = ShardManagerInfo.createShardManagerMBean("shard-manager-" + this.type,
562                     datastoreContext.getDataStoreMXBeanType(), localShardActorNames);
563     }
564
565     /**
566      * Given the name of the shard find the addresses of all it's peers
567      *
568      * @param shardName
569      * @return
570      */
571     private Map<String, String> getPeerAddresses(String shardName){
572
573         Map<String, String> peerAddresses = new HashMap<>();
574
575         List<String> members = this.configuration.getMembersFromShardName(shardName);
576
577         String currentMemberName = this.cluster.getCurrentMemberName();
578
579         for(String memberName : members){
580             if(!currentMemberName.equals(memberName)){
581                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
582                 String path = getShardActorPath(shardName, currentMemberName);
583                 peerAddresses.put(shardId.toString(), path);
584             }
585         }
586         return peerAddresses;
587     }
588
589     @Override
590     public SupervisorStrategy supervisorStrategy() {
591
592         return new OneForOneStrategy(10, Duration.create("1 minute"),
593             new Function<Throwable, SupervisorStrategy.Directive>() {
594                 @Override
595                 public SupervisorStrategy.Directive apply(Throwable t) {
596                     LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
597                     return SupervisorStrategy.resume();
598                 }
599             }
600         );
601
602     }
603
604     @Override
605     public String persistenceId() {
606         return "shard-manager-" + type;
607     }
608
609     @VisibleForTesting
610     Collection<String> getKnownModules() {
611         return knownModules;
612     }
613
614     @VisibleForTesting
615     DataPersistenceProvider getDataPersistenceProvider() {
616         return dataPersistenceProvider;
617     }
618
619     @VisibleForTesting
620     ShardManagerInfoMBean getMBean(){
621         return mBean;
622     }
623
624     @VisibleForTesting
625     protected static class ShardInformation {
626         private final ShardIdentifier shardId;
627         private final String shardName;
628         private ActorRef actor;
629         private ActorPath actorPath;
630         private final Map<String, String> peerAddresses;
631
632         // flag that determines if the actor is ready for business
633         private boolean actorInitialized = false;
634
635         private boolean followerSyncStatus = false;
636
637         private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
638         private String role ;
639         private String leaderId;
640
641         private ShardInformation(String shardName, ShardIdentifier shardId,
642                 Map<String, String> peerAddresses) {
643             this.shardName = shardName;
644             this.shardId = shardId;
645             this.peerAddresses = peerAddresses;
646         }
647
648         String getShardName() {
649             return shardName;
650         }
651
652         ActorRef getActor(){
653             return actor;
654         }
655
656         ActorPath getActorPath() {
657             return actorPath;
658         }
659
660         void setActor(ActorRef actor) {
661             this.actor = actor;
662             this.actorPath = actor.path();
663         }
664
665         ShardIdentifier getShardId() {
666             return shardId;
667         }
668
669         Map<String, String> getPeerAddresses() {
670             return peerAddresses;
671         }
672
673         void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
674             LOG.info("updatePeerAddress for peer {} with address {}", peerId,
675                 peerAddress);
676             if(peerAddresses.containsKey(peerId)){
677                 peerAddresses.put(peerId, peerAddress);
678
679                 if(actor != null) {
680                     if(LOG.isDebugEnabled()) {
681                         LOG.debug("Sending PeerAddressResolved for peer {} with address {} to {}",
682                                 peerId, peerAddress, actor.path());
683                     }
684
685                     actor.tell(new PeerAddressResolved(peerId.toString(), peerAddress), sender);
686                 }
687
688                 notifyOnShardInitializedCallbacks();
689             }
690         }
691
692         boolean isShardReady() {
693             return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
694         }
695
696         boolean isShardReadyWithLeaderId() {
697             return isShardReady() && (isLeader() || peerAddresses.containsKey(leaderId));
698         }
699
700         boolean isShardInitialized() {
701             return getActor() != null && actorInitialized;
702         }
703
704         boolean isLeader() {
705             return Objects.equal(leaderId, shardId.toString());
706         }
707
708         String getSerializedLeaderActor() {
709             if(isLeader()) {
710                 return Serialization.serializedActorPath(getActor());
711             } else {
712                 return peerAddresses.get(leaderId);
713             }
714         }
715
716         void setActorInitialized() {
717             LOG.debug("Shard {} is initialized", shardId);
718
719             this.actorInitialized = true;
720
721             notifyOnShardInitializedCallbacks();
722         }
723
724         private void notifyOnShardInitializedCallbacks() {
725             if(onShardInitializedSet.isEmpty()) {
726                 return;
727             }
728
729             boolean ready = isShardReadyWithLeaderId();
730
731             if(LOG.isDebugEnabled()) {
732                 LOG.debug("Shard {} is {} - notifying {} OnShardInitialized callbacks", shardId,
733                         ready ? "ready" : "initialized", onShardInitializedSet.size());
734             }
735
736             Iterator<OnShardInitialized> iter = onShardInitializedSet.iterator();
737             while(iter.hasNext()) {
738                 OnShardInitialized onShardInitialized = iter.next();
739                 if(!(onShardInitialized instanceof OnShardReady) || ready) {
740                     iter.remove();
741                     onShardInitialized.getTimeoutSchedule().cancel();
742                     onShardInitialized.getReplyRunnable().run();
743                 }
744             }
745         }
746
747         void addOnShardInitialized(OnShardInitialized onShardInitialized) {
748             onShardInitializedSet.add(onShardInitialized);
749         }
750
751         void removeOnShardInitialized(OnShardInitialized onShardInitialized) {
752             onShardInitializedSet.remove(onShardInitialized);
753         }
754
755         void setRole(String newRole) {
756             this.role = newRole;
757
758             notifyOnShardInitializedCallbacks();
759         }
760
761         void setFollowerSyncStatus(boolean syncStatus){
762             this.followerSyncStatus = syncStatus;
763         }
764
765         boolean isInSync(){
766             if(RaftState.Follower.name().equals(this.role)){
767                 return followerSyncStatus;
768             } else if(RaftState.Leader.name().equals(this.role)){
769                 return true;
770             }
771
772             return false;
773         }
774
775         void setLeaderId(String leaderId) {
776             this.leaderId = leaderId;
777
778             notifyOnShardInitializedCallbacks();
779         }
780     }
781
782     private static class ShardManagerCreator implements Creator<ShardManager> {
783         private static final long serialVersionUID = 1L;
784
785         final ClusterWrapper cluster;
786         final Configuration configuration;
787         final DatastoreContext datastoreContext;
788         private final CountDownLatch waitTillReadyCountdownLatch;
789
790         ShardManagerCreator(ClusterWrapper cluster,
791                             Configuration configuration, DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
792             this.cluster = cluster;
793             this.configuration = configuration;
794             this.datastoreContext = datastoreContext;
795             this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
796         }
797
798         @Override
799         public ShardManager create() throws Exception {
800             return new ShardManager(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch);
801         }
802     }
803
804     private static class OnShardInitialized {
805         private final Runnable replyRunnable;
806         private Cancellable timeoutSchedule;
807
808         OnShardInitialized(Runnable replyRunnable) {
809             this.replyRunnable = replyRunnable;
810         }
811
812         Runnable getReplyRunnable() {
813             return replyRunnable;
814         }
815
816         Cancellable getTimeoutSchedule() {
817             return timeoutSchedule;
818         }
819
820         void setTimeoutSchedule(Cancellable timeoutSchedule) {
821             this.timeoutSchedule = timeoutSchedule;
822         }
823     }
824
825     private static class OnShardReady extends OnShardInitialized {
826         OnShardReady(Runnable replyRunnable) {
827             super(replyRunnable);
828         }
829     }
830
831     private static class ShardNotInitializedTimeout {
832         private final ActorRef sender;
833         private final ShardInformation shardInfo;
834         private final OnShardInitialized onShardInitialized;
835
836         ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
837             this.sender = sender;
838             this.shardInfo = shardInfo;
839             this.onShardInitialized = onShardInitialized;
840         }
841
842         ActorRef getSender() {
843             return sender;
844         }
845
846         ShardInformation getShardInfo() {
847             return shardInfo;
848         }
849
850         OnShardInitialized getOnShardInitialized() {
851             return onShardInitialized;
852         }
853     }
854
855     static class SchemaContextModules implements Serializable {
856         private static final long serialVersionUID = -8884620101025936590L;
857
858         private final Set<String> modules;
859
860         SchemaContextModules(Set<String> modules){
861             this.modules = modules;
862         }
863
864         public Set<String> getModules() {
865             return modules;
866         }
867     }
868 }
869
870
871