Merge "Refactor snapshot code"
[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         } else {
201             LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
202         }
203     }
204
205     private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) {
206         ShardInformation shardInfo = message.getShardInfo();
207
208         LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
209                 shardInfo.getShardName());
210
211         shardInfo.removeOnShardInitialized(message.getOnShardInitialized());
212
213         if(!shardInfo.isShardInitialized()) {
214             LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(), shardInfo.getShardName());
215             message.getSender().tell(createNotInitializedException(shardInfo.shardId), getSelf());
216         } else {
217             LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
218             message.getSender().tell(createNoShardLeaderException(shardInfo.shardId), getSelf());
219         }
220     }
221
222     private void onFollowerInitialSyncStatus(FollowerInitialSyncUpStatus status) {
223         LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(),
224                 status.getName(), status.isInitialSyncDone());
225
226         ShardInformation shardInformation = findShardInformation(status.getName());
227
228         if(shardInformation != null) {
229             shardInformation.setFollowerSyncStatus(status.isInitialSyncDone());
230
231             mBean.setSyncStatus(isInSync());
232         }
233
234     }
235
236     private void onRoleChangeNotification(RoleChangeNotification roleChanged) {
237         LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(),
238                 roleChanged.getOldRole(), roleChanged.getNewRole());
239
240         ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
241         if(shardInformation != null) {
242             shardInformation.setRole(roleChanged.getNewRole());
243
244             if (isReady()) {
245                 LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
246                         persistenceId(), type, waitTillReadyCountdownLatch.getCount());
247
248                 waitTillReadyCountdownLatch.countDown();
249             }
250
251             mBean.setSyncStatus(isInSync());
252         }
253     }
254
255
256     private ShardInformation findShardInformation(String memberId) {
257         for(ShardInformation info : localShards.values()){
258             if(info.getShardId().toString().equals(memberId)){
259                 return info;
260             }
261         }
262
263         return null;
264     }
265
266     private boolean isReady() {
267         boolean isReady = true;
268         for (ShardInformation info : localShards.values()) {
269             if(!info.isShardReady()){
270                 isReady = false;
271                 break;
272             }
273         }
274         return isReady;
275     }
276
277     private boolean isInSync(){
278         for (ShardInformation info : localShards.values()) {
279             if(!info.isInSync()){
280                 return false;
281             }
282         }
283         return true;
284     }
285
286     private void onActorInitialized(Object message) {
287         final ActorRef sender = getSender();
288
289         if (sender == null) {
290             return; //why is a non-actor sending this message? Just ignore.
291         }
292
293         String actorName = sender.path().name();
294         //find shard name from actor name; actor name is stringified shardId
295         ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(actorName).build();
296
297         if (shardId.getShardName() == null) {
298             return;
299         }
300
301         markShardAsInitialized(shardId.getShardName());
302     }
303
304     private void markShardAsInitialized(String shardName) {
305         LOG.debug("{}: Initializing shard [{}]", persistenceId(), shardName);
306
307         ShardInformation shardInformation = localShards.get(shardName);
308         if (shardInformation != null) {
309             shardInformation.setActorInitialized();
310
311             shardInformation.getActor().tell(new RegisterRoleChangeListener(), self());
312         }
313     }
314
315     @Override
316     protected void handleRecover(Object message) throws Exception {
317         if(dataPersistenceProvider.isRecoveryApplicable()) {
318             if (message instanceof SchemaContextModules) {
319                 SchemaContextModules msg = (SchemaContextModules) message;
320                 knownModules = ImmutableSet.copyOf(msg.getModules());
321             } else if (message instanceof RecoveryFailure) {
322                 RecoveryFailure failure = (RecoveryFailure) message;
323                 LOG.error("Recovery failed", failure.cause());
324             } else if (message instanceof RecoveryCompleted) {
325                 LOG.info("Recovery complete : {}", persistenceId());
326
327                 // Delete all the messages from the akka journal except the last one
328                 deleteMessages(lastSequenceNr() - 1);
329             }
330         } else {
331             if (message instanceof RecoveryCompleted) {
332                 LOG.info("Recovery complete : {}", persistenceId());
333
334                 // Delete all the messages from the akka journal
335                 deleteMessages(lastSequenceNr());
336             }
337         }
338     }
339
340     private void findLocalShard(FindLocalShard message) {
341         final ShardInformation shardInformation = localShards.get(message.getShardName());
342
343         if(shardInformation == null){
344             getSender().tell(new LocalShardNotFound(message.getShardName()), getSelf());
345             return;
346         }
347
348         sendResponse(shardInformation, message.isWaitUntilInitialized(), false, new Supplier<Object>() {
349             @Override
350             public Object get() {
351                 return new LocalShardFound(shardInformation.getActor());
352             }
353         });
354     }
355
356     private void sendResponse(ShardInformation shardInformation, boolean doWait,
357             boolean wantShardReady, final Supplier<Object> messageSupplier) {
358         if (!shardInformation.isShardInitialized() || (wantShardReady && !shardInformation.isShardReadyWithLeaderId())) {
359             if(doWait) {
360                 final ActorRef sender = getSender();
361                 final ActorRef self = self();
362
363                 Runnable replyRunnable = new Runnable() {
364                     @Override
365                     public void run() {
366                         sender.tell(messageSupplier.get(), self);
367                     }
368                 };
369
370                 OnShardInitialized onShardInitialized = wantShardReady ? new OnShardReady(replyRunnable) :
371                     new OnShardInitialized(replyRunnable);
372
373                 shardInformation.addOnShardInitialized(onShardInitialized);
374
375                 LOG.debug("{}: Scheduling timer to wait for shard {}", persistenceId(), shardInformation.getShardName());
376
377                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
378                         datastoreContext.getShardInitializationTimeout().duration(), getSelf(),
379                         new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
380                         getContext().dispatcher(), getSelf());
381
382                 onShardInitialized.setTimeoutSchedule(timeoutSchedule);
383
384             } else if (!shardInformation.isShardInitialized()) {
385                 LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(),
386                         shardInformation.getShardName());
387                 getSender().tell(createNotInitializedException(shardInformation.shardId), getSelf());
388             } else {
389                 LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
390                         shardInformation.getShardName());
391                 getSender().tell(createNoShardLeaderException(shardInformation.shardId), getSelf());
392             }
393
394             return;
395         }
396
397         getSender().tell(messageSupplier.get(), getSelf());
398     }
399
400     private NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) {
401         return new NoShardLeaderException(String.format(
402                 "Could not find a leader for shard %s. This typically happens when the system is coming up or " +
403                 "recovering and a leader is being elected. Try again later.", shardId));
404     }
405
406     private NotInitializedException createNotInitializedException(ShardIdentifier shardId) {
407         return new NotInitializedException(String.format(
408                 "Found primary shard %s but it's not initialized yet. Please try again later", shardId));
409     }
410
411     private void memberRemoved(ClusterEvent.MemberRemoved message) {
412         String memberName = message.member().roles().head();
413
414         LOG.debug("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName,
415                 message.member().address());
416
417         memberNameToAddress.remove(message.member().roles().head());
418     }
419
420     private void memberUp(ClusterEvent.MemberUp message) {
421         String memberName = message.member().roles().head();
422
423         LOG.debug("{}: Received MemberUp: memberName: {}, address: {}", persistenceId(), memberName,
424                 message.member().address());
425
426         memberNameToAddress.put(memberName, message.member().address());
427
428         for(ShardInformation info : localShards.values()){
429             String shardName = info.getShardName();
430             info.updatePeerAddress(getShardIdentifier(memberName, shardName).toString(),
431                 getShardActorPath(shardName, memberName), getSelf());
432         }
433     }
434
435     private void onDatastoreContext(DatastoreContext context) {
436         datastoreContext = context;
437         for (ShardInformation info : localShards.values()) {
438             if (info.getActor() != null) {
439                 info.getActor().tell(datastoreContext, getSelf());
440             }
441         }
442     }
443
444     /**
445      * Notifies all the local shards of a change in the schema context
446      *
447      * @param message
448      */
449     private void updateSchemaContext(final Object message) {
450         final SchemaContext schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
451
452         Set<ModuleIdentifier> allModuleIdentifiers = schemaContext.getAllModuleIdentifiers();
453         Set<String> newModules = new HashSet<>(128);
454
455         for(ModuleIdentifier moduleIdentifier : allModuleIdentifiers){
456             String s = moduleIdentifier.getNamespace().toString();
457             newModules.add(s);
458         }
459
460         if(newModules.containsAll(knownModules)) {
461
462             LOG.debug("New SchemaContext has a super set of current knownModules - persisting info");
463
464             knownModules = ImmutableSet.copyOf(newModules);
465
466             dataPersistenceProvider.persist(new SchemaContextModules(newModules), new Procedure<SchemaContextModules>() {
467
468                 @Override
469                 public void apply(SchemaContextModules param) throws Exception {
470                     LOG.debug("Sending new SchemaContext to Shards");
471                     for (ShardInformation info : localShards.values()) {
472                         if (info.getActor() == null) {
473                             info.setActor(newShardActor(schemaContext, info));
474                         } else {
475                             info.getActor().tell(message, getSelf());
476                         }
477                     }
478                 }
479
480             });
481         } else {
482             LOG.debug("Rejecting schema context update - not a super set of previously known modules:\nUPDATE: {}\nKNOWN: {}",
483                     newModules, knownModules);
484         }
485
486     }
487
488     @VisibleForTesting
489     protected ClusterWrapper getCluster() {
490         return cluster;
491     }
492
493     @VisibleForTesting
494     protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
495         return getContext().actorOf(Shard.props(info.getShardId(),
496                 info.getPeerAddresses(), datastoreContext, schemaContext)
497                         .withDispatcher(shardDispatcherPath), info.getShardId().toString());
498     }
499
500     private void findPrimary(FindPrimary message) {
501         LOG.debug("{}: In findPrimary: {}", persistenceId(), message);
502
503         final String shardName = message.getShardName();
504
505         // First see if the there is a local replica for the shard
506         final ShardInformation info = localShards.get(shardName);
507         if (info != null) {
508             sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
509                 @Override
510                 public Object get() {
511                     Object found = new PrimaryFound(info.getSerializedLeaderActor());
512
513                     if(LOG.isDebugEnabled()) {
514                         LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
515                     }
516
517                     return found;
518                 }
519             });
520
521             return;
522         }
523
524         for(Map.Entry<String, Address> entry: memberNameToAddress.entrySet()) {
525             if(!cluster.getCurrentMemberName().equals(entry.getKey())) {
526                 String path = getShardManagerActorPathBuilder(entry.getValue()).toString();
527
528                 LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}", persistenceId(),
529                         shardName, path);
530
531                 getContext().actorSelection(path).forward(message, getContext());
532                 return;
533             }
534         }
535
536         LOG.debug("{}: No shard found for {}", persistenceId(), shardName);
537
538         getSender().tell(new PrimaryNotFoundException(
539                 String.format("No primary shard found for %s.", shardName)), getSelf());
540     }
541
542     private StringBuilder getShardManagerActorPathBuilder(Address address) {
543         StringBuilder builder = new StringBuilder();
544         builder.append(address.toString()).append("/user/").append(shardManagerIdentifierString);
545         return builder;
546     }
547
548     private String getShardActorPath(String shardName, String memberName) {
549         Address address = memberNameToAddress.get(memberName);
550         if(address != null) {
551             StringBuilder builder = getShardManagerActorPathBuilder(address);
552             builder.append("/")
553                 .append(getShardIdentifier(memberName, shardName));
554             return builder.toString();
555         }
556         return null;
557     }
558
559     /**
560      * Construct the name of the shard actor given the name of the member on
561      * which the shard resides and the name of the shard
562      *
563      * @param memberName
564      * @param shardName
565      * @return
566      */
567     private ShardIdentifier getShardIdentifier(String memberName, String shardName){
568         return ShardIdentifier.builder().memberName(memberName).shardName(shardName).type(type).build();
569     }
570
571     /**
572      * Create shards that are local to the member on which the ShardManager
573      * runs
574      *
575      */
576     private void createLocalShards() {
577         String memberName = this.cluster.getCurrentMemberName();
578         List<String> memberShardNames =
579             this.configuration.getMemberShardNames(memberName);
580
581         List<String> localShardActorNames = new ArrayList<>();
582         for(String shardName : memberShardNames){
583             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
584             Map<String, String> peerAddresses = getPeerAddresses(shardName);
585             localShardActorNames.add(shardId.toString());
586             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses));
587         }
588
589         mBean = ShardManagerInfo.createShardManagerMBean("shard-manager-" + this.type,
590                     datastoreContext.getDataStoreMXBeanType(), localShardActorNames);
591     }
592
593     /**
594      * Given the name of the shard find the addresses of all it's peers
595      *
596      * @param shardName
597      * @return
598      */
599     private Map<String, String> getPeerAddresses(String shardName){
600
601         Map<String, String> peerAddresses = new HashMap<>();
602
603         List<String> members = this.configuration.getMembersFromShardName(shardName);
604
605         String currentMemberName = this.cluster.getCurrentMemberName();
606
607         for(String memberName : members){
608             if(!currentMemberName.equals(memberName)){
609                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
610                 String path = getShardActorPath(shardName, currentMemberName);
611                 peerAddresses.put(shardId.toString(), path);
612             }
613         }
614         return peerAddresses;
615     }
616
617     @Override
618     public SupervisorStrategy supervisorStrategy() {
619
620         return new OneForOneStrategy(10, Duration.create("1 minute"),
621             new Function<Throwable, SupervisorStrategy.Directive>() {
622                 @Override
623                 public SupervisorStrategy.Directive apply(Throwable t) {
624                     LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
625                     return SupervisorStrategy.resume();
626                 }
627             }
628         );
629
630     }
631
632     @Override
633     public String persistenceId() {
634         return "shard-manager-" + type;
635     }
636
637     @VisibleForTesting
638     Collection<String> getKnownModules() {
639         return knownModules;
640     }
641
642     @VisibleForTesting
643     DataPersistenceProvider getDataPersistenceProvider() {
644         return dataPersistenceProvider;
645     }
646
647     @VisibleForTesting
648     ShardManagerInfoMBean getMBean(){
649         return mBean;
650     }
651
652     @VisibleForTesting
653     protected static class ShardInformation {
654         private final ShardIdentifier shardId;
655         private final String shardName;
656         private ActorRef actor;
657         private ActorPath actorPath;
658         private final Map<String, String> peerAddresses;
659
660         // flag that determines if the actor is ready for business
661         private boolean actorInitialized = false;
662
663         private boolean followerSyncStatus = false;
664
665         private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
666         private String role ;
667         private String leaderId;
668
669         private ShardInformation(String shardName, ShardIdentifier shardId,
670                 Map<String, String> peerAddresses) {
671             this.shardName = shardName;
672             this.shardId = shardId;
673             this.peerAddresses = peerAddresses;
674         }
675
676         String getShardName() {
677             return shardName;
678         }
679
680         ActorRef getActor(){
681             return actor;
682         }
683
684         ActorPath getActorPath() {
685             return actorPath;
686         }
687
688         void setActor(ActorRef actor) {
689             this.actor = actor;
690             this.actorPath = actor.path();
691         }
692
693         ShardIdentifier getShardId() {
694             return shardId;
695         }
696
697         Map<String, String> getPeerAddresses() {
698             return peerAddresses;
699         }
700
701         void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
702             LOG.info("updatePeerAddress for peer {} with address {}", peerId,
703                 peerAddress);
704             if(peerAddresses.containsKey(peerId)){
705                 peerAddresses.put(peerId, peerAddress);
706
707                 if(actor != null) {
708                     if(LOG.isDebugEnabled()) {
709                         LOG.debug("Sending PeerAddressResolved for peer {} with address {} to {}",
710                                 peerId, peerAddress, actor.path());
711                     }
712
713                     actor.tell(new PeerAddressResolved(peerId.toString(), peerAddress), sender);
714                 }
715
716                 notifyOnShardInitializedCallbacks();
717             }
718         }
719
720         boolean isShardReady() {
721             return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
722         }
723
724         boolean isShardReadyWithLeaderId() {
725             return isShardReady() && (isLeader() || peerAddresses.containsKey(leaderId));
726         }
727
728         boolean isShardInitialized() {
729             return getActor() != null && actorInitialized;
730         }
731
732         boolean isLeader() {
733             return Objects.equal(leaderId, shardId.toString());
734         }
735
736         String getSerializedLeaderActor() {
737             if(isLeader()) {
738                 return Serialization.serializedActorPath(getActor());
739             } else {
740                 return peerAddresses.get(leaderId);
741             }
742         }
743
744         void setActorInitialized() {
745             LOG.debug("Shard {} is initialized", shardId);
746
747             this.actorInitialized = true;
748
749             notifyOnShardInitializedCallbacks();
750         }
751
752         private void notifyOnShardInitializedCallbacks() {
753             if(onShardInitializedSet.isEmpty()) {
754                 return;
755             }
756
757             boolean ready = isShardReadyWithLeaderId();
758
759             if(LOG.isDebugEnabled()) {
760                 LOG.debug("Shard {} is {} - notifying {} OnShardInitialized callbacks", shardId,
761                         ready ? "ready" : "initialized", onShardInitializedSet.size());
762             }
763
764             Iterator<OnShardInitialized> iter = onShardInitializedSet.iterator();
765             while(iter.hasNext()) {
766                 OnShardInitialized onShardInitialized = iter.next();
767                 if(!(onShardInitialized instanceof OnShardReady) || ready) {
768                     iter.remove();
769                     onShardInitialized.getTimeoutSchedule().cancel();
770                     onShardInitialized.getReplyRunnable().run();
771                 }
772             }
773         }
774
775         void addOnShardInitialized(OnShardInitialized onShardInitialized) {
776             onShardInitializedSet.add(onShardInitialized);
777         }
778
779         void removeOnShardInitialized(OnShardInitialized onShardInitialized) {
780             onShardInitializedSet.remove(onShardInitialized);
781         }
782
783         void setRole(String newRole) {
784             this.role = newRole;
785
786             notifyOnShardInitializedCallbacks();
787         }
788
789         void setFollowerSyncStatus(boolean syncStatus){
790             this.followerSyncStatus = syncStatus;
791         }
792
793         boolean isInSync(){
794             if(RaftState.Follower.name().equals(this.role)){
795                 return followerSyncStatus;
796             } else if(RaftState.Leader.name().equals(this.role)){
797                 return true;
798             }
799
800             return false;
801         }
802
803         void setLeaderId(String leaderId) {
804             this.leaderId = leaderId;
805
806             notifyOnShardInitializedCallbacks();
807         }
808     }
809
810     private static class ShardManagerCreator implements Creator<ShardManager> {
811         private static final long serialVersionUID = 1L;
812
813         final ClusterWrapper cluster;
814         final Configuration configuration;
815         final DatastoreContext datastoreContext;
816         private final CountDownLatch waitTillReadyCountdownLatch;
817
818         ShardManagerCreator(ClusterWrapper cluster,
819                             Configuration configuration, DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
820             this.cluster = cluster;
821             this.configuration = configuration;
822             this.datastoreContext = datastoreContext;
823             this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
824         }
825
826         @Override
827         public ShardManager create() throws Exception {
828             return new ShardManager(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch);
829         }
830     }
831
832     private static class OnShardInitialized {
833         private final Runnable replyRunnable;
834         private Cancellable timeoutSchedule;
835
836         OnShardInitialized(Runnable replyRunnable) {
837             this.replyRunnable = replyRunnable;
838         }
839
840         Runnable getReplyRunnable() {
841             return replyRunnable;
842         }
843
844         Cancellable getTimeoutSchedule() {
845             return timeoutSchedule;
846         }
847
848         void setTimeoutSchedule(Cancellable timeoutSchedule) {
849             this.timeoutSchedule = timeoutSchedule;
850         }
851     }
852
853     private static class OnShardReady extends OnShardInitialized {
854         OnShardReady(Runnable replyRunnable) {
855             super(replyRunnable);
856         }
857     }
858
859     private static class ShardNotInitializedTimeout {
860         private final ActorRef sender;
861         private final ShardInformation shardInfo;
862         private final OnShardInitialized onShardInitialized;
863
864         ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
865             this.sender = sender;
866             this.shardInfo = shardInfo;
867             this.onShardInitialized = onShardInitialized;
868         }
869
870         ActorRef getSender() {
871             return sender;
872         }
873
874         ShardInformation getShardInfo() {
875             return shardInfo;
876         }
877
878         OnShardInitialized getOnShardInitialized() {
879             return onShardInitialized;
880         }
881     }
882
883     static class SchemaContextModules implements Serializable {
884         private static final long serialVersionUID = -8884620101025936590L;
885
886         private final Set<String> modules;
887
888         SchemaContextModules(Set<String> modules){
889             this.modules = modules;
890         }
891
892         public Set<String> getModules() {
893             return modules;
894         }
895     }
896 }
897
898
899