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