BUG 2134 : Make persistence configurable at the datastore level
[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 org.opendaylight.controller.cluster.DataPersistenceProvider;
31 import org.opendaylight.controller.cluster.common.actor.CommonConfig;
32 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
33 import org.opendaylight.controller.cluster.datastore.ShardCommitCoordinator.CohortEntry;
34 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
35 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
36 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
37 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
38 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
39 import org.opendaylight.controller.cluster.datastore.messages.AbortTransaction;
40 import org.opendaylight.controller.cluster.datastore.messages.AbortTransactionReply;
41 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
42 import org.opendaylight.controller.cluster.datastore.messages.CanCommitTransaction;
43 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
44 import org.opendaylight.controller.cluster.datastore.messages.CommitTransaction;
45 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
46 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
47 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
48 import org.opendaylight.controller.cluster.datastore.messages.EnableNotification;
49 import org.opendaylight.controller.cluster.datastore.messages.ForwardedReadyTransaction;
50 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
51 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
52 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
53 import org.opendaylight.controller.cluster.datastore.messages.ReadyTransactionReply;
54 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
55 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
56 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
57 import org.opendaylight.controller.cluster.datastore.modification.Modification;
58 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
59 import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec;
60 import org.opendaylight.controller.cluster.raft.RaftActor;
61 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
62 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
63 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
64 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
65 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
66 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
67 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
68 import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
69 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
70 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
71 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionFactory;
72 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
73 import org.opendaylight.yangtools.concepts.ListenerRegistration;
74 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
75 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
76 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
77 import scala.concurrent.duration.Duration;
78 import scala.concurrent.duration.FiniteDuration;
79
80 import javax.annotation.Nonnull;
81 import java.util.Collection;
82 import java.util.HashMap;
83 import java.util.List;
84 import java.util.Map;
85 import java.util.concurrent.ExecutionException;
86 import java.util.concurrent.TimeUnit;
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 {}", ready.getTransactionID());
378
379         // This message is forwarded by the ShardTransaction on ready. We cache the cohort in the
380         // commitCoordinator in preparation for the subsequent three phase commit initiated by
381         // the front-end.
382         commitCoordinator.transactionReady(ready.getTransactionID(), ready.getCohort(),
383                 ready.getModification());
384
385         // Return our actor path as we'll handle the three phase commit.
386         ReadyTransactionReply readyTransactionReply =
387             new ReadyTransactionReply(Serialization.serializedActorPath(self()));
388         getSender().tell(
389             ready.isReturnSerialized() ? readyTransactionReply.toSerializable() : readyTransactionReply,
390             getSelf());
391     }
392
393     private void handleAbortTransaction(AbortTransaction abort) {
394         doAbortTransaction(abort.getTransactionID(), getSender());
395     }
396
397     private void doAbortTransaction(String transactionID, final ActorRef sender) {
398         final CohortEntry cohortEntry = commitCoordinator.getCohortEntryIfCurrent(transactionID);
399         if(cohortEntry != null) {
400             LOG.debug("Aborting transaction {}", transactionID);
401
402             // We don't remove the cached cohort entry here (ie pass false) in case the Tx was
403             // aborted during replication in which case we may still commit locally if replication
404             // succeeds.
405             commitCoordinator.currentTransactionComplete(transactionID, false);
406
407             final ListenableFuture<Void> future = cohortEntry.getCohort().abort();
408             final ActorRef self = getSelf();
409
410             Futures.addCallback(future, new FutureCallback<Void>() {
411                 @Override
412                 public void onSuccess(Void v) {
413                     shardMBean.incrementAbortTransactionsCount();
414
415                     if(sender != null) {
416                         sender.tell(new AbortTransactionReply().toSerializable(), self);
417                     }
418                 }
419
420                 @Override
421                 public void onFailure(Throwable t) {
422                     LOG.error(t, "An exception happened during abort");
423
424                     if(sender != null) {
425                         sender.tell(new akka.actor.Status.Failure(t), self);
426                     }
427                 }
428             });
429         }
430     }
431
432     private void handleCreateTransaction(Object message) {
433         if (isLeader()) {
434             createTransaction(CreateTransaction.fromSerializable(message));
435         } else if (getLeader() != null) {
436             getLeader().forward(message, getContext());
437         } else {
438             getSender().tell(new akka.actor.Status.Failure(new NoShardLeaderException(
439                 "Could not find shard leader so transaction cannot be created. This typically happens" +
440                 " when the system is coming up or recovering and a leader is being elected. Try again" +
441                 " later.")), getSelf());
442         }
443     }
444
445     private void handleReadDataReply(Object message) {
446         // This must be for install snapshot. Don't want to open this up and trigger
447         // deSerialization
448
449         self().tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)),
450                 self());
451
452         createSnapshotTransaction = null;
453
454         // Send a PoisonPill instead of sending close transaction because we do not really need
455         // a response
456         getSender().tell(PoisonPill.getInstance(), self());
457     }
458
459     private void closeTransactionChain(CloseTransactionChain closeTransactionChain) {
460         DOMStoreTransactionChain chain =
461             transactionChains.remove(closeTransactionChain.getTransactionChainId());
462
463         if(chain != null) {
464             chain.close();
465         }
466     }
467
468     private ActorRef createTypedTransactionActor(
469         int transactionType,
470         ShardTransactionIdentifier transactionId,
471         String transactionChainId ) {
472
473         DOMStoreTransactionFactory factory = store;
474
475         if(!transactionChainId.isEmpty()) {
476             factory = transactionChains.get(transactionChainId);
477             if(factory == null){
478                 DOMStoreTransactionChain transactionChain = store.createTransactionChain();
479                 transactionChains.put(transactionChainId, transactionChain);
480                 factory = transactionChain;
481             }
482         }
483
484         if(this.schemaContext == null){
485             throw new NullPointerException("schemaContext should not be null");
486         }
487
488         if (transactionType == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
489
490             shardMBean.incrementReadOnlyTransactionCount();
491
492             return getContext().actorOf(
493                 ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(),
494                         schemaContext,datastoreContext, shardMBean,
495                         transactionId.getRemoteTransactionId()), transactionId.toString());
496
497         } else if (transactionType == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
498
499             shardMBean.incrementReadWriteTransactionCount();
500
501             return getContext().actorOf(
502                 ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(),
503                         schemaContext, datastoreContext, shardMBean,
504                         transactionId.getRemoteTransactionId()), transactionId.toString());
505
506
507         } else if (transactionType == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
508
509             shardMBean.incrementWriteOnlyTransactionCount();
510
511             return getContext().actorOf(
512                 ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(),
513                         schemaContext, datastoreContext, shardMBean,
514                         transactionId.getRemoteTransactionId()), transactionId.toString());
515         } else {
516             throw new IllegalArgumentException(
517                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
518                     + transactionType);
519         }
520     }
521
522     private void createTransaction(CreateTransaction createTransaction) {
523         createTransaction(createTransaction.getTransactionType(),
524             createTransaction.getTransactionId(), createTransaction.getTransactionChainId());
525     }
526
527     private ActorRef createTransaction(int transactionType, String remoteTransactionId, String transactionChainId) {
528
529         ShardTransactionIdentifier transactionId =
530             ShardTransactionIdentifier.builder()
531                 .remoteTransactionId(remoteTransactionId)
532                 .build();
533         if(LOG.isDebugEnabled()) {
534             LOG.debug("Creating transaction : {} ", transactionId);
535         }
536         ActorRef transactionActor =
537             createTypedTransactionActor(transactionType, transactionId, transactionChainId);
538
539         getSender()
540             .tell(new CreateTransactionReply(
541                     Serialization.serializedActorPath(transactionActor),
542                     remoteTransactionId).toSerializable(),
543                 getSelf());
544
545         return transactionActor;
546     }
547
548     private void syncCommitTransaction(DOMStoreWriteTransaction transaction)
549         throws ExecutionException, InterruptedException {
550         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
551         commitCohort.preCommit().get();
552         commitCohort.commit().get();
553     }
554
555     private void commitWithNewTransaction(Modification modification) {
556         DOMStoreWriteTransaction tx = store.newWriteOnlyTransaction();
557         modification.apply(tx);
558         try {
559             syncCommitTransaction(tx);
560             shardMBean.incrementCommittedTransactionCount();
561             shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
562         } catch (InterruptedException | ExecutionException e) {
563             shardMBean.incrementFailedTransactionsCount();
564             LOG.error(e, "Failed to commit");
565         }
566     }
567
568     private void updateSchemaContext(UpdateSchemaContext message) {
569         this.schemaContext = message.getSchemaContext();
570         updateSchemaContext(message.getSchemaContext());
571         store.onGlobalContextUpdated(message.getSchemaContext());
572     }
573
574     @VisibleForTesting
575     void updateSchemaContext(SchemaContext schemaContext) {
576         store.onGlobalContextUpdated(schemaContext);
577     }
578
579     private void registerChangeListener(RegisterChangeListener registerChangeListener) {
580
581         LOG.debug("registerDataChangeListener for {}", registerChangeListener.getPath());
582
583         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
584                                                      NormalizedNode<?, ?>>> registration;
585         if(isLeader()) {
586             registration = doChangeListenerRegistration(registerChangeListener);
587         } else {
588             LOG.debug("Shard is not the leader - delaying registration");
589
590             DelayedListenerRegistration delayedReg =
591                     new DelayedListenerRegistration(registerChangeListener);
592             delayedListenerRegistrations.add(delayedReg);
593             registration = delayedReg;
594         }
595
596         ActorRef listenerRegistration = getContext().actorOf(
597                 DataChangeListenerRegistration.props(registration));
598
599         LOG.debug("registerDataChangeListener sending reply, listenerRegistrationPath = {} ",
600                     listenerRegistration.path());
601
602         getSender().tell(new RegisterChangeListenerReply(listenerRegistration.path()),getSelf());
603     }
604
605     private ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
606                                                NormalizedNode<?, ?>>> doChangeListenerRegistration(
607             RegisterChangeListener registerChangeListener) {
608
609         ActorSelection dataChangeListenerPath = getContext().system().actorSelection(
610                 registerChangeListener.getDataChangeListenerPath());
611
612         // Notify the listener if notifications should be enabled or not
613         // If this shard is the leader then it will enable notifications else
614         // it will not
615         dataChangeListenerPath.tell(new EnableNotification(true), getSelf());
616
617         // Now store a reference to the data change listener so it can be notified
618         // at a later point if notifications should be enabled or disabled
619         dataChangeListeners.add(dataChangeListenerPath);
620
621         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> listener =
622                 new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
623
624         LOG.debug("Registering for path {}", registerChangeListener.getPath());
625
626         return store.registerChangeListener(registerChangeListener.getPath(), listener,
627                 registerChangeListener.getScope());
628     }
629
630     private boolean isMetricsCaptureEnabled(){
631         CommonConfig config = new CommonConfig(getContext().system().settings().config());
632         return config.isMetricCaptureEnabled();
633     }
634
635     @Override
636     protected
637     void startLogRecoveryBatch(int maxBatchSize) {
638         currentLogRecoveryBatch = Lists.newArrayListWithCapacity(maxBatchSize);
639
640         if(LOG.isDebugEnabled()) {
641             LOG.debug("{} : starting log recovery batch with max size {}", persistenceId(), maxBatchSize);
642         }
643     }
644
645     @Override
646     protected void appendRecoveredLogEntry(Payload data) {
647         if (data instanceof CompositeModificationPayload) {
648             currentLogRecoveryBatch.add(((CompositeModificationPayload) data).getModification());
649         } else {
650             LOG.error("Unknown state received {} during recovery", data);
651         }
652     }
653
654     @Override
655     protected void applyRecoverySnapshot(ByteString snapshot) {
656         if(recoveryCoordinator == null) {
657             recoveryCoordinator = new ShardRecoveryCoordinator(persistenceId(), schemaContext);
658         }
659
660         recoveryCoordinator.submit(snapshot, store.newWriteOnlyTransaction());
661
662         if(LOG.isDebugEnabled()) {
663             LOG.debug("{} : submitted recovery sbapshot", persistenceId());
664         }
665     }
666
667     @Override
668     protected void applyCurrentLogRecoveryBatch() {
669         if(recoveryCoordinator == null) {
670             recoveryCoordinator = new ShardRecoveryCoordinator(persistenceId(), schemaContext);
671         }
672
673         recoveryCoordinator.submit(currentLogRecoveryBatch, store.newWriteOnlyTransaction());
674
675         if(LOG.isDebugEnabled()) {
676             LOG.debug("{} : submitted log recovery batch with size {}", persistenceId(),
677                     currentLogRecoveryBatch.size());
678         }
679     }
680
681     @Override
682     protected void onRecoveryComplete() {
683         if(recoveryCoordinator != null) {
684             Collection<DOMStoreWriteTransaction> txList = recoveryCoordinator.getTransactions();
685
686             if(LOG.isDebugEnabled()) {
687                 LOG.debug("{} : recovery complete - committing {} Tx's", persistenceId(), txList.size());
688             }
689
690             for(DOMStoreWriteTransaction tx: txList) {
691                 try {
692                     syncCommitTransaction(tx);
693                     shardMBean.incrementCommittedTransactionCount();
694                 } catch (InterruptedException | ExecutionException e) {
695                     shardMBean.incrementFailedTransactionsCount();
696                     LOG.error(e, "Failed to commit");
697                 }
698             }
699         }
700
701         recoveryCoordinator = null;
702         currentLogRecoveryBatch = null;
703         updateJournalStats();
704
705         //notify shard manager
706         getContext().parent().tell(new ActorInitialized(), getSelf());
707
708         // Being paranoid here - this method should only be called once but just in case...
709         if(txCommitTimeoutCheckSchedule == null) {
710             // Schedule a message to be periodically sent to check if the current in-progress
711             // transaction should be expired and aborted.
712             FiniteDuration period = Duration.create(transactionCommitTimeout / 3, TimeUnit.MILLISECONDS);
713             txCommitTimeoutCheckSchedule = getContext().system().scheduler().schedule(
714                     period, period, getSelf(),
715                     TX_COMMIT_TIMEOUT_CHECK_MESSAGE, getContext().dispatcher(), ActorRef.noSender());
716         }
717     }
718
719     @Override
720     protected void applyState(ActorRef clientActor, String identifier, Object data) {
721
722         if (data instanceof CompositeModificationPayload) {
723             Object modification = ((CompositeModificationPayload) data).getModification();
724
725             if(modification == null) {
726                 LOG.error(
727                      "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
728                      identifier, clientActor != null ? clientActor.path().toString() : null);
729             } else if(clientActor == null) {
730                 // There's no clientActor to which to send a commit reply so we must be applying
731                 // replicated state from the leader.
732                 commitWithNewTransaction(MutableCompositeModification.fromSerializable(
733                         modification, schemaContext));
734             } else {
735                 // This must be the OK to commit after replication consensus.
736                 finishCommit(clientActor, identifier);
737             }
738         } else {
739             LOG.error("Unknown state received {} Class loader = {} CompositeNodeMod.ClassLoader = {}",
740                     data, data.getClass().getClassLoader(),
741                     CompositeModificationPayload.class.getClassLoader());
742         }
743
744         updateJournalStats();
745
746     }
747
748     private void updateJournalStats() {
749         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
750
751         if (lastLogEntry != null) {
752             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
753             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
754         }
755
756         shardMBean.setCommitIndex(getCommitIndex());
757         shardMBean.setLastApplied(getLastApplied());
758     }
759
760     @Override
761     protected void createSnapshot() {
762         if (createSnapshotTransaction == null) {
763
764             // Create a transaction. We are really going to treat the transaction as a worker
765             // so that this actor does not get block building the snapshot
766             createSnapshotTransaction = createTransaction(
767                 TransactionProxy.TransactionType.READ_ONLY.ordinal(),
768                 "createSnapshot" + ++createSnapshotTransactionCounter, "");
769
770             createSnapshotTransaction.tell(
771                 new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
772
773         }
774     }
775
776     @VisibleForTesting
777     @Override
778     protected void applySnapshot(ByteString snapshot) {
779         // Since this will be done only on Recovery or when this actor is a Follower
780         // we can safely commit everything in here. We not need to worry about event notifications
781         // as they would have already been disabled on the follower
782
783         LOG.info("Applying snapshot");
784         try {
785             DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
786             NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
787             NormalizedNode<?, ?> node = new NormalizedNodeToNodeCodec(schemaContext)
788                 .decode(serializedNode);
789
790             // delete everything first
791             transaction.delete(YangInstanceIdentifier.builder().build());
792
793             // Add everything from the remote node back
794             transaction.write(YangInstanceIdentifier.builder().build(), node);
795             syncCommitTransaction(transaction);
796         } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
797             LOG.error(e, "An exception occurred when applying snapshot");
798         } finally {
799             LOG.info("Done applying snapshot");
800         }
801     }
802
803     @Override
804     protected void onStateChanged() {
805         boolean isLeader = isLeader();
806         for (ActorSelection dataChangeListener : dataChangeListeners) {
807             dataChangeListener.tell(new EnableNotification(isLeader), getSelf());
808         }
809
810         if(isLeader) {
811             for(DelayedListenerRegistration reg: delayedListenerRegistrations) {
812                 if(!reg.isClosed()) {
813                     reg.setDelegate(doChangeListenerRegistration(reg.getRegisterChangeListener()));
814                 }
815             }
816
817             delayedListenerRegistrations.clear();
818         }
819
820         shardMBean.setRaftState(getRaftState().name());
821         shardMBean.setCurrentTerm(getCurrentTerm());
822
823         // If this actor is no longer the leader close all the transaction chains
824         if(!isLeader){
825             for(Map.Entry<String, DOMStoreTransactionChain> entry : transactionChains.entrySet()){
826                 if(LOG.isDebugEnabled()) {
827                     LOG.debug(
828                         "onStateChanged: Closing transaction chain {} because shard {} is no longer the leader",
829                         entry.getKey(), getId());
830                 }
831                 entry.getValue().close();
832             }
833
834             transactionChains.clear();
835         }
836     }
837
838     @Override
839     protected DataPersistenceProvider persistence() {
840         return dataPersistenceProvider;
841     }
842
843     @Override protected void onLeaderChanged(String oldLeader, String newLeader) {
844         shardMBean.setLeader(newLeader);
845     }
846
847     @Override public String persistenceId() {
848         return this.name.toString();
849     }
850
851     @VisibleForTesting
852     DataPersistenceProvider getDataPersistenceProvider() {
853         return dataPersistenceProvider;
854     }
855
856     private static class ShardCreator implements Creator<Shard> {
857
858         private static final long serialVersionUID = 1L;
859
860         final ShardIdentifier name;
861         final Map<ShardIdentifier, String> peerAddresses;
862         final DatastoreContext datastoreContext;
863         final SchemaContext schemaContext;
864
865         ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
866                 DatastoreContext datastoreContext, SchemaContext schemaContext) {
867             this.name = name;
868             this.peerAddresses = peerAddresses;
869             this.datastoreContext = datastoreContext;
870             this.schemaContext = schemaContext;
871         }
872
873         @Override
874         public Shard create() throws Exception {
875             return new Shard(name, peerAddresses, datastoreContext, schemaContext);
876         }
877     }
878
879     @VisibleForTesting
880     InMemoryDOMDataStore getDataStore() {
881         return store;
882     }
883
884     @VisibleForTesting
885     ShardStats getShardMBean() {
886         return shardMBean;
887     }
888
889     private static class DelayedListenerRegistration implements
890         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>> {
891
892         private volatile boolean closed;
893
894         private final RegisterChangeListener registerChangeListener;
895
896         private volatile ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
897                                                              NormalizedNode<?, ?>>> delegate;
898
899         DelayedListenerRegistration(RegisterChangeListener registerChangeListener) {
900             this.registerChangeListener = registerChangeListener;
901         }
902
903         void setDelegate( ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier,
904                                             NormalizedNode<?, ?>>> registration) {
905             this.delegate = registration;
906         }
907
908         boolean isClosed() {
909             return closed;
910         }
911
912         RegisterChangeListener getRegisterChangeListener() {
913             return registerChangeListener;
914         }
915
916         @Override
917         public AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>> getInstance() {
918             return delegate != null ? delegate.getInstance() : null;
919         }
920
921         @Override
922         public void close() {
923             closed = true;
924             if(delegate != null) {
925                 delegate.close();
926             }
927         }
928     }
929 }