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