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