Bug 2318: Follow-up changes for previous patch 12535
[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.PoisonPill;
15 import akka.actor.Props;
16 import akka.event.Logging;
17 import akka.event.LoggingAdapter;
18 import akka.japi.Creator;
19 import akka.persistence.RecoveryFailure;
20 import akka.serialization.Serialization;
21 import com.google.common.annotations.VisibleForTesting;
22 import com.google.common.base.Optional;
23 import com.google.common.base.Preconditions;
24 import com.google.common.collect.Lists;
25 import com.google.common.util.concurrent.FutureCallback;
26 import com.google.common.util.concurrent.Futures;
27 import com.google.common.util.concurrent.ListenableFuture;
28 import com.google.protobuf.ByteString;
29 import com.google.protobuf.InvalidProtocolBufferException;
30 import java.util.Collection;
31 import java.util.HashMap;
32 import java.util.List;
33 import java.util.Map;
34 import java.util.concurrent.ExecutionException;
35 import java.util.concurrent.TimeUnit;
36 import javax.annotation.Nonnull;
37 import org.opendaylight.controller.cluster.DataPersistenceProvider;
38 import org.opendaylight.controller.cluster.common.actor.CommonConfig;
39 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
40 import org.opendaylight.controller.cluster.datastore.ShardCommitCoordinator.CohortEntry;
41 import org.opendaylight.controller.cluster.datastore.compat.BackwardsCompatibleThreePhaseCommitCohort;
42 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
43 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
44 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
45 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
46 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
47 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
48 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
49 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
50 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
51 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
52 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
53 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
54 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
55 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
56 import org.opendaylight.controller.cluster.datastore.messages.EnableNotification;
57 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
58 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
59 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
60 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
61 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
62 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
63 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
64 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
65 import org.opendaylight.controller.cluster.datastore.modification.Modification;
66 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
67 import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec;
68 import org.opendaylight.controller.cluster.raft.RaftActor;
69 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
70 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
71 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
72 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
73 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
74 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
75 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
76 import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
77 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
78 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
79 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionFactory;
80 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
81 import org.opendaylight.yangtools.concepts.ListenerRegistration;
82 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
83 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
84 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
85 import scala.concurrent.duration.Duration;
86 import scala.concurrent.duration.FiniteDuration;
87
88 /**
89  * A Shard represents a portion of the logical data tree <br/>
90  * <p>
91  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
92  * </p>
93  */
94 public class Shard extends RaftActor {
95
96     private static final Object COMMIT_TRANSACTION_REPLY = new CommitTransactionReply().toSerializable();
97
98     private static final Object TX_COMMIT_TIMEOUT_CHECK_MESSAGE = "txCommitTimeoutCheck";
99
100     public static final String DEFAULT_NAME = "default";
101
102     // The state of this Shard
103     private final InMemoryDOMDataStore store;
104
105     private final LoggingAdapter LOG =
106         Logging.getLogger(getContext().system(), this);
107
108     /// The name of this shard
109     private final ShardIdentifier name;
110
111     private final ShardStats shardMBean;
112
113     private final List<ActorSelection> dataChangeListeners =  Lists.newArrayList();
114
115     private final List<DelayedListenerRegistration> delayedListenerRegistrations =
116                                                                        Lists.newArrayList();
117
118     private final DatastoreContext datastoreContext;
119
120     private final DataPersistenceProvider dataPersistenceProvider;
121
122     private SchemaContext schemaContext;
123
124     private ActorRef createSnapshotTransaction;
125
126     private int createSnapshotTransactionCounter;
127
128     private final ShardCommitCoordinator commitCoordinator;
129
130     private final long transactionCommitTimeout;
131
132     private Cancellable txCommitTimeoutCheckSchedule;
133
134     /**
135      * Coordinates persistence recovery on startup.
136      */
137     private ShardRecoveryCoordinator recoveryCoordinator;
138     private List<Object> currentLogRecoveryBatch;
139
140     private final Map<String, DOMStoreTransactionChain> transactionChains = new HashMap<>();
141
142     protected Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
143             DatastoreContext datastoreContext, SchemaContext schemaContext) {
144         super(name.toString(), mapPeerAddresses(peerAddresses),
145                 Optional.of(datastoreContext.getShardRaftConfig()));
146
147         this.name = name;
148         this.datastoreContext = datastoreContext;
149         this.schemaContext = schemaContext;
150         this.dataPersistenceProvider = (datastoreContext.isPersistent()) ? new PersistentDataProvider() : new NonPersistentRaftDataProvider();
151
152         LOG.info("Shard created : {} persistent : {}", name, datastoreContext.isPersistent());
153
154         store = InMemoryDOMDataStoreFactory.create(name.toString(), null,
155                 datastoreContext.getDataStoreProperties());
156
157         if(schemaContext != null) {
158             store.onGlobalContextUpdated(schemaContext);
159         }
160
161         shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(),
162                 datastoreContext.getDataStoreMXBeanType());
163         shardMBean.setDataStoreExecutor(store.getDomStoreExecutor());
164         shardMBean.setNotificationManager(store.getDataChangeListenerNotificationManager());
165
166         if (isMetricsCaptureEnabled()) {
167             getContext().become(new MeteringBehavior(this));
168         }
169
170         commitCoordinator = new ShardCommitCoordinator(TimeUnit.SECONDS.convert(1, TimeUnit.MINUTES),
171                 datastoreContext.getShardTransactionCommitQueueCapacity());
172
173         transactionCommitTimeout = TimeUnit.MILLISECONDS.convert(
174                 datastoreContext.getShardTransactionCommitTimeoutInSeconds(), TimeUnit.SECONDS);
175     }
176
177     private static Map<String, String> mapPeerAddresses(
178         Map<ShardIdentifier, String> peerAddresses) {
179         Map<String, String> map = new HashMap<>();
180
181         for (Map.Entry<ShardIdentifier, String> entry : peerAddresses
182             .entrySet()) {
183             map.put(entry.getKey().toString(), entry.getValue());
184         }
185
186         return map;
187     }
188
189     public static Props props(final ShardIdentifier name,
190         final Map<ShardIdentifier, String> peerAddresses,
191         DatastoreContext datastoreContext, SchemaContext schemaContext) {
192         Preconditions.checkNotNull(name, "name should not be null");
193         Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
194         Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null");
195         Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
196
197         return Props.create(new ShardCreator(name, peerAddresses, datastoreContext, schemaContext));
198     }
199
200     @Override
201     public void postStop() {
202         super.postStop();
203
204         if(txCommitTimeoutCheckSchedule != null) {
205             txCommitTimeoutCheckSchedule.cancel();
206         }
207     }
208
209     @Override
210     public void onReceiveRecover(Object message) throws Exception {
211         if(LOG.isDebugEnabled()) {
212             LOG.debug("onReceiveRecover: Received message {} from {}",
213                 message.getClass().toString(),
214                 getSender());
215         }
216
217         if (message instanceof RecoveryFailure){
218             LOG.error(((RecoveryFailure) message).cause(), "Recovery failed because of this cause");
219
220             // Even though recovery failed, we still need to finish our recovery, eg send the
221             // ActorInitialized message and start the txCommitTimeoutCheckSchedule.
222             onRecoveryComplete();
223         } else {
224             super.onReceiveRecover(message);
225         }
226     }
227
228     @Override
229     public void onReceiveCommand(Object message) throws Exception {
230         if(LOG.isDebugEnabled()) {
231             LOG.debug("onReceiveCommand: Received message {} from {}", message, getSender());
232         }
233
234         if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
235             handleReadDataReply(message);
236         } else if (message.getClass().equals(CreateTransaction.SERIALIZABLE_CLASS)) {
237             handleCreateTransaction(message);
238         } else if(message instanceof ForwardedReadyTransaction) {
239             handleForwardedReadyTransaction((ForwardedReadyTransaction)message);
240         } else if(message.getClass().equals(CanCommitTransaction.SERIALIZABLE_CLASS)) {
241             handleCanCommitTransaction(CanCommitTransaction.fromSerializable(message));
242         } else if(message.getClass().equals(CommitTransaction.SERIALIZABLE_CLASS)) {
243             handleCommitTransaction(CommitTransaction.fromSerializable(message));
244         } else if(message.getClass().equals(AbortTransaction.SERIALIZABLE_CLASS)) {
245             handleAbortTransaction(AbortTransaction.fromSerializable(message));
246         } else if (message.getClass().equals(CloseTransactionChain.SERIALIZABLE_CLASS)){
247             closeTransactionChain(CloseTransactionChain.fromSerializable(message));
248         } else if (message instanceof RegisterChangeListener) {
249             registerChangeListener((RegisterChangeListener) message);
250         } else if (message instanceof UpdateSchemaContext) {
251             updateSchemaContext((UpdateSchemaContext) message);
252         } else if (message instanceof PeerAddressResolved) {
253             PeerAddressResolved resolved = (PeerAddressResolved) message;
254             setPeerAddress(resolved.getPeerId().toString(),
255                 resolved.getPeerAddress());
256         } else if(message.equals(TX_COMMIT_TIMEOUT_CHECK_MESSAGE)) {
257             handleTransactionCommitTimeoutCheck();
258         } else {
259             super.onReceiveCommand(message);
260         }
261     }
262
263     private void handleTransactionCommitTimeoutCheck() {
264         CohortEntry cohortEntry = commitCoordinator.getCurrentCohortEntry();
265         if(cohortEntry != null) {
266             long elapsed = System.currentTimeMillis() - cohortEntry.getLastAccessTime();
267             if(elapsed > transactionCommitTimeout) {
268                 LOG.warning("Current transaction {} has timed out after {} ms - aborting",
269                         cohortEntry.getTransactionID(), transactionCommitTimeout);
270
271                 doAbortTransaction(cohortEntry.getTransactionID(), null);
272             }
273         }
274     }
275
276     private void handleCommitTransaction(CommitTransaction commit) {
277         final String transactionID = commit.getTransactionID();
278
279         LOG.debug("Committing transaction {}", transactionID);
280
281         // Get the current in-progress cohort entry in the commitCoordinator if it corresponds to
282         // this transaction.
283         final CohortEntry cohortEntry = commitCoordinator.getCohortEntryIfCurrent(transactionID);
284         if(cohortEntry == null) {
285             // We're not the current Tx - the Tx was likely expired b/c it took too long in
286             // between the canCommit and commit messages.
287             IllegalStateException ex = new IllegalStateException(
288                     String.format("Cannot commit transaction %s - it is not the current transaction",
289                             transactionID));
290             LOG.error(ex.getMessage());
291             shardMBean.incrementFailedTransactionsCount();
292             getSender().tell(new akka.actor.Status.Failure(ex), getSelf());
293             return;
294         }
295
296         // We perform the preCommit phase here atomically with the commit phase. This is an
297         // optimization to eliminate the overhead of an extra preCommit message. We lose front-end
298         // coordination of preCommit across shards in case of failure but preCommit should not
299         // normally fail since we ensure only one concurrent 3-phase commit.
300
301         try {
302             // We block on the future here so we don't have to worry about possibly accessing our
303             // state on a different thread outside of our dispatcher. Also, the data store
304             // currently uses a same thread executor anyway.
305             cohortEntry.getCohort().preCommit().get();
306
307             Shard.this.persistData(getSender(), transactionID,
308                     new CompositeModificationPayload(cohortEntry.getModification().toSerializable()));
309         } catch (InterruptedException | ExecutionException e) {
310             LOG.error(e, "An exception occurred while preCommitting transaction {}",
311                     cohortEntry.getTransactionID());
312             shardMBean.incrementFailedTransactionsCount();
313             getSender().tell(new akka.actor.Status.Failure(e), getSelf());
314         }
315
316         cohortEntry.updateLastAccessTime();
317     }
318
319     private void finishCommit(@Nonnull final ActorRef sender, final @Nonnull String transactionID) {
320         // With persistence enabled, this method is called via applyState by the leader strategy
321         // after the commit has been replicated to a majority of the followers.
322
323         CohortEntry cohortEntry = commitCoordinator.getCohortEntryIfCurrent(transactionID);
324         if(cohortEntry == null) {
325             // The transaction is no longer the current commit. This can happen if the transaction
326             // was aborted prior, most likely due to timeout in the front-end. We need to finish
327             // committing the transaction though since it was successfully persisted and replicated
328             // however we can't use the original cohort b/c it was already preCommitted and may
329             // conflict with the current commit or may have been aborted so we commit with a new
330             // transaction.
331             cohortEntry = commitCoordinator.getAndRemoveCohortEntry(transactionID);
332             if(cohortEntry != null) {
333                 commitWithNewTransaction(cohortEntry.getModification());
334                 sender.tell(COMMIT_TRANSACTION_REPLY, getSelf());
335             } else {
336                 // This really shouldn't happen - it likely means that persistence or replication
337                 // took so long to complete such that the cohort entry was expired from the cache.
338                 IllegalStateException ex = new IllegalStateException(
339                         String.format("Could not finish committing transaction %s - no CohortEntry found",
340                                 transactionID));
341                 LOG.error(ex.getMessage());
342                 sender.tell(new akka.actor.Status.Failure(ex), getSelf());
343             }
344
345             return;
346         }
347
348         LOG.debug("Finishing commit for transaction {}", cohortEntry.getTransactionID());
349
350         try {
351             // We block on the future here so we don't have to worry about possibly accessing our
352             // state on a different thread outside of our dispatcher. Also, the data store
353             // currently uses a same thread executor anyway.
354             cohortEntry.getCohort().commit().get();
355
356             sender.tell(COMMIT_TRANSACTION_REPLY, getSelf());
357
358             shardMBean.incrementCommittedTransactionCount();
359             shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
360
361         } catch (InterruptedException | ExecutionException e) {
362             sender.tell(new akka.actor.Status.Failure(e), getSelf());
363
364             LOG.error(e, "An exception occurred while committing transaction {}", transactionID);
365             shardMBean.incrementFailedTransactionsCount();
366         }
367
368         commitCoordinator.currentTransactionComplete(transactionID, true);
369     }
370
371     private void handleCanCommitTransaction(CanCommitTransaction canCommit) {
372         LOG.debug("Can committing transaction {}", canCommit.getTransactionID());
373         commitCoordinator.handleCanCommit(canCommit, getSender(), self());
374     }
375
376     private void handleForwardedReadyTransaction(ForwardedReadyTransaction ready) {
377         LOG.debug("Readying transaction {}, client version {}", ready.getTransactionID(),
378                 ready.getTxnClientVersion());
379
380         // This message is forwarded by the ShardTransaction on ready. We cache the cohort in the
381         // commitCoordinator in preparation for the subsequent three phase commit initiated by
382         // the front-end.
383         commitCoordinator.transactionReady(ready.getTransactionID(), ready.getCohort(),
384                 ready.getModification());
385
386         // Return our actor path as we'll handle the three phase commit, except if the Tx client
387         // version < 1 (Helium-1 version). This means the Tx was initiated by a base Helium version
388         // node. In that case, the subsequent 3-phase commit messages won't contain the
389         // transactionId so to maintain backwards compatibility, we create a separate cohort actor
390         // to provide the compatible behavior.
391         ActorRef replyActorPath = self();
392         if(ready.getTxnClientVersion() < CreateTransaction.HELIUM_1_VERSION) {
393             LOG.debug("Creating BackwardsCompatibleThreePhaseCommitCohort");
394             replyActorPath = getContext().actorOf(BackwardsCompatibleThreePhaseCommitCohort.props(
395                     ready.getTransactionID()));
396         }
397
398         ReadyTransactionReply readyTransactionReply = new ReadyTransactionReply(
399                 Serialization.serializedActorPath(replyActorPath));
400         getSender().tell(ready.isReturnSerialized() ? readyTransactionReply.toSerializable() :
401                 readyTransactionReply, getSelf());
402     }
403
404     private void handleAbortTransaction(AbortTransaction abort) {
405         doAbortTransaction(abort.getTransactionID(), getSender());
406     }
407
408     private void doAbortTransaction(String transactionID, final ActorRef sender) {
409         final CohortEntry cohortEntry = commitCoordinator.getCohortEntryIfCurrent(transactionID);
410         if(cohortEntry != null) {
411             LOG.debug("Aborting transaction {}", transactionID);
412
413             // We don't remove the cached cohort entry here (ie pass false) in case the Tx was
414             // aborted during replication in which case we may still commit locally if replication
415             // succeeds.
416             commitCoordinator.currentTransactionComplete(transactionID, false);
417
418             final ListenableFuture<Void> future = cohortEntry.getCohort().abort();
419             final ActorRef self = getSelf();
420
421             Futures.addCallback(future, new FutureCallback<Void>() {
422                 @Override
423                 public void onSuccess(Void v) {
424                     shardMBean.incrementAbortTransactionsCount();
425
426                     if(sender != null) {
427                         sender.tell(new AbortTransactionReply().toSerializable(), self);
428                     }
429                 }
430
431                 @Override
432                 public void onFailure(Throwable t) {
433                     LOG.error(t, "An exception happened during abort");
434
435                     if(sender != null) {
436                         sender.tell(new akka.actor.Status.Failure(t), self);
437                     }
438                 }
439             });
440         }
441     }
442
443     private void handleCreateTransaction(Object message) {
444         if (isLeader()) {
445             createTransaction(CreateTransaction.fromSerializable(message));
446         } else if (getLeader() != null) {
447             getLeader().forward(message, getContext());
448         } else {
449             getSender().tell(new akka.actor.Status.Failure(new NoShardLeaderException(
450                 "Could not find shard leader so transaction cannot be created. This typically happens" +
451                 " when the system is coming up or recovering and a leader is being elected. Try again" +
452                 " later.")), getSelf());
453         }
454     }
455
456     private void handleReadDataReply(Object message) {
457         // This must be for install snapshot. Don't want to open this up and trigger
458         // deSerialization
459
460         self().tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)),
461                 self());
462
463         createSnapshotTransaction = null;
464
465         // Send a PoisonPill instead of sending close transaction because we do not really need
466         // a response
467         getSender().tell(PoisonPill.getInstance(), self());
468     }
469
470     private void closeTransactionChain(CloseTransactionChain closeTransactionChain) {
471         DOMStoreTransactionChain chain =
472             transactionChains.remove(closeTransactionChain.getTransactionChainId());
473
474         if(chain != null) {
475             chain.close();
476         }
477     }
478
479     private ActorRef createTypedTransactionActor(int transactionType,
480             ShardTransactionIdentifier transactionId, String transactionChainId, int clientVersion ) {
481
482         DOMStoreTransactionFactory factory = store;
483
484         if(!transactionChainId.isEmpty()) {
485             factory = transactionChains.get(transactionChainId);
486             if(factory == null){
487                 DOMStoreTransactionChain transactionChain = store.createTransactionChain();
488                 transactionChains.put(transactionChainId, transactionChain);
489                 factory = transactionChain;
490             }
491         }
492
493         if(this.schemaContext == null) {
494             throw new IllegalStateException("SchemaContext is not set");
495         }
496
497         if (transactionType == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
498
499             shardMBean.incrementReadOnlyTransactionCount();
500
501             return getContext().actorOf(
502                 ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(),
503                         schemaContext,datastoreContext, shardMBean,
504                         transactionId.getRemoteTransactionId(), clientVersion),
505                         transactionId.toString());
506
507         } else if (transactionType == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
508
509             shardMBean.incrementReadWriteTransactionCount();
510
511             return getContext().actorOf(
512                 ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(),
513                         schemaContext, datastoreContext, shardMBean,
514                         transactionId.getRemoteTransactionId(), clientVersion),
515                         transactionId.toString());
516
517
518         } else if (transactionType == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
519
520             shardMBean.incrementWriteOnlyTransactionCount();
521
522             return getContext().actorOf(
523                 ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(),
524                         schemaContext, datastoreContext, shardMBean,
525                         transactionId.getRemoteTransactionId(), clientVersion),
526                         transactionId.toString());
527         } else {
528             throw new IllegalArgumentException(
529                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
530                     + transactionType);
531         }
532     }
533
534     private void createTransaction(CreateTransaction createTransaction) {
535         try {
536             ActorRef transactionActor = createTransaction(createTransaction.getTransactionType(),
537                 createTransaction.getTransactionId(), createTransaction.getTransactionChainId(),
538                 createTransaction.getVersion());
539
540             getSender().tell(new CreateTransactionReply(Serialization.serializedActorPath(transactionActor),
541                     createTransaction.getTransactionId()).toSerializable(), getSelf());
542         } catch (Exception e) {
543             getSender().tell(new akka.actor.Status.Failure(e), getSelf());
544         }
545     }
546
547     private ActorRef createTransaction(int transactionType, String remoteTransactionId,
548             String transactionChainId, int clientVersion) {
549
550         ShardTransactionIdentifier transactionId =
551             ShardTransactionIdentifier.builder()
552                 .remoteTransactionId(remoteTransactionId)
553                 .build();
554
555         if(LOG.isDebugEnabled()) {
556             LOG.debug("Creating transaction : {} ", transactionId);
557         }
558
559         ActorRef transactionActor = createTypedTransactionActor(transactionType, transactionId,
560                 transactionChainId, clientVersion);
561
562         return transactionActor;
563     }
564
565     private void syncCommitTransaction(DOMStoreWriteTransaction transaction)
566         throws ExecutionException, InterruptedException {
567         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
568         commitCohort.preCommit().get();
569         commitCohort.commit().get();
570     }
571
572     private void commitWithNewTransaction(Modification modification) {
573         DOMStoreWriteTransaction tx = store.newWriteOnlyTransaction();
574         modification.apply(tx);
575         try {
576             syncCommitTransaction(tx);
577             shardMBean.incrementCommittedTransactionCount();
578             shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
579         } catch (InterruptedException | ExecutionException e) {
580             shardMBean.incrementFailedTransactionsCount();
581             LOG.error(e, "Failed to commit");
582         }
583     }
584
585     private void updateSchemaContext(UpdateSchemaContext message) {
586         this.schemaContext = message.getSchemaContext();
587         updateSchemaContext(message.getSchemaContext());
588         store.onGlobalContextUpdated(message.getSchemaContext());
589     }
590
591     @VisibleForTesting
592     void updateSchemaContext(SchemaContext schemaContext) {
593         store.onGlobalContextUpdated(schemaContext);
594     }
595
596     private void registerChangeListener(RegisterChangeListener registerChangeListener) {
597
598         LOG.debug("registerDataChangeListener for {}", registerChangeListener.getPath());
599
600         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
601                                                      NormalizedNode<?, ?>>> registration;
602         if(isLeader()) {
603             registration = doChangeListenerRegistration(registerChangeListener);
604         } else {
605             LOG.debug("Shard is not the leader - delaying registration");
606
607             DelayedListenerRegistration delayedReg =
608                     new DelayedListenerRegistration(registerChangeListener);
609             delayedListenerRegistrations.add(delayedReg);
610             registration = delayedReg;
611         }
612
613         ActorRef listenerRegistration = getContext().actorOf(
614                 DataChangeListenerRegistration.props(registration));
615
616         LOG.debug("registerDataChangeListener sending reply, listenerRegistrationPath = {} ",
617                     listenerRegistration.path());
618
619         getSender().tell(new RegisterChangeListenerReply(listenerRegistration.path()), getSelf());
620     }
621
622     private ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
623                                                NormalizedNode<?, ?>>> doChangeListenerRegistration(
624             RegisterChangeListener registerChangeListener) {
625
626         ActorSelection dataChangeListenerPath = getContext().system().actorSelection(
627                 registerChangeListener.getDataChangeListenerPath());
628
629         // Notify the listener if notifications should be enabled or not
630         // If this shard is the leader then it will enable notifications else
631         // it will not
632         dataChangeListenerPath.tell(new EnableNotification(true), getSelf());
633
634         // Now store a reference to the data change listener so it can be notified
635         // at a later point if notifications should be enabled or disabled
636         dataChangeListeners.add(dataChangeListenerPath);
637
638         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> listener =
639                 new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
640
641         LOG.debug("Registering for path {}", registerChangeListener.getPath());
642
643         return store.registerChangeListener(registerChangeListener.getPath(), listener,
644                 registerChangeListener.getScope());
645     }
646
647     private boolean isMetricsCaptureEnabled(){
648         CommonConfig config = new CommonConfig(getContext().system().settings().config());
649         return config.isMetricCaptureEnabled();
650     }
651
652     @Override
653     protected
654     void startLogRecoveryBatch(int maxBatchSize) {
655         currentLogRecoveryBatch = Lists.newArrayListWithCapacity(maxBatchSize);
656
657         if(LOG.isDebugEnabled()) {
658             LOG.debug("{} : starting log recovery batch with max size {}", persistenceId(), maxBatchSize);
659         }
660     }
661
662     @Override
663     protected void appendRecoveredLogEntry(Payload data) {
664         if (data instanceof CompositeModificationPayload) {
665             currentLogRecoveryBatch.add(((CompositeModificationPayload) data).getModification());
666         } else {
667             LOG.error("Unknown state received {} during recovery", data);
668         }
669     }
670
671     @Override
672     protected void applyRecoverySnapshot(ByteString snapshot) {
673         if(recoveryCoordinator == null) {
674             recoveryCoordinator = new ShardRecoveryCoordinator(persistenceId(), schemaContext);
675         }
676
677         recoveryCoordinator.submit(snapshot, store.newWriteOnlyTransaction());
678
679         if(LOG.isDebugEnabled()) {
680             LOG.debug("{} : submitted recovery sbapshot", persistenceId());
681         }
682     }
683
684     @Override
685     protected void applyCurrentLogRecoveryBatch() {
686         if(recoveryCoordinator == null) {
687             recoveryCoordinator = new ShardRecoveryCoordinator(persistenceId(), schemaContext);
688         }
689
690         recoveryCoordinator.submit(currentLogRecoveryBatch, store.newWriteOnlyTransaction());
691
692         if(LOG.isDebugEnabled()) {
693             LOG.debug("{} : submitted log recovery batch with size {}", persistenceId(),
694                     currentLogRecoveryBatch.size());
695         }
696     }
697
698     @Override
699     protected void onRecoveryComplete() {
700         if(recoveryCoordinator != null) {
701             Collection<DOMStoreWriteTransaction> txList = recoveryCoordinator.getTransactions();
702
703             if(LOG.isDebugEnabled()) {
704                 LOG.debug("{} : recovery complete - committing {} Tx's", persistenceId(), txList.size());
705             }
706
707             for(DOMStoreWriteTransaction tx: txList) {
708                 try {
709                     syncCommitTransaction(tx);
710                     shardMBean.incrementCommittedTransactionCount();
711                 } catch (InterruptedException | ExecutionException e) {
712                     shardMBean.incrementFailedTransactionsCount();
713                     LOG.error(e, "Failed to commit");
714                 }
715             }
716         }
717
718         recoveryCoordinator = null;
719         currentLogRecoveryBatch = null;
720         updateJournalStats();
721
722         //notify shard manager
723         getContext().parent().tell(new ActorInitialized(), getSelf());
724
725         // Being paranoid here - this method should only be called once but just in case...
726         if(txCommitTimeoutCheckSchedule == null) {
727             // Schedule a message to be periodically sent to check if the current in-progress
728             // transaction should be expired and aborted.
729             FiniteDuration period = Duration.create(transactionCommitTimeout / 3, TimeUnit.MILLISECONDS);
730             txCommitTimeoutCheckSchedule = getContext().system().scheduler().schedule(
731                     period, period, getSelf(),
732                     TX_COMMIT_TIMEOUT_CHECK_MESSAGE, getContext().dispatcher(), ActorRef.noSender());
733         }
734     }
735
736     @Override
737     protected void applyState(ActorRef clientActor, String identifier, Object data) {
738
739         if (data instanceof CompositeModificationPayload) {
740             Object modification = ((CompositeModificationPayload) data).getModification();
741
742             if(modification == null) {
743                 LOG.error(
744                      "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
745                      identifier, clientActor != null ? clientActor.path().toString() : null);
746             } else if(clientActor == null) {
747                 // There's no clientActor to which to send a commit reply so we must be applying
748                 // replicated state from the leader.
749                 commitWithNewTransaction(MutableCompositeModification.fromSerializable(
750                         modification, schemaContext));
751             } else {
752                 // This must be the OK to commit after replication consensus.
753                 finishCommit(clientActor, identifier);
754             }
755         } else {
756             LOG.error("Unknown state received {} Class loader = {} CompositeNodeMod.ClassLoader = {}",
757                     data, data.getClass().getClassLoader(),
758                     CompositeModificationPayload.class.getClassLoader());
759         }
760
761         updateJournalStats();
762
763     }
764
765     private void updateJournalStats() {
766         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
767
768         if (lastLogEntry != null) {
769             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
770             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
771         }
772
773         shardMBean.setCommitIndex(getCommitIndex());
774         shardMBean.setLastApplied(getLastApplied());
775     }
776
777     @Override
778     protected void createSnapshot() {
779         if (createSnapshotTransaction == null) {
780
781             // Create a transaction. We are really going to treat the transaction as a worker
782             // so that this actor does not get block building the snapshot
783             createSnapshotTransaction = createTransaction(
784                 TransactionProxy.TransactionType.READ_ONLY.ordinal(),
785                 "createSnapshot" + ++createSnapshotTransactionCounter, "",
786                 CreateTransaction.CURRENT_VERSION);
787
788             createSnapshotTransaction.tell(
789                 new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
790
791         }
792     }
793
794     @VisibleForTesting
795     @Override
796     protected void applySnapshot(ByteString snapshot) {
797         // Since this will be done only on Recovery or when this actor is a Follower
798         // we can safely commit everything in here. We not need to worry about event notifications
799         // as they would have already been disabled on the follower
800
801         LOG.info("Applying snapshot");
802         try {
803             DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
804             NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
805             NormalizedNode<?, ?> node = new NormalizedNodeToNodeCodec(schemaContext)
806                 .decode(serializedNode);
807
808             // delete everything first
809             transaction.delete(YangInstanceIdentifier.builder().build());
810
811             // Add everything from the remote node back
812             transaction.write(YangInstanceIdentifier.builder().build(), node);
813             syncCommitTransaction(transaction);
814         } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
815             LOG.error(e, "An exception occurred when applying snapshot");
816         } finally {
817             LOG.info("Done applying snapshot");
818         }
819     }
820
821     @Override
822     protected void onStateChanged() {
823         boolean isLeader = isLeader();
824         for (ActorSelection dataChangeListener : dataChangeListeners) {
825             dataChangeListener.tell(new EnableNotification(isLeader), getSelf());
826         }
827
828         if(isLeader) {
829             for(DelayedListenerRegistration reg: delayedListenerRegistrations) {
830                 if(!reg.isClosed()) {
831                     reg.setDelegate(doChangeListenerRegistration(reg.getRegisterChangeListener()));
832                 }
833             }
834
835             delayedListenerRegistrations.clear();
836         }
837
838         shardMBean.setRaftState(getRaftState().name());
839         shardMBean.setCurrentTerm(getCurrentTerm());
840
841         // If this actor is no longer the leader close all the transaction chains
842         if(!isLeader){
843             for(Map.Entry<String, DOMStoreTransactionChain> entry : transactionChains.entrySet()){
844                 if(LOG.isDebugEnabled()) {
845                     LOG.debug(
846                         "onStateChanged: Closing transaction chain {} because shard {} is no longer the leader",
847                         entry.getKey(), getId());
848                 }
849                 entry.getValue().close();
850             }
851
852             transactionChains.clear();
853         }
854     }
855
856     @Override
857     protected DataPersistenceProvider persistence() {
858         return dataPersistenceProvider;
859     }
860
861     @Override protected void onLeaderChanged(String oldLeader, String newLeader) {
862         shardMBean.setLeader(newLeader);
863     }
864
865     @Override public String persistenceId() {
866         return this.name.toString();
867     }
868
869     @VisibleForTesting
870     DataPersistenceProvider getDataPersistenceProvider() {
871         return dataPersistenceProvider;
872     }
873
874     private static class ShardCreator implements Creator<Shard> {
875
876         private static final long serialVersionUID = 1L;
877
878         final ShardIdentifier name;
879         final Map<ShardIdentifier, String> peerAddresses;
880         final DatastoreContext datastoreContext;
881         final SchemaContext schemaContext;
882
883         ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
884                 DatastoreContext datastoreContext, SchemaContext schemaContext) {
885             this.name = name;
886             this.peerAddresses = peerAddresses;
887             this.datastoreContext = datastoreContext;
888             this.schemaContext = schemaContext;
889         }
890
891         @Override
892         public Shard create() throws Exception {
893             return new Shard(name, peerAddresses, datastoreContext, schemaContext);
894         }
895     }
896
897     @VisibleForTesting
898     InMemoryDOMDataStore getDataStore() {
899         return store;
900     }
901
902     @VisibleForTesting
903     ShardStats getShardMBean() {
904         return shardMBean;
905     }
906
907     private static class DelayedListenerRegistration implements
908         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>> {
909
910         private volatile boolean closed;
911
912         private final RegisterChangeListener registerChangeListener;
913
914         private volatile ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
915                                                              NormalizedNode<?, ?>>> delegate;
916
917         DelayedListenerRegistration(RegisterChangeListener registerChangeListener) {
918             this.registerChangeListener = registerChangeListener;
919         }
920
921         void setDelegate( ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
922                                             NormalizedNode<?, ?>>> registration) {
923             this.delegate = registration;
924         }
925
926         boolean isClosed() {
927             return closed;
928         }
929
930         RegisterChangeListener getRegisterChangeListener() {
931             return registerChangeListener;
932         }
933
934         @Override
935         public AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> getInstance() {
936             return delegate != null ? delegate.getInstance() : null;
937         }
938
939         @Override
940         public void close() {
941             closed = true;
942             if(delegate != null) {
943                 delegate.close();
944             }
945         }
946     }
947 }