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