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