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