Fix issues when persistence enabled
[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.persisted.DataTreeCandidateSupplier;
54 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
55 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
56 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
57 import org.opendaylight.controller.cluster.notifications.RoleChangeNotifier;
58 import org.opendaylight.controller.cluster.raft.RaftActor;
59 import org.opendaylight.controller.cluster.raft.RaftActorRecoveryCohort;
60 import org.opendaylight.controller.cluster.raft.RaftActorSnapshotCohort;
61 import org.opendaylight.controller.cluster.raft.RaftState;
62 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
63 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
64 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
65 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
66 import org.opendaylight.yangtools.concepts.Identifier;
67 import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException;
68 import org.opendaylight.yangtools.yang.data.api.schema.tree.TipProducingDataTree;
69 import org.opendaylight.yangtools.yang.data.api.schema.tree.TreeType;
70 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
71 import scala.concurrent.duration.Duration;
72 import scala.concurrent.duration.FiniteDuration;
73
74 /**
75  * A Shard represents a portion of the logical data tree <br/>
76  * <p>
77  * Our Shard uses InMemoryDataTree as it's internal representation and delegates all requests it
78  * </p>
79  */
80 public class Shard extends RaftActor {
81
82     @VisibleForTesting
83     static final Object TX_COMMIT_TIMEOUT_CHECK_MESSAGE = new Object() {
84         @Override
85         public String toString() {
86             return "txCommitTimeoutCheck";
87         }
88     };
89
90     @VisibleForTesting
91     static final Object GET_SHARD_MBEAN_MESSAGE = new Object() {
92         @Override
93         public String toString() {
94             return "getShardMBeanMessage";
95         }
96     };
97
98     // FIXME: shard names should be encapsulated in their own class and this should be exposed as a constant.
99     public static final String DEFAULT_NAME = "default";
100
101     // The state of this Shard
102     private final ShardDataTree store;
103
104     /// The name of this shard
105     private final String name;
106
107     private final ShardStats shardMBean;
108
109     private DatastoreContext datastoreContext;
110
111     private final ShardCommitCoordinator commitCoordinator;
112
113     private long transactionCommitTimeout;
114
115     private Cancellable txCommitTimeoutCheckSchedule;
116
117     private final Optional<ActorRef> roleChangeNotifier;
118
119     private final MessageTracker appendEntriesReplyTracker;
120
121     private final ShardTransactionActorFactory transactionActorFactory;
122
123     private final ShardSnapshotCohort snapshotCohort;
124
125     private final DataTreeChangeListenerSupport treeChangeSupport = new DataTreeChangeListenerSupport(this);
126     private final DataChangeListenerSupport changeSupport = new DataChangeListenerSupport(this);
127
128
129     private ShardSnapshot restoreFromSnapshot;
130
131     private final ShardTransactionMessageRetrySupport messageRetrySupport;
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, final short leaderPayloadVersion) {
305         return isLeader() ? new ShardLeaderStateChanged(memberId, leaderId, store.getDataTree(), leaderPayloadVersion)
306                 : new ShardLeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
307     }
308
309     protected void onDatastoreContext(final DatastoreContext context) {
310         datastoreContext = context;
311
312         setTransactionCommitTimeout();
313
314         setPersistence(datastoreContext.isPersistent());
315
316         updateConfigParams(datastoreContext.getShardRaftConfig());
317     }
318
319     boolean canSkipPayload() {
320         // If we do not have any followers and we are not using persistence we can apply modification to the state
321         // immediately
322         return !hasFollowers() && !persistence().isRecoveryApplicable();
323     }
324
325     // applyState() will be invoked once consensus is reached on the payload
326     void persistPayload(final TransactionIdentifier transactionId, final Payload payload) {
327         // We are faking the sender
328         persistData(self(), transactionId, payload);
329     }
330
331     private void handleCommitTransaction(final CommitTransaction commit) {
332         if (isLeader()) {
333             commitCoordinator.handleCommit(commit.getTransactionID(), getSender(), this);
334         } else {
335             ActorSelection leader = getLeader();
336             if (leader == null) {
337                 messageRetrySupport.addMessageToRetry(commit, getSender(),
338                         "Could not commit transaction " + commit.getTransactionID());
339             } else {
340                 LOG.debug("{}: Forwarding CommitTransaction to leader {}", persistenceId(), leader);
341                 leader.forward(commit, getContext());
342             }
343         }
344     }
345
346     private void handleCanCommitTransaction(final CanCommitTransaction canCommit) {
347         LOG.debug("{}: Can committing transaction {}", persistenceId(), canCommit.getTransactionID());
348
349         if (isLeader()) {
350         commitCoordinator.handleCanCommit(canCommit.getTransactionID(), getSender(), this);
351         } else {
352             ActorSelection leader = getLeader();
353             if (leader == null) {
354                 messageRetrySupport.addMessageToRetry(canCommit, getSender(),
355                         "Could not canCommit transaction " + canCommit.getTransactionID());
356             } else {
357                 LOG.debug("{}: Forwarding CanCommitTransaction to leader {}", persistenceId(), leader);
358                 leader.forward(canCommit, getContext());
359             }
360         }
361     }
362
363     protected void handleBatchedModificationsLocal(final BatchedModifications batched, final ActorRef sender) {
364         try {
365             commitCoordinator.handleBatchedModifications(batched, sender, this);
366         } catch (Exception e) {
367             LOG.error("{}: Error handling BatchedModifications for Tx {}", persistenceId(),
368                     batched.getTransactionID(), e);
369             sender.tell(new akka.actor.Status.Failure(e), getSelf());
370         }
371     }
372
373     private void handleBatchedModifications(final BatchedModifications batched) {
374         // This message is sent to prepare the modifications transaction directly on the Shard as an
375         // optimization to avoid the extra overhead of a separate ShardTransaction actor. On the last
376         // BatchedModifications message, the caller sets the ready flag in the message indicating
377         // modifications are complete. The reply contains the cohort actor path (this actor) for the caller
378         // to initiate the 3-phase commit. This also avoids the overhead of sending an additional
379         // ReadyTransaction message.
380
381         // If we're not the leader then forward to the leader. This is a safety measure - we shouldn't
382         // normally get here if we're not the leader as the front-end (TransactionProxy) should determine
383         // the primary/leader shard. However with timing and caching on the front-end, there's a small
384         // window where it could have a stale leader during leadership transitions.
385         //
386         boolean isLeaderActive = isLeaderActive();
387         if (isLeader() && isLeaderActive) {
388             handleBatchedModificationsLocal(batched, getSender());
389         } else {
390             ActorSelection leader = getLeader();
391             if (!isLeaderActive || leader == null) {
392                 messageRetrySupport.addMessageToRetry(batched, getSender(),
393                         "Could not commit transaction " + batched.getTransactionID());
394             } else {
395                 // If this is not the first batch and leadership changed in between batched messages,
396                 // we need to reconstruct previous BatchedModifications from the transaction
397                 // DataTreeModification, honoring the max batched modification count, and forward all the
398                 // previous BatchedModifications to the new leader.
399                 Collection<BatchedModifications> newModifications = commitCoordinator.createForwardedBatchedModifications(
400                         batched, 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     private void handleReadyLocalTransaction(final ReadyLocalTransaction message) {
429         LOG.debug("{}: handleReadyLocalTransaction for {}", persistenceId(), message.getTransactionID());
430
431         boolean isLeaderActive = isLeaderActive();
432         if (isLeader() && isLeaderActive) {
433             try {
434                 commitCoordinator.handleReadyLocalTransaction(message, getSender(), this);
435             } catch (Exception e) {
436                 LOG.error("{}: Error handling ReadyLocalTransaction for Tx {}", persistenceId(),
437                         message.getTransactionID(), e);
438                 getSender().tell(new akka.actor.Status.Failure(e), getSelf());
439             }
440         } else {
441             ActorSelection leader = getLeader();
442             if (!isLeaderActive || leader == null) {
443                 messageRetrySupport.addMessageToRetry(message, getSender(),
444                         "Could not commit transaction " + message.getTransactionID());
445             } else {
446                 LOG.debug("{}: Forwarding ReadyLocalTransaction to leader {}", persistenceId(), leader);
447                 message.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion());
448                 leader.forward(message, getContext());
449             }
450         }
451     }
452
453     private void handleForwardedReadyTransaction(final ForwardedReadyTransaction forwardedReady) {
454         LOG.debug("{}: handleForwardedReadyTransaction for {}", persistenceId(), forwardedReady.getTransactionID());
455
456         boolean isLeaderActive = isLeaderActive();
457         if (isLeader() && isLeaderActive) {
458             commitCoordinator.handleForwardedReadyTransaction(forwardedReady, getSender(), this);
459         } else {
460             ActorSelection leader = getLeader();
461             if (!isLeaderActive || leader == null) {
462                 messageRetrySupport.addMessageToRetry(forwardedReady, getSender(),
463                         "Could not commit transaction " + forwardedReady.getTransactionID());
464             } else {
465                 LOG.debug("{}: Forwarding ForwardedReadyTransaction to leader {}", persistenceId(), leader);
466
467                 ReadyLocalTransaction readyLocal = new ReadyLocalTransaction(forwardedReady.getTransactionID(),
468                         forwardedReady.getTransaction().getSnapshot(), forwardedReady.isDoImmediateCommit());
469                 readyLocal.setRemoteVersion(getCurrentBehavior().getLeaderPayloadVersion());
470                 leader.forward(readyLocal, getContext());
471             }
472         }
473     }
474
475     private void handleAbortTransaction(final AbortTransaction abort) {
476         doAbortTransaction(abort.getTransactionID(), getSender());
477     }
478
479     void doAbortTransaction(final TransactionIdentifier transactionID, final ActorRef sender) {
480         commitCoordinator.handleAbort(transactionID, sender, this);
481     }
482
483     private void handleCreateTransaction(final Object message) {
484         if (isLeader()) {
485             createTransaction(CreateTransaction.fromSerializable(message));
486         } else if (getLeader() != null) {
487             getLeader().forward(message, getContext());
488         } else {
489             getSender().tell(new akka.actor.Status.Failure(new NoShardLeaderException(
490                     "Could not create a shard transaction", persistenceId())), getSelf());
491         }
492     }
493
494     private void closeTransactionChain(final CloseTransactionChain closeTransactionChain) {
495         store.closeTransactionChain(closeTransactionChain.getIdentifier());
496     }
497
498     private void createTransaction(final CreateTransaction createTransaction) {
499         try {
500             if (TransactionType.fromInt(createTransaction.getTransactionType()) != TransactionType.READ_ONLY &&
501                     failIfIsolatedLeader(getSender())) {
502                 return;
503             }
504
505             ActorRef transactionActor = createTransaction(createTransaction.getTransactionType(),
506                 createTransaction.getTransactionId());
507
508             getSender().tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor),
509                     createTransaction.getTransactionId(), createTransaction.getVersion()).toSerializable(), getSelf());
510         } catch (Exception e) {
511             getSender().tell(new akka.actor.Status.Failure(e), getSelf());
512         }
513     }
514
515     private ActorRef createTransaction(final int transactionType, final TransactionIdentifier transactionId) {
516         LOG.debug("{}: Creating transaction : {} ", persistenceId(), transactionId);
517         return transactionActorFactory.newShardTransaction(TransactionType.fromInt(transactionType),
518             transactionId);
519     }
520
521     private void updateSchemaContext(final UpdateSchemaContext message) {
522         updateSchemaContext(message.getSchemaContext());
523     }
524
525     @VisibleForTesting
526     void updateSchemaContext(final SchemaContext schemaContext) {
527         store.updateSchemaContext(schemaContext);
528     }
529
530     private boolean isMetricsCaptureEnabled() {
531         CommonConfig config = new CommonConfig(getContext().system().settings().config());
532         return config.isMetricCaptureEnabled();
533     }
534
535     @Override
536     @VisibleForTesting
537     public RaftActorSnapshotCohort getRaftActorSnapshotCohort() {
538         return snapshotCohort;
539     }
540
541     @Override
542     @Nonnull
543     protected RaftActorRecoveryCohort getRaftActorRecoveryCohort() {
544         return new ShardRecoveryCoordinator(store,
545             restoreFromSnapshot != null ? restoreFromSnapshot.getSnapshot() : null, persistenceId(), LOG);
546     }
547
548     @Override
549     protected void onRecoveryComplete() {
550         restoreFromSnapshot = null;
551
552         //notify shard manager
553         getContext().parent().tell(new ActorInitialized(), getSelf());
554
555         // Being paranoid here - this method should only be called once but just in case...
556         if (txCommitTimeoutCheckSchedule == null) {
557             // Schedule a message to be periodically sent to check if the current in-progress
558             // transaction should be expired and aborted.
559             FiniteDuration period = Duration.create(transactionCommitTimeout / 3, TimeUnit.MILLISECONDS);
560             txCommitTimeoutCheckSchedule = getContext().system().scheduler().schedule(
561                     period, period, getSelf(),
562                     TX_COMMIT_TIMEOUT_CHECK_MESSAGE, getContext().dispatcher(), ActorRef.noSender());
563         }
564     }
565
566     @Override
567     protected void applyState(final ActorRef clientActor, final Identifier identifier, final Object data) {
568         if (data instanceof DataTreeCandidateSupplier) {
569             if (clientActor == null) {
570                 // No clientActor indicates a replica coming from the leader
571                 try {
572                     store.applyStateFromLeader(identifier, (DataTreeCandidateSupplier)data);
573                 } catch (DataValidationFailedException | IOException e) {
574                     LOG.error("{}: Error applying replica {}", persistenceId(), identifier, e);
575                 }
576             } else {
577                 // Replication consensus reached, proceed to commit
578                 store.payloadReplicationComplete(identifier, (DataTreeCandidateSupplier)data);
579             }
580         } else {
581             LOG.error("{}: Unknown state received {} ClassLoader {}", persistenceId(), data,
582                 data.getClass().getClassLoader());
583         }
584     }
585
586     @Override
587     protected void onStateChanged() {
588         boolean isLeader = isLeader();
589         boolean hasLeader = hasLeader();
590         changeSupport.onLeadershipChange(isLeader, hasLeader);
591         treeChangeSupport.onLeadershipChange(isLeader, hasLeader);
592
593         // If this actor is no longer the leader close all the transaction chains
594         if (!isLeader) {
595             if (LOG.isDebugEnabled()) {
596                 LOG.debug(
597                     "{}: onStateChanged: Closing all transaction chains because shard {} is no longer the leader",
598                     persistenceId(), getId());
599             }
600
601             store.closeAllTransactionChains();
602         }
603
604         if (hasLeader && !isIsolatedLeader()) {
605             messageRetrySupport.retryMessages();
606         }
607     }
608
609     @Override
610     protected void onLeaderChanged(final String oldLeader, final String newLeader) {
611         shardMBean.incrementLeadershipChangeCount();
612
613         boolean hasLeader = hasLeader();
614         if (hasLeader && !isLeader()) {
615             // Another leader was elected. If we were the previous leader and had pending transactions, convert
616             // them to transaction messages and send to the new leader.
617             ActorSelection leader = getLeader();
618             if (leader != null) {
619                 Collection<?> messagesToForward = commitCoordinator.convertPendingTransactionsToMessages(
620                             datastoreContext.getShardBatchedModificationCount());
621
622                 if (!messagesToForward.isEmpty()) {
623                     LOG.debug("{}: Forwarding {} pending transaction messages to leader {}", persistenceId(),
624                             messagesToForward.size(), leader);
625
626                     for (Object message : messagesToForward) {
627                         leader.tell(message, self());
628                     }
629                 }
630             } else {
631                 commitCoordinator.abortPendingTransactions(
632                         "The transacton was aborted due to inflight leadership change and the leader address isn't available.",
633                         this);
634             }
635         }
636
637         if (hasLeader && !isIsolatedLeader()) {
638             messageRetrySupport.retryMessages();
639         }
640     }
641
642     @Override
643     protected void pauseLeader(final Runnable operation) {
644         LOG.debug("{}: In pauseLeader, operation: {}", persistenceId(), operation);
645         store.setRunOnPendingTransactionsComplete(operation);
646     }
647
648     @Override
649     public String persistenceId() {
650         return this.name;
651     }
652
653     @VisibleForTesting
654     ShardCommitCoordinator getCommitCoordinator() {
655         return commitCoordinator;
656     }
657
658     public DatastoreContext getDatastoreContext() {
659         return datastoreContext;
660     }
661
662     @VisibleForTesting
663     public ShardDataTree getDataStore() {
664         return store;
665     }
666
667     @VisibleForTesting
668     ShardStats getShardMBean() {
669         return shardMBean;
670     }
671
672     public static Builder builder() {
673         return new Builder();
674     }
675
676     public static abstract class AbstractBuilder<T extends AbstractBuilder<T, S>, S extends Shard> {
677         private final Class<S> shardClass;
678         private ShardIdentifier id;
679         private Map<String, String> peerAddresses = Collections.emptyMap();
680         private DatastoreContext datastoreContext;
681         private SchemaContext schemaContext;
682         private DatastoreSnapshot.ShardSnapshot restoreFromSnapshot;
683         private TipProducingDataTree dataTree;
684         private volatile boolean sealed;
685
686         protected AbstractBuilder(final Class<S> shardClass) {
687             this.shardClass = shardClass;
688         }
689
690         protected void checkSealed() {
691             Preconditions.checkState(!sealed, "Builder isalready sealed - further modifications are not allowed");
692         }
693
694         @SuppressWarnings("unchecked")
695         private T self() {
696             return (T) this;
697         }
698
699         public T id(final ShardIdentifier id) {
700             checkSealed();
701             this.id = id;
702             return self();
703         }
704
705         public T peerAddresses(final Map<String, String> peerAddresses) {
706             checkSealed();
707             this.peerAddresses = peerAddresses;
708             return self();
709         }
710
711         public T datastoreContext(final DatastoreContext datastoreContext) {
712             checkSealed();
713             this.datastoreContext = datastoreContext;
714             return self();
715         }
716
717         public T schemaContext(final SchemaContext schemaContext) {
718             checkSealed();
719             this.schemaContext = schemaContext;
720             return self();
721         }
722
723         public T restoreFromSnapshot(final DatastoreSnapshot.ShardSnapshot restoreFromSnapshot) {
724             checkSealed();
725             this.restoreFromSnapshot = restoreFromSnapshot;
726             return self();
727         }
728
729         public T dataTree(final TipProducingDataTree dataTree) {
730             checkSealed();
731             this.dataTree = dataTree;
732             return self();
733         }
734
735         public ShardIdentifier getId() {
736             return id;
737         }
738
739         public Map<String, String> getPeerAddresses() {
740             return peerAddresses;
741         }
742
743         public DatastoreContext getDatastoreContext() {
744             return datastoreContext;
745         }
746
747         public SchemaContext getSchemaContext() {
748             return schemaContext;
749         }
750
751         public DatastoreSnapshot.ShardSnapshot getRestoreFromSnapshot() {
752             return restoreFromSnapshot;
753         }
754
755         public TipProducingDataTree getDataTree() {
756             return dataTree;
757         }
758
759         public TreeType getTreeType() {
760             switch (datastoreContext.getLogicalStoreType()) {
761             case CONFIGURATION:
762                 return TreeType.CONFIGURATION;
763             case OPERATIONAL:
764                 return TreeType.OPERATIONAL;
765             }
766
767             throw new IllegalStateException("Unhandled logical store type " + datastoreContext.getLogicalStoreType());
768         }
769
770         protected void verify() {
771             Preconditions.checkNotNull(id, "id should not be null");
772             Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
773             Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null");
774             Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
775         }
776
777         public Props props() {
778             sealed = true;
779             verify();
780             return Props.create(shardClass, this);
781         }
782     }
783
784     public static class Builder extends AbstractBuilder<Builder, Shard> {
785         private Builder() {
786             super(Shard.class);
787         }
788     }
789
790     Ticker ticker() {
791         return Ticker.systemTicker();
792     }
793 }