Fix warnings/javadocs in sal-distributed-datastore
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / Shard.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.datastore;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.actor.Cancellable;
14 import akka.actor.Props;
15 import akka.serialization.Serialization;
16 import com.google.common.annotations.VisibleForTesting;
17 import com.google.common.base.Optional;
18 import com.google.common.base.Preconditions;
19 import com.google.common.base.Ticker;
20 import java.io.IOException;
21 import java.util.Collection;
22 import java.util.Collections;
23 import java.util.Map;
24 import java.util.concurrent.TimeUnit;
25 import javax.annotation.Nonnull;
26 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
27 import org.opendaylight.controller.cluster.common.actor.CommonConfig;
28 import org.opendaylight.controller.cluster.common.actor.MessageTracker;
29 import org.opendaylight.controller.cluster.common.actor.MessageTracker.Error;
30 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
31 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
32 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
33 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
34 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
35 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
36 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
37 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
38 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
39 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
40 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
41 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
42 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
43 import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot;
44 import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot.ShardSnapshot;
45 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
46 import org.opendaylight.controller.cluster.datastore.messages.GetShardDataTree;
47 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
48 import org.opendaylight.controller.cluster.datastore.messages.ReadyLocalTransaction;
49 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
50 import org.opendaylight.controller.cluster.datastore.messages.RegisterDataTreeChangeListener;
51 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
52 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
53 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
54 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
55 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
56 import org.opendaylight.controller.cluster.notifications.RoleChangeNotifier;
57 import org.opendaylight.controller.cluster.raft.RaftActor;
58 import org.opendaylight.controller.cluster.raft.RaftActorRecoveryCohort;
59 import org.opendaylight.controller.cluster.raft.RaftActorSnapshotCohort;
60 import org.opendaylight.controller.cluster.raft.RaftState;
61 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
62 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
63 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
64 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
65 import org.opendaylight.yangtools.concepts.Identifier;
66 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
67 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
68 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
69 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
70 import scala.concurrent.duration.Duration;
71 import scala.concurrent.duration.FiniteDuration;
72
73 /**
74  * A Shard represents a portion of the logical data tree.
75  * <p/>
76  * Our Shard uses InMemoryDataTree as it's internal representation and delegates all requests it
77  */
78 public class Shard extends RaftActor {
79
80     @VisibleForTesting
81     static final Object TX_COMMIT_TIMEOUT_CHECK_MESSAGE = new Object() {
82         @Override
83         public String toString() {
84             return "txCommitTimeoutCheck";
85         }
86     };
87
88     @VisibleForTesting
89     static final Object GET_SHARD_MBEAN_MESSAGE = new Object() {
90         @Override
91         public String toString() {
92             return "getShardMBeanMessage";
93         }
94     };
95
96     // FIXME: shard names should be encapsulated in their own class and this should be exposed as a constant.
97     public static final String DEFAULT_NAME = "default";
98
99     // The state of this Shard
100     private final ShardDataTree store;
101
102     /// The name of this shard
103     private final String name;
104
105     private final ShardStats shardMBean;
106
107     private DatastoreContext datastoreContext;
108
109     private final ShardCommitCoordinator commitCoordinator;
110
111     private long transactionCommitTimeout;
112
113     private Cancellable txCommitTimeoutCheckSchedule;
114
115     private final Optional<ActorRef> roleChangeNotifier;
116
117     private final MessageTracker appendEntriesReplyTracker;
118
119     private final ShardTransactionActorFactory transactionActorFactory;
120
121     private final ShardSnapshotCohort snapshotCohort;
122
123     private final DataTreeChangeListenerSupport treeChangeSupport = new DataTreeChangeListenerSupport(this);
124     private final DataChangeListenerSupport changeSupport = new DataChangeListenerSupport(this);
125
126
127     private ShardSnapshot restoreFromSnapshot;
128
129     private final ShardTransactionMessageRetrySupport messageRetrySupport;
130
131     private final FrontendMetadata frontendMetadata = new FrontendMetadata();
132
133     protected Shard(final AbstractBuilder<?, ?> builder) {
134         super(builder.getId().toString(), builder.getPeerAddresses(),
135                 Optional.of(builder.getDatastoreContext().getShardRaftConfig()), DataStoreVersions.CURRENT_VERSION);
136
137         this.name = builder.getId().toString();
138         this.datastoreContext = builder.getDatastoreContext();
139         this.restoreFromSnapshot = builder.getRestoreFromSnapshot();
140
141         setPersistence(datastoreContext.isPersistent());
142
143         LOG.info("Shard created : {}, persistent : {}", name, datastoreContext.isPersistent());
144
145         ShardDataTreeChangeListenerPublisherActorProxy treeChangeListenerPublisher =
146                 new ShardDataTreeChangeListenerPublisherActorProxy(getContext(), name + "-DTCL-publisher");
147         ShardDataChangeListenerPublisherActorProxy dataChangeListenerPublisher =
148                 new ShardDataChangeListenerPublisherActorProxy(getContext(), name + "-DCL-publisher");
149         if (builder.getDataTree() != null) {
150             store = new ShardDataTree(this, builder.getSchemaContext(), builder.getDataTree(),
151                     treeChangeListenerPublisher, dataChangeListenerPublisher, name);
152         } else {
153             store = new ShardDataTree(this, builder.getSchemaContext(), builder.getTreeType(),
154                     treeChangeListenerPublisher, dataChangeListenerPublisher, name);
155         }
156
157         shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(),
158                 datastoreContext.getDataStoreMXBeanType());
159         shardMBean.setShard(this);
160
161         if (isMetricsCaptureEnabled()) {
162             getContext().become(new MeteringBehavior(this));
163         }
164
165         commitCoordinator = new ShardCommitCoordinator(store, LOG, this.name);
166
167         setTransactionCommitTimeout();
168
169         // create a notifier actor for each cluster member
170         roleChangeNotifier = createRoleChangeNotifier(name.toString());
171
172         appendEntriesReplyTracker = new MessageTracker(AppendEntriesReply.class,
173                 getRaftActorContext().getConfigParams().getIsolatedCheckIntervalInMillis());
174
175         transactionActorFactory = new ShardTransactionActorFactory(store, datastoreContext,
176                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(
177                         Dispatchers.DispatcherType.Transaction), self(), getContext(), shardMBean);
178
179         snapshotCohort = ShardSnapshotCohort.create(getContext(), builder.getId().getMemberName(), store, LOG,
180             this.name);
181
182         messageRetrySupport = new ShardTransactionMessageRetrySupport(this);
183     }
184
185     private void setTransactionCommitTimeout() {
186         transactionCommitTimeout = TimeUnit.MILLISECONDS.convert(
187                 datastoreContext.getShardTransactionCommitTimeoutInSeconds(), TimeUnit.SECONDS) / 2;
188     }
189
190     private Optional<ActorRef> createRoleChangeNotifier(final String shardId) {
191         ActorRef shardRoleChangeNotifier = this.getContext().actorOf(
192             RoleChangeNotifier.getProps(shardId), shardId + "-notifier");
193         return Optional.of(shardRoleChangeNotifier);
194     }
195
196     @Override
197     public void postStop() {
198         LOG.info("Stopping Shard {}", persistenceId());
199
200         super.postStop();
201
202         messageRetrySupport.close();
203
204         if (txCommitTimeoutCheckSchedule != null) {
205             txCommitTimeoutCheckSchedule.cancel();
206         }
207
208         commitCoordinator.abortPendingTransactions("Transaction aborted due to shutdown.", this);
209
210         shardMBean.unregisterMBean();
211     }
212
213     @Override
214     protected void handleRecover(final Object message) {
215         LOG.debug("{}: onReceiveRecover: Received message {} from {}", persistenceId(), message.getClass(),
216             getSender());
217
218         super.handleRecover(message);
219         if (LOG.isTraceEnabled()) {
220             appendEntriesReplyTracker.begin();
221         }
222     }
223
224     @Override
225     protected void handleNonRaftCommand(final Object message) {
226         try (final MessageTracker.Context context = appendEntriesReplyTracker.received(message)) {
227             final Optional<Error> maybeError = context.error();
228             if (maybeError.isPresent()) {
229                 LOG.trace("{} : AppendEntriesReply failed to arrive at the expected interval {}", persistenceId(),
230                     maybeError.get());
231             }
232
233             if (CreateTransaction.isSerializedType(message)) {
234                 handleCreateTransaction(message);
235             } else if (message instanceof BatchedModifications) {
236                 handleBatchedModifications((BatchedModifications)message);
237             } else if (message instanceof ForwardedReadyTransaction) {
238                 handleForwardedReadyTransaction((ForwardedReadyTransaction) message);
239             } else if (message instanceof ReadyLocalTransaction) {
240                 handleReadyLocalTransaction((ReadyLocalTransaction)message);
241             } else if (CanCommitTransaction.isSerializedType(message)) {
242                 handleCanCommitTransaction(CanCommitTransaction.fromSerializable(message));
243             } else if (CommitTransaction.isSerializedType(message)) {
244                 handleCommitTransaction(CommitTransaction.fromSerializable(message));
245             } else if (AbortTransaction.isSerializedType(message)) {
246                 handleAbortTransaction(AbortTransaction.fromSerializable(message));
247             } else if (CloseTransactionChain.isSerializedType(message)) {
248                 closeTransactionChain(CloseTransactionChain.fromSerializable(message));
249             } else if (message instanceof RegisterChangeListener) {
250                 changeSupport.onMessage((RegisterChangeListener) message, isLeader(), hasLeader());
251             } else if (message instanceof RegisterDataTreeChangeListener) {
252                 treeChangeSupport.onMessage((RegisterDataTreeChangeListener) message, isLeader(), hasLeader());
253             } else if (message instanceof UpdateSchemaContext) {
254                 updateSchemaContext((UpdateSchemaContext) message);
255             } else if (message instanceof PeerAddressResolved) {
256                 PeerAddressResolved resolved = (PeerAddressResolved) message;
257                 setPeerAddress(resolved.getPeerId().toString(),
258                         resolved.getPeerAddress());
259             } else if (TX_COMMIT_TIMEOUT_CHECK_MESSAGE.equals(message)) {
260                 store.checkForExpiredTransactions(transactionCommitTimeout);
261                 commitCoordinator.checkForExpiredTransactions(transactionCommitTimeout, this);
262             } else if (message instanceof DatastoreContext) {
263                 onDatastoreContext((DatastoreContext)message);
264             } else if (message instanceof RegisterRoleChangeListener) {
265                 roleChangeNotifier.get().forward(message, context());
266             } else if (message instanceof FollowerInitialSyncUpStatus) {
267                 shardMBean.setFollowerInitialSyncStatus(((FollowerInitialSyncUpStatus) message).isInitialSyncDone());
268                 context().parent().tell(message, self());
269             } else if (GET_SHARD_MBEAN_MESSAGE.equals(message)) {
270                 sender().tell(getShardMBean(), self());
271             } else if (message instanceof GetShardDataTree) {
272                 sender().tell(store.getDataTree(), self());
273             } else if (message instanceof ServerRemoved) {
274                 context().parent().forward(message, context());
275             } else if (ShardTransactionMessageRetrySupport.TIMER_MESSAGE_CLASS.isInstance(message)) {
276                 messageRetrySupport.onTimerMessage(message);
277             } else if (message instanceof DataTreeCohortActorRegistry.CohortRegistryCommand) {
278                 store.processCohortRegistryCommand(getSender(),
279                         (DataTreeCohortActorRegistry.CohortRegistryCommand) message);
280             } else {
281                 super.handleNonRaftCommand(message);
282             }
283         }
284     }
285
286     private boolean hasLeader() {
287         return getLeaderId() != null;
288     }
289
290     public int getPendingTxCommitQueueSize() {
291         return store.getQueueSize();
292     }
293
294     public int getCohortCacheSize() {
295         return commitCoordinator.getCohortCacheSize();
296     }
297
298     @Override
299     protected Optional<ActorRef> getRoleChangeNotifier() {
300         return roleChangeNotifier;
301     }
302
303     @Override
304     protected LeaderStateChanged newLeaderStateChanged(final String memberId, final String leaderId,
305             final short leaderPayloadVersion) {
306         return isLeader() ? new ShardLeaderStateChanged(memberId, leaderId, store.getDataTree(), leaderPayloadVersion)
307                 : new ShardLeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
308     }
309
310     protected void onDatastoreContext(final DatastoreContext context) {
311         datastoreContext = context;
312
313         setTransactionCommitTimeout();
314
315         setPersistence(datastoreContext.isPersistent());
316
317         updateConfigParams(datastoreContext.getShardRaftConfig());
318     }
319
320     boolean canSkipPayload() {
321         // If we do not have any followers and we are not using persistence we can apply modification to the state
322         // immediately
323         return !hasFollowers() && !persistence().isRecoveryApplicable();
324     }
325
326     // applyState() will be invoked once consensus is reached on the payload
327     void persistPayload(final TransactionIdentifier transactionId, final Payload payload) {
328         // We are faking the sender
329         persistData(self(), transactionId, payload);
330     }
331
332     private void handleCommitTransaction(final CommitTransaction commit) {
333         if (isLeader()) {
334             commitCoordinator.handleCommit(commit.getTransactionID(), getSender(), this);
335         } else {
336             ActorSelection leader = getLeader();
337             if (leader == null) {
338                 messageRetrySupport.addMessageToRetry(commit, getSender(),
339                         "Could not commit transaction " + commit.getTransactionID());
340             } else {
341                 LOG.debug("{}: Forwarding CommitTransaction to leader {}", persistenceId(), leader);
342                 leader.forward(commit, getContext());
343             }
344         }
345     }
346
347     private void handleCanCommitTransaction(final CanCommitTransaction canCommit) {
348         LOG.debug("{}: Can committing transaction {}", persistenceId(), canCommit.getTransactionID());
349
350         if (isLeader()) {
351             commitCoordinator.handleCanCommit(canCommit.getTransactionID(), getSender(), this);
352         } else {
353             ActorSelection leader = getLeader();
354             if (leader == null) {
355                 messageRetrySupport.addMessageToRetry(canCommit, getSender(),
356                         "Could not canCommit transaction " + canCommit.getTransactionID());
357             } else {
358                 LOG.debug("{}: Forwarding CanCommitTransaction to leader {}", persistenceId(), leader);
359                 leader.forward(canCommit, getContext());
360             }
361         }
362     }
363
364     @SuppressWarnings("checkstyle:IllegalCatch")
365     protected void handleBatchedModificationsLocal(final BatchedModifications batched, final ActorRef sender) {
366         try {
367             commitCoordinator.handleBatchedModifications(batched, sender, this);
368         } catch (Exception e) {
369             LOG.error("{}: Error handling BatchedModifications for Tx {}", persistenceId(),
370                     batched.getTransactionID(), e);
371             sender.tell(new akka.actor.Status.Failure(e), getSelf());
372         }
373     }
374
375     private void handleBatchedModifications(final BatchedModifications batched) {
376         // This message is sent to prepare the modifications transaction directly on the Shard as an
377         // optimization to avoid the extra overhead of a separate ShardTransaction actor. On the last
378         // BatchedModifications message, the caller sets the ready flag in the message indicating
379         // modifications are complete. The reply contains the cohort actor path (this actor) for the caller
380         // to initiate the 3-phase commit. This also avoids the overhead of sending an additional
381         // ReadyTransaction message.
382
383         // If we're not the leader then forward to the leader. This is a safety measure - we shouldn't
384         // normally get here if we're not the leader as the front-end (TransactionProxy) should determine
385         // the primary/leader shard. However with timing and caching on the front-end, there's a small
386         // window where it could have a stale leader during leadership transitions.
387         //
388         boolean isLeaderActive = isLeaderActive();
389         if (isLeader() && isLeaderActive) {
390             handleBatchedModificationsLocal(batched, getSender());
391         } else {
392             ActorSelection leader = getLeader();
393             if (!isLeaderActive || leader == null) {
394                 messageRetrySupport.addMessageToRetry(batched, getSender(),
395                         "Could not commit transaction " + batched.getTransactionID());
396             } else {
397                 // If this is not the first batch and leadership changed in between batched messages,
398                 // we need to reconstruct previous BatchedModifications from the transaction
399                 // DataTreeModification, honoring the max batched modification count, and forward all the
400                 // previous BatchedModifications to the new leader.
401                 Collection<BatchedModifications> newModifications = commitCoordinator
402                         .createForwardedBatchedModifications(batched,
403                                 datastoreContext.getShardBatchedModificationCount());
404
405                 LOG.debug("{}: Forwarding {} BatchedModifications to leader {}", persistenceId(),
406                         newModifications.size(), leader);
407
408                 for (BatchedModifications bm : newModifications) {
409                     leader.forward(bm, getContext());
410                 }
411             }
412         }
413     }
414
415     private boolean failIfIsolatedLeader(final ActorRef sender) {
416         if (isIsolatedLeader()) {
417             sender.tell(new akka.actor.Status.Failure(new NoShardLeaderException(String.format(
418                     "Shard %s was the leader but has lost contact with all of its followers. Either all"
419                     + " other follower nodes are down or this node is isolated by a network partition.",
420                     persistenceId()))), getSelf());
421             return true;
422         }
423
424         return false;
425     }
426
427     protected boolean isIsolatedLeader() {
428         return getRaftState() == RaftState.IsolatedLeader;
429     }
430
431     @SuppressWarnings("checkstyle:IllegalCatch")
432     private void handleReadyLocalTransaction(final ReadyLocalTransaction message) {
433         LOG.debug("{}: handleReadyLocalTransaction for {}", persistenceId(), message.getTransactionID());
434
435         boolean isLeaderActive = isLeaderActive();
436         if (isLeader() && isLeaderActive) {
437             try {
438                 commitCoordinator.handleReadyLocalTransaction(message, getSender(), this);
439             } catch (Exception e) {
440                 LOG.error("{}: Error handling ReadyLocalTransaction for Tx {}", persistenceId(),
441                         message.getTransactionID(), e);
442                 getSender().tell(new akka.actor.Status.Failure(e), getSelf());
443             }
444         } else {
445             ActorSelection leader = getLeader();
446             if (!isLeaderActive || leader == null) {
447                 messageRetrySupport.addMessageToRetry(message, getSender(),
448                         "Could not commit transaction " + message.getTransactionID());
449             } else {
450                 LOG.debug("{}: Forwarding ReadyLocalTransaction to leader {}", persistenceId(), leader);
451                 message.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion());
452                 leader.forward(message, getContext());
453             }
454         }
455     }
456
457     private void handleForwardedReadyTransaction(final ForwardedReadyTransaction forwardedReady) {
458         LOG.debug("{}: handleForwardedReadyTransaction for {}", persistenceId(), forwardedReady.getTransactionID());
459
460         boolean isLeaderActive = isLeaderActive();
461         if (isLeader() && isLeaderActive) {
462             commitCoordinator.handleForwardedReadyTransaction(forwardedReady, getSender(), this);
463         } else {
464             ActorSelection leader = getLeader();
465             if (!isLeaderActive || leader == null) {
466                 messageRetrySupport.addMessageToRetry(forwardedReady, getSender(),
467                         "Could not commit transaction " + forwardedReady.getTransactionID());
468             } else {
469                 LOG.debug("{}: Forwarding ForwardedReadyTransaction to leader {}", persistenceId(), leader);
470
471                 ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(forwardedReady.getTransactionID(),
472                         forwardedReady.getTransaction().getSnapshot(), forwardedReady.isDoImmediateCommit());
473                 readyLocal.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion());
474                 leader.forward(readyLocal, getContext());
475             }
476         }
477     }
478
479     private void handleAbortTransaction(final AbortTransaction abort) {
480         doAbortTransaction(abort.getTransactionID(), getSender());
481     }
482
483     void doAbortTransaction(final TransactionIdentifier transactionID, final ActorRef sender) {
484         commitCoordinator.handleAbort(transactionID, sender, this);
485     }
486
487     private void handleCreateTransaction(final Object message) {
488         if (isLeader()) {
489             createTransaction(CreateTransaction.fromSerializable(message));
490         } else if (getLeader() != null) {
491             getLeader().forward(message, getContext());
492         } else {
493             getSender().tell(new akka.actor.Status.Failure(new NoShardLeaderException(
494                     "Could not create a shard transaction", persistenceId())), getSelf());
495         }
496     }
497
498     private void closeTransactionChain(final CloseTransactionChain closeTransactionChain) {
499         store.closeTransactionChain(closeTransactionChain.getIdentifier());
500     }
501
502     @SuppressWarnings("checkstyle:IllegalCatch")
503     private void createTransaction(final CreateTransaction createTransaction) {
504         try {
505             if (TransactionType.fromInt(createTransaction.getTransactionType()) != TransactionType.READ_ONLY
506                     && failIfIsolatedLeader(getSender())) {
507                 return;
508             }
509
510             ActorRef transactionActor = createTransaction(createTransaction.getTransactionType(),
511                 createTransaction.getTransactionId());
512
513             getSender().tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor),
514                     createTransaction.getTransactionId(), createTransaction.getVersion()).toSerializable(), getSelf());
515         } catch (Exception e) {
516             getSender().tell(new akka.actor.Status.Failure(e), getSelf());
517         }
518     }
519
520     private ActorRef createTransaction(final int transactionType, final TransactionIdentifier transactionId) {
521         LOG.debug("{}: Creating transaction : {} ", persistenceId(), transactionId);
522         return transactionActorFactory.newShardTransaction(TransactionType.fromInt(transactionType),
523             transactionId);
524     }
525
526     private void updateSchemaContext(final UpdateSchemaContext message) {
527         updateSchemaContext(message.getSchemaContext());
528     }
529
530     @VisibleForTesting
531     void updateSchemaContext(final SchemaContext schemaContext) {
532         store.updateSchemaContext(schemaContext);
533     }
534
535     private boolean isMetricsCaptureEnabled() {
536         CommonConfig config = new CommonConfig(getContext().system().settings().config());
537         return config.isMetricCaptureEnabled();
538     }
539
540     @Override
541     @VisibleForTesting
542     public RaftActorSnapshotCohort getRaftActorSnapshotCohort() {
543         return snapshotCohort;
544     }
545
546     @Override
547     @Nonnull
548     protected RaftActorRecoveryCohort getRaftActorRecoveryCohort() {
549         return new ShardRecoveryCoordinator(store,
550             restoreFromSnapshot != null ? restoreFromSnapshot.getSnapshot() : null, persistenceId(), LOG);
551     }
552
553     @Override
554     protected void onRecoveryComplete() {
555         restoreFromSnapshot = null;
556
557         //notify shard manager
558         getContext().parent().tell(new ActorInitialized(), getSelf());
559
560         // Being paranoid here - this method should only be called once but just in case...
561         if (txCommitTimeoutCheckSchedule == null) {
562             // Schedule a message to be periodically sent to check if the current in-progress
563             // transaction should be expired and aborted.
564             FiniteDuration period = Duration.create(transactionCommitTimeout / 3, TimeUnit.MILLISECONDS);
565             txCommitTimeoutCheckSchedule = getContext().system().scheduler().schedule(
566                     period, period, getSelf(),
567                     TX_COMMIT_TIMEOUT_CHECK_MESSAGE, getContext().dispatcher(), ActorRef.noSender());
568         }
569     }
570
571     @Override
572     protected void applyState(final ActorRef clientActor, final Identifier identifier, final Object data) {
573         if (data instanceof Payload) {
574             try {
575                 store.applyReplicatedPayload(identifier, (Payload)data);
576             } catch (DataValidationFailedException | IOException e) {
577                 LOG.error("{}: Error applying replica {}", persistenceId(), identifier, e);
578             }
579         } else {
580             LOG.error("{}: Unknown state for {} received {}", persistenceId(), identifier, data);
581         }
582     }
583
584     @Override
585     protected void onStateChanged() {
586         boolean isLeader = isLeader();
587         boolean hasLeader = hasLeader();
588         changeSupport.onLeadershipChange(isLeader, hasLeader);
589         treeChangeSupport.onLeadershipChange(isLeader, hasLeader);
590
591         // If this actor is no longer the leader close all the transaction chains
592         if (!isLeader) {
593             if (LOG.isDebugEnabled()) {
594                 LOG.debug(
595                     "{}: onStateChanged: Closing all transaction chains because shard {} is no longer the leader",
596                     persistenceId(), getId());
597             }
598
599             store.closeAllTransactionChains();
600         }
601
602         if (hasLeader && !isIsolatedLeader()) {
603             messageRetrySupport.retryMessages();
604         }
605     }
606
607     @Override
608     protected void onLeaderChanged(final String oldLeader, final String newLeader) {
609         shardMBean.incrementLeadershipChangeCount();
610
611         boolean hasLeader = hasLeader();
612         if (hasLeader && !isLeader()) {
613             // Another leader was elected. If we were the previous leader and had pending transactions, convert
614             // them to transaction messages and send to the new leader.
615             ActorSelection leader = getLeader();
616             if (leader != null) {
617                 Collection<?> messagesToForward = convertPendingTransactionsToMessages();
618
619                 if (!messagesToForward.isEmpty()) {
620                     LOG.debug("{}: Forwarding {} pending transaction messages to leader {}", persistenceId(),
621                             messagesToForward.size(), leader);
622
623                     for (Object message : messagesToForward) {
624                         leader.tell(message, self());
625                     }
626                 }
627             } else {
628                 commitCoordinator.abortPendingTransactions("The transacton was aborted due to inflight leadership "
629                         + "change and the leader address isn't available.", this);
630             }
631         }
632
633         if (hasLeader && !isIsolatedLeader()) {
634             messageRetrySupport.retryMessages();
635         }
636     }
637
638     /**
639      * Clears all pending transactions and converts them to messages to be forwarded to a new leader.
640      *
641      * @return the converted messages
642      */
643     public Collection<?> convertPendingTransactionsToMessages() {
644         return commitCoordinator.convertPendingTransactionsToMessages(
645                 datastoreContext.getShardBatchedModificationCount());
646     }
647
648     @Override
649     protected void pauseLeader(final Runnable operation) {
650         LOG.debug("{}: In pauseLeader, operation: {}", persistenceId(), operation);
651         store.setRunOnPendingTransactionsComplete(operation);
652     }
653
654     @Override
655     public String persistenceId() {
656         return this.name;
657     }
658
659     @VisibleForTesting
660     ShardCommitCoordinator getCommitCoordinator() {
661         return commitCoordinator;
662     }
663
664     public DatastoreContext getDatastoreContext() {
665         return datastoreContext;
666     }
667
668     @VisibleForTesting
669     public ShardDataTree getDataStore() {
670         return store;
671     }
672
673     @VisibleForTesting
674     ShardStats getShardMBean() {
675         return shardMBean;
676     }
677
678     public static Builder builder() {
679         return new Builder();
680     }
681
682     public abstract static class AbstractBuilder<T extends AbstractBuilder<T, S>, S extends Shard> {
683         private final Class<S> shardClass;
684         private ShardIdentifier id;
685         private Map<String, String> peerAddresses = Collections.emptyMap();
686         private DatastoreContext datastoreContext;
687         private SchemaContext schemaContext;
688         private DatastoreSnapshot.ShardSnapshot restoreFromSnapshot;
689         private TipProducingDataTree dataTree;
690         private volatile boolean sealed;
691
692         protected AbstractBuilder(final Class<S> shardClass) {
693             this.shardClass = shardClass;
694         }
695
696         protected void checkSealed() {
697             Preconditions.checkState(!sealed, "Builder isalready sealed - further modifications are not allowed");
698         }
699
700         @SuppressWarnings("unchecked")
701         private T self() {
702             return (T) this;
703         }
704
705         public T id(final ShardIdentifier newId) {
706             checkSealed();
707             this.id = newId;
708             return self();
709         }
710
711         public T peerAddresses(final Map<String, String> newPeerAddresses) {
712             checkSealed();
713             this.peerAddresses = newPeerAddresses;
714             return self();
715         }
716
717         public T datastoreContext(final DatastoreContext newDatastoreContext) {
718             checkSealed();
719             this.datastoreContext = newDatastoreContext;
720             return self();
721         }
722
723         public T schemaContext(final SchemaContext newSchemaContext) {
724             checkSealed();
725             this.schemaContext = newSchemaContext;
726             return self();
727         }
728
729         public T restoreFromSnapshot(final DatastoreSnapshot.ShardSnapshot newRestoreFromSnapshot) {
730             checkSealed();
731             this.restoreFromSnapshot = newRestoreFromSnapshot;
732             return self();
733         }
734
735         public T dataTree(final TipProducingDataTree newDataTree) {
736             checkSealed();
737             this.dataTree = newDataTree;
738             return self();
739         }
740
741         public ShardIdentifier getId() {
742             return id;
743         }
744
745         public Map<String, String> getPeerAddresses() {
746             return peerAddresses;
747         }
748
749         public DatastoreContext getDatastoreContext() {
750             return datastoreContext;
751         }
752
753         public SchemaContext getSchemaContext() {
754             return schemaContext;
755         }
756
757         public DatastoreSnapshot.ShardSnapshot getRestoreFromSnapshot() {
758             return restoreFromSnapshot;
759         }
760
761         public TipProducingDataTree getDataTree() {
762             return dataTree;
763         }
764
765         public TreeType getTreeType() {
766             switch (datastoreContext.getLogicalStoreType()) {
767                 case CONFIGURATION:
768                     return TreeType.CONFIGURATION;
769                 case OPERATIONAL:
770                     return TreeType.OPERATIONAL;
771                 default:
772                     throw new IllegalStateException("Unhandled logical store type "
773                             + datastoreContext.getLogicalStoreType());
774             }
775         }
776
777         protected void verify() {
778             Preconditions.checkNotNull(id, "id should not be null");
779             Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
780             Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null");
781             Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
782         }
783
784         public Props props() {
785             sealed = true;
786             verify();
787             return Props.create(shardClass, this);
788         }
789     }
790
791     public static class Builder extends AbstractBuilder<Builder, Shard> {
792         private Builder() {
793             super(Shard.class);
794         }
795     }
796
797     Ticker ticker() {
798         return Ticker.systemTicker();
799     }
800 }