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