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