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