Bug -3221 : Adding a new DataStoreUnavailableException for external applications.
[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         }
482     }
483
484     private void markMemberAvailable(final String memberName) {
485         for(ShardInformation info : localShards.values()){
486             String leaderId = info.getLeaderId();
487             if(leaderId != null && leaderId.contains(memberName)) {
488                 LOG.debug("Marking Leader {} as available.", leaderId);
489                 info.setLeaderAvailable(true);
490             }
491         }
492     }
493
494     private void onDatastoreContext(DatastoreContext context) {
495         datastoreContext = context;
496         for (ShardInformation info : localShards.values()) {
497             if (info.getActor() != null) {
498                 info.getActor().tell(datastoreContext, getSelf());
499             }
500         }
501     }
502
503     /**
504      * Notifies all the local shards of a change in the schema context
505      *
506      * @param message
507      */
508     private void updateSchemaContext(final Object message) {
509         final SchemaContext schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
510
511         Set<ModuleIdentifier> allModuleIdentifiers = schemaContext.getAllModuleIdentifiers();
512         Set<String> newModules = new HashSet<>(128);
513
514         for(ModuleIdentifier moduleIdentifier : allModuleIdentifiers){
515             String s = moduleIdentifier.getNamespace().toString();
516             newModules.add(s);
517         }
518
519         if(newModules.containsAll(knownModules)) {
520
521             LOG.debug("New SchemaContext has a super set of current knownModules - persisting info");
522
523             knownModules = ImmutableSet.copyOf(newModules);
524
525             dataPersistenceProvider.persist(new SchemaContextModules(newModules), new Procedure<SchemaContextModules>() {
526
527                 @Override
528                 public void apply(SchemaContextModules param) throws Exception {
529                     LOG.debug("Sending new SchemaContext to Shards");
530                     for (ShardInformation info : localShards.values()) {
531                         if (info.getActor() == null) {
532                             info.setActor(newShardActor(schemaContext, info));
533                         } else {
534                             info.getActor().tell(message, getSelf());
535                         }
536                     }
537                 }
538
539             });
540         } else {
541             LOG.debug("Rejecting schema context update - not a super set of previously known modules:\nUPDATE: {}\nKNOWN: {}",
542                     newModules, knownModules);
543         }
544
545     }
546
547     @VisibleForTesting
548     protected ClusterWrapper getCluster() {
549         return cluster;
550     }
551
552     @VisibleForTesting
553     protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
554         return getContext().actorOf(Shard.props(info.getShardId(),
555                 info.getPeerAddresses(), datastoreContext, schemaContext)
556                         .withDispatcher(shardDispatcherPath), info.getShardId().toString());
557     }
558
559     private void findPrimary(FindPrimary message) {
560         LOG.debug("{}: In findPrimary: {}", persistenceId(), message);
561
562         final String shardName = message.getShardName();
563         final boolean canReturnLocalShardState = !(message instanceof RemoteFindPrimary);
564
565         // First see if the there is a local replica for the shard
566         final ShardInformation info = localShards.get(shardName);
567         if (info != null) {
568             sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
569                 @Override
570                 public Object get() {
571                     String primaryPath = info.getSerializedLeaderActor();
572                     Object found = canReturnLocalShardState && info.isLeader() ?
573                             new LocalPrimaryShardFound(primaryPath, info.getLocalShardDataTree().get()) :
574                                 new RemotePrimaryShardFound(primaryPath);
575
576                     if(LOG.isDebugEnabled()) {
577                         LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
578                     }
579
580                     return found;
581                 }
582             });
583
584             return;
585         }
586
587         for(Map.Entry<String, Address> entry: memberNameToAddress.entrySet()) {
588             if(!cluster.getCurrentMemberName().equals(entry.getKey())) {
589                 String path = getShardManagerActorPathBuilder(entry.getValue()).toString();
590
591                 LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}", persistenceId(),
592                         shardName, path);
593
594                 getContext().actorSelection(path).forward(new RemoteFindPrimary(shardName,
595                         message.isWaitUntilReady()), getContext());
596                 return;
597             }
598         }
599
600         LOG.debug("{}: No shard found for {}", persistenceId(), shardName);
601
602         getSender().tell(new PrimaryNotFoundException(
603                 String.format("No primary shard found for %s.", shardName)), getSelf());
604     }
605
606     private StringBuilder getShardManagerActorPathBuilder(Address address) {
607         StringBuilder builder = new StringBuilder();
608         builder.append(address.toString()).append("/user/").append(shardManagerIdentifierString);
609         return builder;
610     }
611
612     private String getShardActorPath(String shardName, String memberName) {
613         Address address = memberNameToAddress.get(memberName);
614         if(address != null) {
615             StringBuilder builder = getShardManagerActorPathBuilder(address);
616             builder.append("/")
617                 .append(getShardIdentifier(memberName, shardName));
618             return builder.toString();
619         }
620         return null;
621     }
622
623     /**
624      * Construct the name of the shard actor given the name of the member on
625      * which the shard resides and the name of the shard
626      *
627      * @param memberName
628      * @param shardName
629      * @return
630      */
631     private ShardIdentifier getShardIdentifier(String memberName, String shardName){
632         return ShardIdentifier.builder().memberName(memberName).shardName(shardName).type(type).build();
633     }
634
635     /**
636      * Create shards that are local to the member on which the ShardManager
637      * runs
638      *
639      */
640     private void createLocalShards() {
641         String memberName = this.cluster.getCurrentMemberName();
642         List<String> memberShardNames =
643             this.configuration.getMemberShardNames(memberName);
644
645         List<String> localShardActorNames = new ArrayList<>();
646         for(String shardName : memberShardNames){
647             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
648             Map<String, String> peerAddresses = getPeerAddresses(shardName);
649             localShardActorNames.add(shardId.toString());
650             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses));
651         }
652
653         mBean = ShardManagerInfo.createShardManagerMBean("shard-manager-" + this.type,
654                     datastoreContext.getDataStoreMXBeanType(), localShardActorNames);
655     }
656
657     /**
658      * Given the name of the shard find the addresses of all it's peers
659      *
660      * @param shardName
661      * @return
662      */
663     private Map<String, String> getPeerAddresses(String shardName){
664
665         Map<String, String> peerAddresses = new HashMap<>();
666
667         List<String> members = this.configuration.getMembersFromShardName(shardName);
668
669         String currentMemberName = this.cluster.getCurrentMemberName();
670
671         for(String memberName : members){
672             if(!currentMemberName.equals(memberName)){
673                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
674                 String path = getShardActorPath(shardName, currentMemberName);
675                 peerAddresses.put(shardId.toString(), path);
676             }
677         }
678         return peerAddresses;
679     }
680
681     @Override
682     public SupervisorStrategy supervisorStrategy() {
683
684         return new OneForOneStrategy(10, Duration.create("1 minute"),
685             new Function<Throwable, SupervisorStrategy.Directive>() {
686                 @Override
687                 public SupervisorStrategy.Directive apply(Throwable t) {
688                     LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
689                     return SupervisorStrategy.resume();
690                 }
691             }
692         );
693
694     }
695
696     @Override
697     public String persistenceId() {
698         return "shard-manager-" + type;
699     }
700
701     @VisibleForTesting
702     Collection<String> getKnownModules() {
703         return knownModules;
704     }
705
706     @VisibleForTesting
707     DataPersistenceProvider getDataPersistenceProvider() {
708         return dataPersistenceProvider;
709     }
710
711     @VisibleForTesting
712     ShardManagerInfoMBean getMBean(){
713         return mBean;
714     }
715
716     @VisibleForTesting
717     protected static class ShardInformation {
718         private final ShardIdentifier shardId;
719         private final String shardName;
720         private ActorRef actor;
721         private ActorPath actorPath;
722         private final Map<String, String> peerAddresses;
723         private Optional<DataTree> localShardDataTree;
724         private boolean leaderAvailable = false;
725
726         // flag that determines if the actor is ready for business
727         private boolean actorInitialized = false;
728
729         private boolean followerSyncStatus = false;
730
731         private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
732         private String role ;
733         private String leaderId;
734
735         private ShardInformation(String shardName, ShardIdentifier shardId,
736                 Map<String, String> peerAddresses) {
737             this.shardName = shardName;
738             this.shardId = shardId;
739             this.peerAddresses = peerAddresses;
740         }
741
742         String getShardName() {
743             return shardName;
744         }
745
746         ActorRef getActor(){
747             return actor;
748         }
749
750         ActorPath getActorPath() {
751             return actorPath;
752         }
753
754         void setActor(ActorRef actor) {
755             this.actor = actor;
756             this.actorPath = actor.path();
757         }
758
759         ShardIdentifier getShardId() {
760             return shardId;
761         }
762
763         void setLocalDataTree(Optional<DataTree> localShardDataTree) {
764             this.localShardDataTree = localShardDataTree;
765         }
766
767         Optional<DataTree> getLocalShardDataTree() {
768             return localShardDataTree;
769         }
770
771         Map<String, String> getPeerAddresses() {
772             return peerAddresses;
773         }
774
775         void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
776             LOG.info("updatePeerAddress for peer {} with address {}", peerId,
777                 peerAddress);
778             if(peerAddresses.containsKey(peerId)){
779                 peerAddresses.put(peerId, peerAddress);
780
781                 if(actor != null) {
782                     if(LOG.isDebugEnabled()) {
783                         LOG.debug("Sending PeerAddressResolved for peer {} with address {} to {}",
784                                 peerId, peerAddress, actor.path());
785                     }
786
787                     actor.tell(new PeerAddressResolved(peerId.toString(), peerAddress), sender);
788                 }
789
790                 notifyOnShardInitializedCallbacks();
791             }
792         }
793
794         boolean isShardReady() {
795             return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
796         }
797
798         boolean isShardReadyWithLeaderId() {
799             return leaderAvailable && isShardReady() && (isLeader() || peerAddresses.get(leaderId) != null);
800         }
801
802         boolean isShardInitialized() {
803             return getActor() != null && actorInitialized;
804         }
805
806         boolean isLeader() {
807             return Objects.equal(leaderId, shardId.toString());
808         }
809
810         String getSerializedLeaderActor() {
811             if(isLeader()) {
812                 return Serialization.serializedActorPath(getActor());
813             } else {
814                 return peerAddresses.get(leaderId);
815             }
816         }
817
818         void setActorInitialized() {
819             LOG.debug("Shard {} is initialized", shardId);
820
821             this.actorInitialized = true;
822
823             notifyOnShardInitializedCallbacks();
824         }
825
826         private void notifyOnShardInitializedCallbacks() {
827             if(onShardInitializedSet.isEmpty()) {
828                 return;
829             }
830
831             boolean ready = isShardReadyWithLeaderId();
832
833             if(LOG.isDebugEnabled()) {
834                 LOG.debug("Shard {} is {} - notifying {} OnShardInitialized callbacks", shardId,
835                         ready ? "ready" : "initialized", onShardInitializedSet.size());
836             }
837
838             Iterator<OnShardInitialized> iter = onShardInitializedSet.iterator();
839             while(iter.hasNext()) {
840                 OnShardInitialized onShardInitialized = iter.next();
841                 if(!(onShardInitialized instanceof OnShardReady) || ready) {
842                     iter.remove();
843                     onShardInitialized.getTimeoutSchedule().cancel();
844                     onShardInitialized.getReplyRunnable().run();
845                 }
846             }
847         }
848
849         void addOnShardInitialized(OnShardInitialized onShardInitialized) {
850             onShardInitializedSet.add(onShardInitialized);
851         }
852
853         void removeOnShardInitialized(OnShardInitialized onShardInitialized) {
854             onShardInitializedSet.remove(onShardInitialized);
855         }
856
857         void setRole(String newRole) {
858             this.role = newRole;
859
860             notifyOnShardInitializedCallbacks();
861         }
862
863         void setFollowerSyncStatus(boolean syncStatus){
864             this.followerSyncStatus = syncStatus;
865         }
866
867         boolean isInSync(){
868             if(RaftState.Follower.name().equals(this.role)){
869                 return followerSyncStatus;
870             } else if(RaftState.Leader.name().equals(this.role)){
871                 return true;
872             }
873
874             return false;
875         }
876
877         boolean setLeaderId(String leaderId) {
878             boolean changed = !Objects.equal(this.leaderId, leaderId);
879             this.leaderId = leaderId;
880             if(leaderId != null) {
881                 this.leaderAvailable = true;
882             }
883             notifyOnShardInitializedCallbacks();
884
885             return changed;
886         }
887
888         public String getLeaderId() {
889             return leaderId;
890         }
891
892         public void setLeaderAvailable(boolean leaderAvailable) {
893             this.leaderAvailable = leaderAvailable;
894         }
895     }
896
897     private static class ShardManagerCreator implements Creator<ShardManager> {
898         private static final long serialVersionUID = 1L;
899
900         final ClusterWrapper cluster;
901         final Configuration configuration;
902         final DatastoreContext datastoreContext;
903         private final CountDownLatch waitTillReadyCountdownLatch;
904         private final PrimaryShardInfoFutureCache primaryShardInfoCache;
905
906         ShardManagerCreator(ClusterWrapper cluster, Configuration configuration, DatastoreContext datastoreContext,
907                 CountDownLatch waitTillReadyCountdownLatch, PrimaryShardInfoFutureCache primaryShardInfoCache) {
908             this.cluster = cluster;
909             this.configuration = configuration;
910             this.datastoreContext = datastoreContext;
911             this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
912             this.primaryShardInfoCache = primaryShardInfoCache;
913         }
914
915         @Override
916         public ShardManager create() throws Exception {
917             return new ShardManager(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch,
918                     primaryShardInfoCache);
919         }
920     }
921
922     private static class OnShardInitialized {
923         private final Runnable replyRunnable;
924         private Cancellable timeoutSchedule;
925
926         OnShardInitialized(Runnable replyRunnable) {
927             this.replyRunnable = replyRunnable;
928         }
929
930         Runnable getReplyRunnable() {
931             return replyRunnable;
932         }
933
934         Cancellable getTimeoutSchedule() {
935             return timeoutSchedule;
936         }
937
938         void setTimeoutSchedule(Cancellable timeoutSchedule) {
939             this.timeoutSchedule = timeoutSchedule;
940         }
941     }
942
943     private static class OnShardReady extends OnShardInitialized {
944         OnShardReady(Runnable replyRunnable) {
945             super(replyRunnable);
946         }
947     }
948
949     private static class ShardNotInitializedTimeout {
950         private final ActorRef sender;
951         private final ShardInformation shardInfo;
952         private final OnShardInitialized onShardInitialized;
953
954         ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
955             this.sender = sender;
956             this.shardInfo = shardInfo;
957             this.onShardInitialized = onShardInitialized;
958         }
959
960         ActorRef getSender() {
961             return sender;
962         }
963
964         ShardInformation getShardInfo() {
965             return shardInfo;
966         }
967
968         OnShardInitialized getOnShardInitialized() {
969             return onShardInitialized;
970         }
971     }
972
973     static class SchemaContextModules implements Serializable {
974         private static final long serialVersionUID = -8884620101025936590L;
975
976         private final Set<String> modules;
977
978         SchemaContextModules(Set<String> modules){
979             this.modules = modules;
980         }
981
982         public Set<String> getModules() {
983             return modules;
984         }
985     }
986 }
987
988
989