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