814f117f3155ddbe4fed0223331f2403ad2b42fb
[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 static akka.pattern.Patterns.ask;
12 import akka.actor.ActorPath;
13 import akka.actor.ActorRef;
14 import akka.actor.Address;
15 import akka.actor.Cancellable;
16 import akka.actor.OneForOneStrategy;
17 import akka.actor.PoisonPill;
18 import akka.actor.Props;
19 import akka.actor.SupervisorStrategy;
20 import akka.cluster.ClusterEvent;
21 import akka.dispatch.OnComplete;
22 import akka.japi.Function;
23 import akka.persistence.RecoveryCompleted;
24 import akka.persistence.SaveSnapshotFailure;
25 import akka.persistence.SaveSnapshotSuccess;
26 import akka.persistence.SnapshotOffer;
27 import akka.serialization.Serialization;
28 import akka.util.Timeout;
29 import com.google.common.annotations.VisibleForTesting;
30 import com.google.common.base.Objects;
31 import com.google.common.base.Optional;
32 import com.google.common.base.Preconditions;
33 import com.google.common.base.Strings;
34 import com.google.common.base.Supplier;
35 import com.google.common.collect.Sets;
36 import java.io.Serializable;
37 import java.util.ArrayList;
38 import java.util.Collection;
39 import java.util.Collections;
40 import java.util.HashMap;
41 import java.util.HashSet;
42 import java.util.Iterator;
43 import java.util.List;
44 import java.util.Map;
45 import java.util.Set;
46 import java.util.concurrent.CountDownLatch;
47 import java.util.concurrent.TimeUnit;
48 import java.util.concurrent.TimeoutException;
49 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
50 import org.opendaylight.controller.cluster.datastore.config.Configuration;
51 import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration;
52 import org.opendaylight.controller.cluster.datastore.exceptions.AlreadyExistsException;
53 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
54 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
55 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
56 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
57 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfo;
58 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfoMBean;
59 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
60 import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica;
61 import org.opendaylight.controller.cluster.datastore.messages.CreateShard;
62 import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot;
63 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
64 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
65 import org.opendaylight.controller.cluster.datastore.messages.LocalPrimaryShardFound;
66 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
67 import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound;
68 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
69 import org.opendaylight.controller.cluster.datastore.messages.PeerDown;
70 import org.opendaylight.controller.cluster.datastore.messages.PeerUp;
71 import org.opendaylight.controller.cluster.datastore.messages.RemoteFindPrimary;
72 import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound;
73 import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica;
74 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
75 import org.opendaylight.controller.cluster.datastore.messages.SwitchShardBehavior;
76 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
77 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
78 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
79 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
80 import org.opendaylight.controller.cluster.notifications.RoleChangeNotification;
81 import org.opendaylight.controller.cluster.raft.RaftState;
82 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
83 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
84 import org.opendaylight.controller.cluster.raft.client.messages.GetSnapshot;
85 import org.opendaylight.controller.cluster.raft.messages.AddServer;
86 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
87 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
88 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
89 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
90 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
91 import org.slf4j.Logger;
92 import org.slf4j.LoggerFactory;
93 import scala.concurrent.Future;
94 import scala.concurrent.duration.Duration;
95 import scala.concurrent.duration.FiniteDuration;
96
97 /**
98  * The ShardManager has the following jobs,
99  * <ul>
100  * <li> Create all the local shard replicas that belong on this cluster member
101  * <li> Find the address of the local shard
102  * <li> Find the primary replica for any given shard
103  * <li> Monitor the cluster members and store their addresses
104  * <ul>
105  */
106 public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
107
108     private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class);
109
110     // Stores a mapping between a shard name and it's corresponding information
111     // Shard names look like inventory, topology etc and are as specified in
112     // configuration
113     private final Map<String, ShardInformation> localShards = new HashMap<>();
114
115     // The type of a ShardManager reflects the type of the datastore itself
116     // A data store could be of type config/operational
117     private final String type;
118
119     private final ClusterWrapper cluster;
120
121     private final Configuration configuration;
122
123     private final String shardDispatcherPath;
124
125     private final ShardManagerInfo mBean;
126
127     private DatastoreContextFactory datastoreContextFactory;
128
129     private final CountDownLatch waitTillReadyCountdownLatch;
130
131     private final PrimaryShardInfoFutureCache primaryShardInfoCache;
132
133     private final ShardPeerAddressResolver peerAddressResolver;
134
135     private SchemaContext schemaContext;
136
137     private DatastoreSnapshot restoreFromSnapshot;
138
139     private final Set<String> shardReplicaOperationsInProgress = new HashSet<>();
140
141     private final String id;
142
143     /**
144      */
145     protected ShardManager(Builder builder) {
146
147         this.cluster = builder.cluster;
148         this.configuration = builder.configuration;
149         this.datastoreContextFactory = builder.datastoreContextFactory;
150         this.type = builder.datastoreContextFactory.getBaseDatastoreContext().getDataStoreType();
151         this.shardDispatcherPath =
152                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
153         this.waitTillReadyCountdownLatch = builder.waitTillReadyCountdownLatch;
154         this.primaryShardInfoCache = builder.primaryShardInfoCache;
155         this.restoreFromSnapshot = builder.restoreFromSnapshot;
156
157         id = "shard-manager-" + type;
158
159         peerAddressResolver = new ShardPeerAddressResolver(type, cluster.getCurrentMemberName());
160
161         // Subscribe this actor to cluster member events
162         cluster.subscribeToMemberEvents(getSelf());
163
164         List<String> localShardActorNames = new ArrayList<>();
165         mBean = ShardManagerInfo.createShardManagerMBean(cluster.getCurrentMemberName(),
166                 "shard-manager-" + this.type,
167                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType(),
168                 localShardActorNames);
169         mBean.setShardManager(this);
170     }
171
172     @Override
173     public void postStop() {
174         LOG.info("Stopping ShardManager");
175
176         mBean.unregisterMBean();
177     }
178
179     @Override
180     public void handleCommand(Object message) throws Exception {
181         if (message  instanceof FindPrimary) {
182             findPrimary((FindPrimary)message);
183         } else if(message instanceof FindLocalShard){
184             findLocalShard((FindLocalShard) message);
185         } else if (message instanceof UpdateSchemaContext) {
186             updateSchemaContext(message);
187         } else if(message instanceof ActorInitialized) {
188             onActorInitialized(message);
189         } else if (message instanceof ClusterEvent.MemberUp){
190             memberUp((ClusterEvent.MemberUp) message);
191         } else if (message instanceof ClusterEvent.MemberExited){
192             memberExited((ClusterEvent.MemberExited) message);
193         } else if(message instanceof ClusterEvent.MemberRemoved) {
194             memberRemoved((ClusterEvent.MemberRemoved) message);
195         } else if(message instanceof ClusterEvent.UnreachableMember) {
196             memberUnreachable((ClusterEvent.UnreachableMember)message);
197         } else if(message instanceof ClusterEvent.ReachableMember) {
198             memberReachable((ClusterEvent.ReachableMember) message);
199         } else if(message instanceof DatastoreContextFactory) {
200             onDatastoreContextFactory((DatastoreContextFactory)message);
201         } else if(message instanceof RoleChangeNotification) {
202             onRoleChangeNotification((RoleChangeNotification) message);
203         } else if(message instanceof FollowerInitialSyncUpStatus){
204             onFollowerInitialSyncStatus((FollowerInitialSyncUpStatus) message);
205         } else if(message instanceof ShardNotInitializedTimeout) {
206             onShardNotInitializedTimeout((ShardNotInitializedTimeout)message);
207         } else if(message instanceof ShardLeaderStateChanged) {
208             onLeaderStateChanged((ShardLeaderStateChanged) message);
209         } else if(message instanceof SwitchShardBehavior){
210             onSwitchShardBehavior((SwitchShardBehavior) message);
211         } else if(message instanceof CreateShard) {
212             onCreateShard((CreateShard)message);
213         } else if(message instanceof AddShardReplica){
214             onAddShardReplica((AddShardReplica)message);
215         } else if(message instanceof ForwardedAddServerReply) {
216             ForwardedAddServerReply msg = (ForwardedAddServerReply)message;
217             onAddServerReply(msg.shardInfo, msg.addServerReply, getSender(), msg.leaderPath,
218                     msg.removeShardOnFailure);
219         } else if(message instanceof ForwardedAddServerFailure) {
220             ForwardedAddServerFailure msg = (ForwardedAddServerFailure)message;
221             onAddServerFailure(msg.shardName, msg.failureMessage, msg.failure, getSender(), msg.removeShardOnFailure);
222         } else if(message instanceof ForwardedAddServerPrimaryShardFound) {
223             ForwardedAddServerPrimaryShardFound msg = (ForwardedAddServerPrimaryShardFound)message;
224             addShard(msg.shardName, msg.primaryFound, getSender());
225         } else if(message instanceof RemoveShardReplica){
226             onRemoveShardReplica((RemoveShardReplica)message);
227         } else if(message instanceof GetSnapshot) {
228             onGetSnapshot();
229         } else if (message instanceof SaveSnapshotSuccess) {
230             LOG.debug ("{} saved ShardManager snapshot successfully", persistenceId());
231         } else if (message instanceof SaveSnapshotFailure) {
232             LOG.error ("{}: SaveSnapshotFailure received for saving snapshot of shards",
233                 persistenceId(), ((SaveSnapshotFailure)message).cause());
234         } else {
235             unknownMessage(message);
236         }
237     }
238
239     private void onGetSnapshot() {
240         LOG.debug("{}: onGetSnapshot", persistenceId());
241
242         List<String> notInitialized = null;
243         for(ShardInformation shardInfo: localShards.values()) {
244             if(!shardInfo.isShardInitialized()) {
245                 if(notInitialized == null) {
246                     notInitialized = new ArrayList<>();
247                 }
248
249                 notInitialized.add(shardInfo.getShardName());
250             }
251         }
252
253         if(notInitialized != null) {
254             getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(String.format(
255                     "%d shard(s) %s are not initialized", notInitialized.size(), notInitialized))), getSelf());
256             return;
257         }
258
259         byte[] shardManagerSnapshot = null;
260         ActorRef replyActor = getContext().actorOf(ShardManagerGetSnapshotReplyActor.props(
261                 new ArrayList<>(localShards.keySet()), type, shardManagerSnapshot , getSender(), persistenceId(),
262                 datastoreContextFactory.getBaseDatastoreContext().getShardInitializationTimeout().duration()));
263
264         for(ShardInformation shardInfo: localShards.values()) {
265             shardInfo.getActor().tell(GetSnapshot.INSTANCE, replyActor);
266         }
267     }
268
269     private void onCreateShard(CreateShard createShard) {
270         Object reply;
271         try {
272             String shardName = createShard.getModuleShardConfig().getShardName();
273             if(localShards.containsKey(shardName)) {
274                 reply = new akka.actor.Status.Success(String.format("Shard with name %s already exists", shardName));
275             } else {
276                 doCreateShard(createShard);
277                 reply = new akka.actor.Status.Success(null);
278             }
279         } catch (Exception e) {
280             LOG.error("onCreateShard failed", e);
281             reply = new akka.actor.Status.Failure(e);
282         }
283
284         if(getSender() != null && !getContext().system().deadLetters().equals(getSender())) {
285             getSender().tell(reply, getSelf());
286         }
287     }
288
289     private void doCreateShard(CreateShard createShard) {
290         ModuleShardConfiguration moduleShardConfig = createShard.getModuleShardConfig();
291         String shardName = moduleShardConfig.getShardName();
292
293         configuration.addModuleShardConfiguration(moduleShardConfig);
294
295         DatastoreContext shardDatastoreContext = createShard.getDatastoreContext();
296         if(shardDatastoreContext == null) {
297             shardDatastoreContext = newShardDatastoreContext(shardName);
298         } else {
299             shardDatastoreContext = DatastoreContext.newBuilderFrom(shardDatastoreContext).shardPeerAddressResolver(
300                     peerAddressResolver).build();
301         }
302
303         ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
304
305         Map<String, String> peerAddresses;
306         boolean isActiveMember;
307         if(configuration.getMembersFromShardName(shardName).contains(cluster.getCurrentMemberName())) {
308             peerAddresses = getPeerAddresses(shardName);
309             isActiveMember = true;
310         } else {
311             // The local member is not in the given shard member configuration. In this case we'll create
312             // the shard with no peers and with elections disabled so it stays as follower. A
313             // subsequent AddServer request will be needed to make it an active member.
314             isActiveMember = false;
315             peerAddresses = Collections.emptyMap();
316             shardDatastoreContext = DatastoreContext.newBuilderFrom(shardDatastoreContext).
317                     customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()).build();
318         }
319
320         LOG.debug("onCreateShard: shardId: {}, memberNames: {}. peerAddresses: {}", shardId,
321                 moduleShardConfig.getShardMemberNames(), peerAddresses);
322
323         ShardInformation info = new ShardInformation(shardName, shardId, peerAddresses,
324                 shardDatastoreContext, createShard.getShardBuilder(), peerAddressResolver);
325         info.setActiveMember(isActiveMember);
326         localShards.put(info.getShardName(), info);
327
328         mBean.addLocalShard(shardId.toString());
329
330         if(schemaContext != null) {
331             info.setActor(newShardActor(schemaContext, info));
332         }
333     }
334
335     private DatastoreContext.Builder newShardDatastoreContextBuilder(String shardName) {
336         return DatastoreContext.newBuilderFrom(datastoreContextFactory.getShardDatastoreContext(shardName)).
337                 shardPeerAddressResolver(peerAddressResolver);
338     }
339
340     private DatastoreContext newShardDatastoreContext(String shardName) {
341         return newShardDatastoreContextBuilder(shardName).build();
342     }
343
344     private void checkReady(){
345         if (isReadyWithLeaderId()) {
346             LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
347                     persistenceId(), type, waitTillReadyCountdownLatch.getCount());
348
349             waitTillReadyCountdownLatch.countDown();
350         }
351     }
352
353     private void onLeaderStateChanged(ShardLeaderStateChanged leaderStateChanged) {
354         LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged);
355
356         ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId());
357         if(shardInformation != null) {
358             shardInformation.setLocalDataTree(leaderStateChanged.getLocalShardDataTree());
359             shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion());
360             if(shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) {
361                 primaryShardInfoCache.remove(shardInformation.getShardName());
362             }
363
364             checkReady();
365         } else {
366             LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
367         }
368     }
369
370     private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) {
371         ShardInformation shardInfo = message.getShardInfo();
372
373         LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
374                 shardInfo.getShardName());
375
376         shardInfo.removeOnShardInitialized(message.getOnShardInitialized());
377
378         if(!shardInfo.isShardInitialized()) {
379             LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(), shardInfo.getShardName());
380             message.getSender().tell(createNotInitializedException(shardInfo.shardId), getSelf());
381         } else {
382             LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
383             message.getSender().tell(createNoShardLeaderException(shardInfo.shardId), getSelf());
384         }
385     }
386
387     private void onFollowerInitialSyncStatus(FollowerInitialSyncUpStatus status) {
388         LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(),
389                 status.getName(), status.isInitialSyncDone());
390
391         ShardInformation shardInformation = findShardInformation(status.getName());
392
393         if(shardInformation != null) {
394             shardInformation.setFollowerSyncStatus(status.isInitialSyncDone());
395
396             mBean.setSyncStatus(isInSync());
397         }
398
399     }
400
401     private void onRoleChangeNotification(RoleChangeNotification roleChanged) {
402         LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(),
403                 roleChanged.getOldRole(), roleChanged.getNewRole());
404
405         ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
406         if(shardInformation != null) {
407             shardInformation.setRole(roleChanged.getNewRole());
408             checkReady();
409             mBean.setSyncStatus(isInSync());
410         }
411     }
412
413
414     private ShardInformation findShardInformation(String memberId) {
415         for(ShardInformation info : localShards.values()){
416             if(info.getShardId().toString().equals(memberId)){
417                 return info;
418             }
419         }
420
421         return null;
422     }
423
424     private boolean isReadyWithLeaderId() {
425         boolean isReady = true;
426         for (ShardInformation info : localShards.values()) {
427             if(!info.isShardReadyWithLeaderId()){
428                 isReady = false;
429                 break;
430             }
431         }
432         return isReady;
433     }
434
435     private boolean isInSync(){
436         for (ShardInformation info : localShards.values()) {
437             if(!info.isInSync()){
438                 return false;
439             }
440         }
441         return true;
442     }
443
444     private void onActorInitialized(Object message) {
445         final ActorRef sender = getSender();
446
447         if (sender == null) {
448             return; //why is a non-actor sending this message? Just ignore.
449         }
450
451         String actorName = sender.path().name();
452         //find shard name from actor name; actor name is stringified shardId
453         ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(actorName).build();
454
455         if (shardId.getShardName() == null) {
456             return;
457         }
458
459         markShardAsInitialized(shardId.getShardName());
460     }
461
462     private void markShardAsInitialized(String shardName) {
463         LOG.debug("{}: Initializing shard [{}]", persistenceId(), shardName);
464
465         ShardInformation shardInformation = localShards.get(shardName);
466         if (shardInformation != null) {
467             shardInformation.setActorInitialized();
468
469             shardInformation.getActor().tell(new RegisterRoleChangeListener(), self());
470         }
471     }
472
473     @Override
474     protected void handleRecover(Object message) throws Exception {
475         if (message instanceof RecoveryCompleted) {
476             LOG.info("Recovery complete : {}", persistenceId());
477
478             // We no longer persist SchemaContext modules so delete all the prior messages from the akka
479             // journal on upgrade from Helium.
480             deleteMessages(lastSequenceNr());
481             createLocalShards();
482         } else if (message instanceof SnapshotOffer) {
483             handleShardRecovery((SnapshotOffer) message);
484         }
485     }
486
487     private void findLocalShard(FindLocalShard message) {
488         final ShardInformation shardInformation = localShards.get(message.getShardName());
489
490         if(shardInformation == null){
491             getSender().tell(new LocalShardNotFound(message.getShardName()), getSelf());
492             return;
493         }
494
495         sendResponse(shardInformation, message.isWaitUntilInitialized(), false, new Supplier<Object>() {
496             @Override
497             public Object get() {
498                 return new LocalShardFound(shardInformation.getActor());
499             }
500         });
501     }
502
503     private void sendResponse(ShardInformation shardInformation, boolean doWait,
504             boolean wantShardReady, final Supplier<Object> messageSupplier) {
505         if (!shardInformation.isShardInitialized() || (wantShardReady && !shardInformation.isShardReadyWithLeaderId())) {
506             if(doWait) {
507                 final ActorRef sender = getSender();
508                 final ActorRef self = self();
509
510                 Runnable replyRunnable = new Runnable() {
511                     @Override
512                     public void run() {
513                         sender.tell(messageSupplier.get(), self);
514                     }
515                 };
516
517                 OnShardInitialized onShardInitialized = wantShardReady ? new OnShardReady(replyRunnable) :
518                     new OnShardInitialized(replyRunnable);
519
520                 shardInformation.addOnShardInitialized(onShardInitialized);
521
522                 FiniteDuration timeout = shardInformation.getDatastoreContext().getShardInitializationTimeout().duration();
523                 if(shardInformation.isShardInitialized()) {
524                     // If the shard is already initialized then we'll wait enough time for the shard to
525                     // elect a leader, ie 2 times the election timeout.
526                     timeout = FiniteDuration.create(shardInformation.getDatastoreContext().getShardRaftConfig()
527                             .getElectionTimeOutInterval().toMillis() * 2, TimeUnit.MILLISECONDS);
528                 }
529
530                 LOG.debug("{}: Scheduling {} ms timer to wait for shard {}", persistenceId(), timeout.toMillis(),
531                         shardInformation.getShardName());
532
533                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
534                         timeout, getSelf(),
535                         new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
536                         getContext().dispatcher(), getSelf());
537
538                 onShardInitialized.setTimeoutSchedule(timeoutSchedule);
539
540             } else if (!shardInformation.isShardInitialized()) {
541                 LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(),
542                         shardInformation.getShardName());
543                 getSender().tell(createNotInitializedException(shardInformation.shardId), getSelf());
544             } else {
545                 LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
546                         shardInformation.getShardName());
547                 getSender().tell(createNoShardLeaderException(shardInformation.shardId), getSelf());
548             }
549
550             return;
551         }
552
553         getSender().tell(messageSupplier.get(), getSelf());
554     }
555
556     private static NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) {
557         return new NoShardLeaderException(null, shardId.toString());
558     }
559
560     private static NotInitializedException createNotInitializedException(ShardIdentifier shardId) {
561         return new NotInitializedException(String.format(
562                 "Found primary shard %s but it's not initialized yet. Please try again later", shardId));
563     }
564
565     private void memberRemoved(ClusterEvent.MemberRemoved message) {
566         String memberName = message.member().roles().head();
567
568         LOG.debug("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName,
569                 message.member().address());
570
571         peerAddressResolver.removePeerAddress(memberName);
572
573         for(ShardInformation info : localShards.values()){
574             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
575         }
576     }
577
578     private void memberExited(ClusterEvent.MemberExited message) {
579         String memberName = message.member().roles().head();
580
581         LOG.debug("{}: Received MemberExited: memberName: {}, address: {}", persistenceId(), memberName,
582                 message.member().address());
583
584         peerAddressResolver.removePeerAddress(memberName);
585
586         for(ShardInformation info : localShards.values()){
587             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
588         }
589     }
590
591     private void memberUp(ClusterEvent.MemberUp message) {
592         String memberName = message.member().roles().head();
593
594         LOG.debug("{}: Received MemberUp: memberName: {}, address: {}", persistenceId(), memberName,
595                 message.member().address());
596
597         addPeerAddress(memberName, message.member().address());
598
599         checkReady();
600     }
601
602     private void addPeerAddress(String memberName, Address address) {
603         peerAddressResolver.addPeerAddress(memberName, address);
604
605         for(ShardInformation info : localShards.values()){
606             String shardName = info.getShardName();
607             String peerId = getShardIdentifier(memberName, shardName).toString();
608             info.updatePeerAddress(peerId, peerAddressResolver.getShardActorAddress(shardName, memberName), getSelf());
609
610             info.peerUp(memberName, peerId, getSelf());
611         }
612     }
613
614     private void memberReachable(ClusterEvent.ReachableMember message) {
615         String memberName = message.member().roles().head();
616         LOG.debug("Received ReachableMember: memberName {}, address: {}", memberName, message.member().address());
617
618         addPeerAddress(memberName, message.member().address());
619
620         markMemberAvailable(memberName);
621     }
622
623     private void memberUnreachable(ClusterEvent.UnreachableMember message) {
624         String memberName = message.member().roles().head();
625         LOG.debug("Received UnreachableMember: memberName {}, address: {}", memberName, message.member().address());
626
627         markMemberUnavailable(memberName);
628     }
629
630     private void markMemberUnavailable(final String memberName) {
631         for(ShardInformation info : localShards.values()){
632             String leaderId = info.getLeaderId();
633             if(leaderId != null && leaderId.contains(memberName)) {
634                 LOG.debug("Marking Leader {} as unavailable.", leaderId);
635                 info.setLeaderAvailable(false);
636
637                 primaryShardInfoCache.remove(info.getShardName());
638             }
639
640             info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
641         }
642     }
643
644     private void markMemberAvailable(final String memberName) {
645         for(ShardInformation info : localShards.values()){
646             String leaderId = info.getLeaderId();
647             if(leaderId != null && leaderId.contains(memberName)) {
648                 LOG.debug("Marking Leader {} as available.", leaderId);
649                 info.setLeaderAvailable(true);
650             }
651
652             info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
653         }
654     }
655
656     private void onDatastoreContextFactory(DatastoreContextFactory factory) {
657         datastoreContextFactory = factory;
658         for (ShardInformation info : localShards.values()) {
659             info.setDatastoreContext(newShardDatastoreContext(info.getShardName()), getSelf());
660         }
661     }
662
663     private void onSwitchShardBehavior(SwitchShardBehavior message) {
664         ShardIdentifier identifier = ShardIdentifier.builder().fromShardIdString(message.getShardName()).build();
665
666         ShardInformation shardInformation = localShards.get(identifier.getShardName());
667
668         if(shardInformation != null && shardInformation.getActor() != null) {
669             shardInformation.getActor().tell(
670                     new SwitchBehavior(RaftState.valueOf(message.getNewState()), message.getTerm()), getSelf());
671         } else {
672             LOG.warn("Could not switch the behavior of shard {} to {} - shard is not yet available",
673                     message.getShardName(), message.getNewState());
674         }
675     }
676
677     /**
678      * Notifies all the local shards of a change in the schema context
679      *
680      * @param message
681      */
682     private void updateSchemaContext(final Object message) {
683         schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
684
685         LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getAllModuleIdentifiers().size());
686
687         for (ShardInformation info : localShards.values()) {
688             if (info.getActor() == null) {
689                 LOG.debug("Creating Shard {}", info.getShardId());
690                 info.setActor(newShardActor(schemaContext, info));
691             } else {
692                 info.getActor().tell(message, getSelf());
693             }
694         }
695     }
696
697     @VisibleForTesting
698     protected ClusterWrapper getCluster() {
699         return cluster;
700     }
701
702     @VisibleForTesting
703     protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
704         return getContext().actorOf(info.newProps(schemaContext)
705                 .withDispatcher(shardDispatcherPath), info.getShardId().toString());
706     }
707
708     private void findPrimary(FindPrimary message) {
709         LOG.debug("{}: In findPrimary: {}", persistenceId(), message);
710
711         final String shardName = message.getShardName();
712         final boolean canReturnLocalShardState = !(message instanceof RemoteFindPrimary);
713
714         // First see if the there is a local replica for the shard
715         final ShardInformation info = localShards.get(shardName);
716         if (info != null && info.isActiveMember()) {
717             sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
718                 @Override
719                 public Object get() {
720                     String primaryPath = info.getSerializedLeaderActor();
721                     Object found = canReturnLocalShardState && info.isLeader() ?
722                             new LocalPrimaryShardFound(primaryPath, info.getLocalShardDataTree().get()) :
723                                 new RemotePrimaryShardFound(primaryPath, info.getLeaderVersion());
724
725                             if(LOG.isDebugEnabled()) {
726                                 LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
727                             }
728
729                             return found;
730                 }
731             });
732
733             return;
734         }
735
736         for(String address: peerAddressResolver.getShardManagerPeerActorAddresses()) {
737             LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}", persistenceId(),
738                     shardName, address);
739
740             getContext().actorSelection(address).forward(new RemoteFindPrimary(shardName,
741                     message.isWaitUntilReady()), getContext());
742             return;
743         }
744
745         LOG.debug("{}: No shard found for {}", persistenceId(), shardName);
746
747         getSender().tell(new PrimaryNotFoundException(
748                 String.format("No primary shard found for %s.", shardName)), getSelf());
749     }
750
751     /**
752      * Construct the name of the shard actor given the name of the member on
753      * which the shard resides and the name of the shard
754      *
755      * @param memberName
756      * @param shardName
757      * @return
758      */
759     private ShardIdentifier getShardIdentifier(String memberName, String shardName){
760         return peerAddressResolver.getShardIdentifier(memberName, shardName);
761     }
762
763     /**
764      * Create shards that are local to the member on which the ShardManager
765      * runs
766      *
767      */
768     private void createLocalShards() {
769         String memberName = this.cluster.getCurrentMemberName();
770         Collection<String> memberShardNames = this.configuration.getMemberShardNames(memberName);
771
772         Map<String, DatastoreSnapshot.ShardSnapshot> shardSnapshots = new HashMap<>();
773         if(restoreFromSnapshot != null)
774         {
775             for(DatastoreSnapshot.ShardSnapshot snapshot: restoreFromSnapshot.getShardSnapshots()) {
776                 shardSnapshots.put(snapshot.getName(), snapshot);
777             }
778         }
779
780         restoreFromSnapshot = null; // null out to GC
781
782         for(String shardName : memberShardNames){
783             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
784             Map<String, String> peerAddresses = getPeerAddresses(shardName);
785             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses,
786                     newShardDatastoreContext(shardName), Shard.builder().restoreFromSnapshot(
787                         shardSnapshots.get(shardName)), peerAddressResolver));
788             mBean.addLocalShard(shardId.toString());
789         }
790     }
791
792     /**
793      * Given the name of the shard find the addresses of all it's peers
794      *
795      * @param shardName
796      */
797     private Map<String, String> getPeerAddresses(String shardName) {
798         Collection<String> members = configuration.getMembersFromShardName(shardName);
799         Map<String, String> peerAddresses = new HashMap<>();
800
801         String currentMemberName = this.cluster.getCurrentMemberName();
802
803         for(String memberName : members) {
804             if(!currentMemberName.equals(memberName)) {
805                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
806                 String address = peerAddressResolver.getShardActorAddress(shardName, memberName);
807                 peerAddresses.put(shardId.toString(), address);
808             }
809         }
810         return peerAddresses;
811     }
812
813     @Override
814     public SupervisorStrategy supervisorStrategy() {
815
816         return new OneForOneStrategy(10, Duration.create("1 minute"),
817                 new Function<Throwable, SupervisorStrategy.Directive>() {
818             @Override
819             public SupervisorStrategy.Directive apply(Throwable t) {
820                 LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
821                 return SupervisorStrategy.resume();
822             }
823         }
824                 );
825
826     }
827
828     @Override
829     public String persistenceId() {
830         return id;
831     }
832
833     @VisibleForTesting
834     ShardManagerInfoMBean getMBean(){
835         return mBean;
836     }
837
838     private boolean isShardReplicaOperationInProgress(final String shardName, final ActorRef sender) {
839         if (shardReplicaOperationsInProgress.contains(shardName)) {
840             String msg = String.format("A shard replica operation for %s is already in progress", shardName);
841             LOG.debug ("{}: {}", persistenceId(), msg);
842             sender.tell(new akka.actor.Status.Failure(new IllegalStateException(msg)), getSelf());
843             return true;
844         }
845
846         return false;
847     }
848
849     private void onAddShardReplica (AddShardReplica shardReplicaMsg) {
850         final String shardName = shardReplicaMsg.getShardName();
851
852         LOG.debug("{}: onAddShardReplica: {}", persistenceId(), shardReplicaMsg);
853
854         // verify the shard with the specified name is present in the cluster configuration
855         if (!(this.configuration.isShardConfigured(shardName))) {
856             String msg = String.format("No module configuration exists for shard %s", shardName);
857             LOG.debug ("{}: {}", persistenceId(), msg);
858             getSender().tell(new akka.actor.Status.Failure(new IllegalArgumentException(msg)), getSelf());
859             return;
860         }
861
862         // Create the localShard
863         if (schemaContext == null) {
864             String msg = String.format(
865                   "No SchemaContext is available in order to create a local shard instance for %s", shardName);
866             LOG.debug ("{}: {}", persistenceId(), msg);
867             getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(msg)), getSelf());
868             return;
869         }
870
871         Timeout findPrimaryTimeout = new Timeout(datastoreContextFactory.getBaseDatastoreContext().
872                 getShardInitializationTimeout().duration().$times(2));
873
874         final ActorRef sender = getSender();
875         Future<Object> futureObj = ask(getSelf(), new FindPrimary(shardName, true), findPrimaryTimeout);
876         futureObj.onComplete(new OnComplete<Object>() {
877             @Override
878             public void onComplete(Throwable failure, Object response) {
879                 if (failure != null) {
880                     LOG.debug ("{}: Received failure from FindPrimary for shard {}", persistenceId(), shardName, failure);
881                     sender.tell(new akka.actor.Status.Failure(new RuntimeException(
882                         String.format("Failed to find leader for shard %s", shardName), failure)), getSelf());
883                 } else {
884                     if(response instanceof RemotePrimaryShardFound) {
885                         self().tell(new ForwardedAddServerPrimaryShardFound(shardName,
886                                 (RemotePrimaryShardFound)response), sender);
887                     } else if(response instanceof LocalPrimaryShardFound) {
888                         sendLocalReplicaAlreadyExistsReply(shardName, sender);
889                     } else {
890                         String msg = String.format("Failed to find leader for shard %s: received response: %s",
891                                 shardName, response);
892                         LOG.debug ("{}: {}", persistenceId(), msg);
893                         sender.tell(new akka.actor.Status.Failure(response instanceof Throwable ? (Throwable)response :
894                             new RuntimeException(msg)), getSelf());
895                     }
896                 }
897             }
898         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
899     }
900
901     private void sendLocalReplicaAlreadyExistsReply(String shardName, ActorRef sender) {
902         String msg = String.format("Local shard %s already exists", shardName);
903         LOG.debug ("{}: {}", persistenceId(), msg);
904         sender.tell(new akka.actor.Status.Failure(new AlreadyExistsException(msg)), getSelf());
905     }
906
907     private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
908         if(isShardReplicaOperationInProgress(shardName, sender)) {
909             return;
910         }
911
912         shardReplicaOperationsInProgress.add(shardName);
913
914         final ShardInformation shardInfo;
915         final boolean removeShardOnFailure;
916         ShardInformation existingShardInfo = localShards.get(shardName);
917         if(existingShardInfo == null) {
918             removeShardOnFailure = true;
919             ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
920
921             DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName).customRaftPolicyImplementation(
922                     DisableElectionsRaftPolicy.class.getName()).build();
923
924             shardInfo = new ShardInformation(shardName, shardId, getPeerAddresses(shardName), datastoreContext,
925                     Shard.builder(), peerAddressResolver);
926             shardInfo.setActiveMember(false);
927             localShards.put(shardName, shardInfo);
928             shardInfo.setActor(newShardActor(schemaContext, shardInfo));
929         } else {
930             removeShardOnFailure = false;
931             shardInfo = existingShardInfo;
932         }
933
934         String localShardAddress = peerAddressResolver.getShardActorAddress(shardName, cluster.getCurrentMemberName());
935
936         //inform ShardLeader to add this shard as a replica by sending an AddServer message
937         LOG.debug ("{}: Sending AddServer message to peer {} for shard {}", persistenceId(),
938                 response.getPrimaryPath(), shardInfo.getShardId());
939
940         Timeout addServerTimeout = new Timeout(shardInfo.getDatastoreContext().getShardLeaderElectionTimeout().
941                 duration());
942         Future<Object> futureObj = ask(getContext().actorSelection(response.getPrimaryPath()),
943             new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout);
944
945         futureObj.onComplete(new OnComplete<Object>() {
946             @Override
947             public void onComplete(Throwable failure, Object addServerResponse) {
948                 if (failure != null) {
949                     LOG.debug ("{}: AddServer request to {} for {} failed", persistenceId(),
950                             response.getPrimaryPath(), shardName, failure);
951
952                     String msg = String.format("AddServer request to leader %s for shard %s failed",
953                             response.getPrimaryPath(), shardName);
954                     self().tell(new ForwardedAddServerFailure(shardName, msg, failure, removeShardOnFailure), sender);
955                 } else {
956                     self().tell(new ForwardedAddServerReply(shardInfo, (AddServerReply)addServerResponse,
957                             response.getPrimaryPath(), removeShardOnFailure), sender);
958                 }
959             }
960         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
961     }
962
963     private void onAddServerFailure(String shardName, String message, Throwable failure, ActorRef sender,
964             boolean removeShardOnFailure) {
965         shardReplicaOperationsInProgress.remove(shardName);
966
967         if(removeShardOnFailure) {
968             ShardInformation shardInfo = localShards.remove(shardName);
969             if (shardInfo.getActor() != null) {
970                 shardInfo.getActor().tell(PoisonPill.getInstance(), getSelf());
971             }
972         }
973
974         sender.tell(new akka.actor.Status.Failure(message == null ? failure :
975             new RuntimeException(message, failure)), getSelf());
976     }
977
978     private void onAddServerReply(ShardInformation shardInfo, AddServerReply replyMsg, ActorRef sender,
979             String leaderPath, boolean removeShardOnFailure) {
980         String shardName = shardInfo.getShardName();
981         shardReplicaOperationsInProgress.remove(shardName);
982
983         LOG.debug ("{}: Received {} for shard {} from leader {}", persistenceId(), replyMsg, shardName, leaderPath);
984
985         if (replyMsg.getStatus() == ServerChangeStatus.OK) {
986             LOG.debug ("{}: Leader shard successfully added the replica shard {}", persistenceId(), shardName);
987
988             // Make the local shard voting capable
989             shardInfo.setDatastoreContext(newShardDatastoreContext(shardName), getSelf());
990             shardInfo.setActiveMember(true);
991             persistShardList();
992
993             mBean.addLocalShard(shardInfo.getShardId().toString());
994             sender.tell(new akka.actor.Status.Success(null), getSelf());
995         } else if(replyMsg.getStatus() == ServerChangeStatus.ALREADY_EXISTS) {
996             sendLocalReplicaAlreadyExistsReply(shardName, sender);
997         } else {
998             LOG.warn ("{}: Leader failed to add shard replica {} with status {}",
999                     persistenceId(), shardName, replyMsg.getStatus());
1000
1001             Exception failure;
1002             switch (replyMsg.getStatus()) {
1003                 case TIMEOUT:
1004                     failure = new TimeoutException(String.format(
1005                             "The shard leader %s timed out trying to replicate the initial data to the new shard %s." +
1006                             "Possible causes - there was a problem replicating the data or shard leadership changed while replicating the shard data",
1007                             leaderPath, shardName));
1008                     break;
1009                 case NO_LEADER:
1010                     failure = createNoShardLeaderException(shardInfo.getShardId());
1011                     break;
1012                 default :
1013                     failure = new RuntimeException(String.format(
1014                             "AddServer request to leader %s for shard %s failed with status %s",
1015                             leaderPath, shardName, replyMsg.getStatus()));
1016             }
1017
1018             onAddServerFailure(shardName, null, failure, sender, removeShardOnFailure);
1019         }
1020     }
1021
1022     private void onRemoveShardReplica (RemoveShardReplica shardReplicaMsg) {
1023         String shardName = shardReplicaMsg.getShardName();
1024
1025         // verify the local shard replica is available in the controller node
1026         if (!localShards.containsKey(shardName)) {
1027             String msg = String.format("Local shard %s does not", shardName);
1028             LOG.debug ("{}: {}", persistenceId(), msg);
1029             getSender().tell(new akka.actor.Status.Failure(new IllegalArgumentException(msg)), getSelf());
1030             return;
1031         }
1032         // call RemoveShard for the shardName
1033         getSender().tell(new akka.actor.Status.Success(true), getSelf());
1034         return;
1035     }
1036
1037     private void persistShardList() {
1038         List<String> shardList = new ArrayList<>(localShards.keySet());
1039         for (ShardInformation shardInfo : localShards.values()) {
1040             if (!shardInfo.isActiveMember()) {
1041                 shardList.remove(shardInfo.getShardName());
1042             }
1043         }
1044         LOG.debug ("{}: persisting the shard list {}", persistenceId(), shardList);
1045         saveSnapshot(new ShardManagerSnapshot(shardList));
1046     }
1047
1048     private void handleShardRecovery(SnapshotOffer offer) {
1049         LOG.debug ("{}: in handleShardRecovery", persistenceId());
1050         ShardManagerSnapshot snapshot = (ShardManagerSnapshot)offer.snapshot();
1051         String currentMember = cluster.getCurrentMemberName();
1052         Set<String> configuredShardList =
1053             new HashSet<>(configuration.getMemberShardNames(currentMember));
1054         for (String shard : snapshot.getShardList()) {
1055             if (!configuredShardList.contains(shard)) {
1056                 // add the current member as a replica for the shard
1057                 LOG.debug ("{}: adding shard {}", persistenceId(), shard);
1058                 configuration.addMemberReplicaForShard(shard, currentMember);
1059             } else {
1060                 configuredShardList.remove(shard);
1061             }
1062         }
1063         for (String shard : configuredShardList) {
1064             // remove the member as a replica for the shard
1065             LOG.debug ("{}: removing shard {}", persistenceId(), shard);
1066             configuration.removeMemberReplicaForShard(shard, currentMember);
1067         }
1068     }
1069
1070     private static class ForwardedAddServerPrimaryShardFound {
1071         String shardName;
1072         RemotePrimaryShardFound primaryFound;
1073
1074         ForwardedAddServerPrimaryShardFound(String shardName, RemotePrimaryShardFound primaryFound) {
1075             this.shardName = shardName;
1076             this.primaryFound = primaryFound;
1077         }
1078     }
1079
1080     private static class ForwardedAddServerReply {
1081         ShardInformation shardInfo;
1082         AddServerReply addServerReply;
1083         String leaderPath;
1084         boolean removeShardOnFailure;
1085
1086         ForwardedAddServerReply(ShardInformation shardInfo, AddServerReply addServerReply, String leaderPath,
1087                 boolean removeShardOnFailure) {
1088             this.shardInfo = shardInfo;
1089             this.addServerReply = addServerReply;
1090             this.leaderPath = leaderPath;
1091             this.removeShardOnFailure = removeShardOnFailure;
1092         }
1093     }
1094
1095     private static class ForwardedAddServerFailure {
1096         String shardName;
1097         String failureMessage;
1098         Throwable failure;
1099         boolean removeShardOnFailure;
1100
1101         ForwardedAddServerFailure(String shardName, String failureMessage, Throwable failure,
1102                 boolean removeShardOnFailure) {
1103             this.shardName = shardName;
1104             this.failureMessage = failureMessage;
1105             this.failure = failure;
1106             this.removeShardOnFailure = removeShardOnFailure;
1107         }
1108     }
1109
1110     @VisibleForTesting
1111     protected static class ShardInformation {
1112         private final ShardIdentifier shardId;
1113         private final String shardName;
1114         private ActorRef actor;
1115         private ActorPath actorPath;
1116         private final Map<String, String> initialPeerAddresses;
1117         private Optional<DataTree> localShardDataTree;
1118         private boolean leaderAvailable = false;
1119
1120         // flag that determines if the actor is ready for business
1121         private boolean actorInitialized = false;
1122
1123         private boolean followerSyncStatus = false;
1124
1125         private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
1126         private String role ;
1127         private String leaderId;
1128         private short leaderVersion;
1129
1130         private DatastoreContext datastoreContext;
1131         private Shard.AbstractBuilder<?, ?> builder;
1132         private final ShardPeerAddressResolver addressResolver;
1133         private boolean isActiveMember = true;
1134
1135         private ShardInformation(String shardName, ShardIdentifier shardId,
1136                 Map<String, String> initialPeerAddresses, DatastoreContext datastoreContext,
1137                 Shard.AbstractBuilder<?, ?> builder, ShardPeerAddressResolver addressResolver) {
1138             this.shardName = shardName;
1139             this.shardId = shardId;
1140             this.initialPeerAddresses = initialPeerAddresses;
1141             this.datastoreContext = datastoreContext;
1142             this.builder = builder;
1143             this.addressResolver = addressResolver;
1144         }
1145
1146         Props newProps(SchemaContext schemaContext) {
1147             Preconditions.checkNotNull(builder);
1148             Props props = builder.id(shardId).peerAddresses(initialPeerAddresses).datastoreContext(datastoreContext).
1149                     schemaContext(schemaContext).props();
1150             builder = null;
1151             return props;
1152         }
1153
1154         String getShardName() {
1155             return shardName;
1156         }
1157
1158         ActorRef getActor(){
1159             return actor;
1160         }
1161
1162         ActorPath getActorPath() {
1163             return actorPath;
1164         }
1165
1166         void setActor(ActorRef actor) {
1167             this.actor = actor;
1168             this.actorPath = actor.path();
1169         }
1170
1171         ShardIdentifier getShardId() {
1172             return shardId;
1173         }
1174
1175         void setLocalDataTree(Optional<DataTree> localShardDataTree) {
1176             this.localShardDataTree = localShardDataTree;
1177         }
1178
1179         Optional<DataTree> getLocalShardDataTree() {
1180             return localShardDataTree;
1181         }
1182
1183         DatastoreContext getDatastoreContext() {
1184             return datastoreContext;
1185         }
1186
1187         void setDatastoreContext(DatastoreContext datastoreContext, ActorRef sender) {
1188             this.datastoreContext = datastoreContext;
1189             if (actor != null) {
1190                 LOG.debug ("Sending new DatastoreContext to {}", shardId);
1191                 actor.tell(this.datastoreContext, sender);
1192             }
1193         }
1194
1195         void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
1196             LOG.info("updatePeerAddress for peer {} with address {}", peerId, peerAddress);
1197
1198             if(actor != null) {
1199                 if(LOG.isDebugEnabled()) {
1200                     LOG.debug("Sending PeerAddressResolved for peer {} with address {} to {}",
1201                             peerId, peerAddress, actor.path());
1202                 }
1203
1204                 actor.tell(new PeerAddressResolved(peerId, peerAddress), sender);
1205             }
1206
1207             notifyOnShardInitializedCallbacks();
1208         }
1209
1210         void peerDown(String memberName, String peerId, ActorRef sender) {
1211             if(actor != null) {
1212                 actor.tell(new PeerDown(memberName, peerId), sender);
1213             }
1214         }
1215
1216         void peerUp(String memberName, String peerId, ActorRef sender) {
1217             if(actor != null) {
1218                 actor.tell(new PeerUp(memberName, peerId), sender);
1219             }
1220         }
1221
1222         boolean isShardReady() {
1223             return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
1224         }
1225
1226         boolean isShardReadyWithLeaderId() {
1227             return leaderAvailable && isShardReady() && !RaftState.IsolatedLeader.name().equals(role) &&
1228                     (isLeader() || addressResolver.resolve(leaderId) != null);
1229         }
1230
1231         boolean isShardInitialized() {
1232             return getActor() != null && actorInitialized;
1233         }
1234
1235         boolean isLeader() {
1236             return Objects.equal(leaderId, shardId.toString());
1237         }
1238
1239         String getSerializedLeaderActor() {
1240             if(isLeader()) {
1241                 return Serialization.serializedActorPath(getActor());
1242             } else {
1243                 return addressResolver.resolve(leaderId);
1244             }
1245         }
1246
1247         void setActorInitialized() {
1248             LOG.debug("Shard {} is initialized", shardId);
1249
1250             this.actorInitialized = true;
1251
1252             notifyOnShardInitializedCallbacks();
1253         }
1254
1255         private void notifyOnShardInitializedCallbacks() {
1256             if(onShardInitializedSet.isEmpty()) {
1257                 return;
1258             }
1259
1260             boolean ready = isShardReadyWithLeaderId();
1261
1262             if(LOG.isDebugEnabled()) {
1263                 LOG.debug("Shard {} is {} - notifying {} OnShardInitialized callbacks", shardId,
1264                         ready ? "ready" : "initialized", onShardInitializedSet.size());
1265             }
1266
1267             Iterator<OnShardInitialized> iter = onShardInitializedSet.iterator();
1268             while(iter.hasNext()) {
1269                 OnShardInitialized onShardInitialized = iter.next();
1270                 if(!(onShardInitialized instanceof OnShardReady) || ready) {
1271                     iter.remove();
1272                     onShardInitialized.getTimeoutSchedule().cancel();
1273                     onShardInitialized.getReplyRunnable().run();
1274                 }
1275             }
1276         }
1277
1278         void addOnShardInitialized(OnShardInitialized onShardInitialized) {
1279             onShardInitializedSet.add(onShardInitialized);
1280         }
1281
1282         void removeOnShardInitialized(OnShardInitialized onShardInitialized) {
1283             onShardInitializedSet.remove(onShardInitialized);
1284         }
1285
1286         void setRole(String newRole) {
1287             this.role = newRole;
1288
1289             notifyOnShardInitializedCallbacks();
1290         }
1291
1292         void setFollowerSyncStatus(boolean syncStatus){
1293             this.followerSyncStatus = syncStatus;
1294         }
1295
1296         boolean isInSync(){
1297             if(RaftState.Follower.name().equals(this.role)){
1298                 return followerSyncStatus;
1299             } else if(RaftState.Leader.name().equals(this.role)){
1300                 return true;
1301             }
1302
1303             return false;
1304         }
1305
1306         boolean setLeaderId(String leaderId) {
1307             boolean changed = !Objects.equal(this.leaderId, leaderId);
1308             this.leaderId = leaderId;
1309             if(leaderId != null) {
1310                 this.leaderAvailable = true;
1311             }
1312             notifyOnShardInitializedCallbacks();
1313
1314             return changed;
1315         }
1316
1317         String getLeaderId() {
1318             return leaderId;
1319         }
1320
1321         void setLeaderAvailable(boolean leaderAvailable) {
1322             this.leaderAvailable = leaderAvailable;
1323         }
1324
1325         short getLeaderVersion() {
1326             return leaderVersion;
1327         }
1328
1329         void setLeaderVersion(short leaderVersion) {
1330             this.leaderVersion = leaderVersion;
1331         }
1332
1333         boolean isActiveMember() {
1334             return isActiveMember;
1335         }
1336
1337         void setActiveMember(boolean isActiveMember) {
1338             this.isActiveMember = isActiveMember;
1339         }
1340     }
1341
1342     private static class OnShardInitialized {
1343         private final Runnable replyRunnable;
1344         private Cancellable timeoutSchedule;
1345
1346         OnShardInitialized(Runnable replyRunnable) {
1347             this.replyRunnable = replyRunnable;
1348         }
1349
1350         Runnable getReplyRunnable() {
1351             return replyRunnable;
1352         }
1353
1354         Cancellable getTimeoutSchedule() {
1355             return timeoutSchedule;
1356         }
1357
1358         void setTimeoutSchedule(Cancellable timeoutSchedule) {
1359             this.timeoutSchedule = timeoutSchedule;
1360         }
1361     }
1362
1363     private static class OnShardReady extends OnShardInitialized {
1364         OnShardReady(Runnable replyRunnable) {
1365             super(replyRunnable);
1366         }
1367     }
1368
1369     private static class ShardNotInitializedTimeout {
1370         private final ActorRef sender;
1371         private final ShardInformation shardInfo;
1372         private final OnShardInitialized onShardInitialized;
1373
1374         ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
1375             this.sender = sender;
1376             this.shardInfo = shardInfo;
1377             this.onShardInitialized = onShardInitialized;
1378         }
1379
1380         ActorRef getSender() {
1381             return sender;
1382         }
1383
1384         ShardInformation getShardInfo() {
1385             return shardInfo;
1386         }
1387
1388         OnShardInitialized getOnShardInitialized() {
1389             return onShardInitialized;
1390         }
1391     }
1392
1393     /**
1394      * We no longer persist SchemaContextModules but keep this class around for now for backwards
1395      * compatibility so we don't get de-serialization failures on upgrade from Helium.
1396      */
1397     @Deprecated
1398     static class SchemaContextModules implements Serializable {
1399         private static final long serialVersionUID = -8884620101025936590L;
1400
1401         private final Set<String> modules;
1402
1403         SchemaContextModules(Set<String> modules){
1404             this.modules = modules;
1405         }
1406
1407         public Set<String> getModules() {
1408             return modules;
1409         }
1410     }
1411
1412     public static Builder builder() {
1413         return new Builder();
1414     }
1415
1416     public static class Builder {
1417         private ClusterWrapper cluster;
1418         private Configuration configuration;
1419         private DatastoreContextFactory datastoreContextFactory;
1420         private CountDownLatch waitTillReadyCountdownLatch;
1421         private PrimaryShardInfoFutureCache primaryShardInfoCache;
1422         private DatastoreSnapshot restoreFromSnapshot;
1423         private volatile boolean sealed;
1424
1425         protected void checkSealed() {
1426             Preconditions.checkState(!sealed, "Builder is already sealed - further modifications are not allowed");
1427         }
1428
1429         public Builder cluster(ClusterWrapper cluster) {
1430             checkSealed();
1431             this.cluster = cluster;
1432             return this;
1433         }
1434
1435         public Builder configuration(Configuration configuration) {
1436             checkSealed();
1437             this.configuration = configuration;
1438             return this;
1439         }
1440
1441         public Builder datastoreContextFactory(DatastoreContextFactory datastoreContextFactory) {
1442             checkSealed();
1443             this.datastoreContextFactory = datastoreContextFactory;
1444             return this;
1445         }
1446
1447         public Builder waitTillReadyCountdownLatch(CountDownLatch waitTillReadyCountdownLatch) {
1448             checkSealed();
1449             this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
1450             return this;
1451         }
1452
1453         public Builder primaryShardInfoCache(PrimaryShardInfoFutureCache primaryShardInfoCache) {
1454             checkSealed();
1455             this.primaryShardInfoCache = primaryShardInfoCache;
1456             return this;
1457         }
1458
1459         public Builder restoreFromSnapshot(DatastoreSnapshot restoreFromSnapshot) {
1460             checkSealed();
1461             this.restoreFromSnapshot = restoreFromSnapshot;
1462             return this;
1463         }
1464
1465         protected void verify() {
1466             sealed = true;
1467             Preconditions.checkNotNull(cluster, "cluster should not be null");
1468             Preconditions.checkNotNull(configuration, "configuration should not be null");
1469             Preconditions.checkNotNull(datastoreContextFactory, "datastoreContextFactory should not be null");
1470             Preconditions.checkNotNull(waitTillReadyCountdownLatch, "waitTillReadyCountdownLatch should not be null");
1471             Preconditions.checkNotNull(primaryShardInfoCache, "primaryShardInfoCache should not be null");
1472         }
1473
1474         public Props props() {
1475             verify();
1476             return Props.create(ShardManager.class, this);
1477         }
1478     }
1479 }
1480
1481
1482