Merge "BUG 2047 - HTTP GET - no returning error message"
[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.PoisonPill;
14 import akka.actor.Props;
15 import akka.event.Logging;
16 import akka.event.LoggingAdapter;
17 import akka.japi.Creator;
18 import akka.persistence.RecoveryFailure;
19 import akka.serialization.Serialization;
20 import com.google.common.annotations.VisibleForTesting;
21 import com.google.common.base.Optional;
22 import com.google.common.base.Preconditions;
23 import com.google.common.collect.Lists;
24 import com.google.common.util.concurrent.CheckedFuture;
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.common.actor.CommonConfig;
31 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
32 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
33 import org.opendaylight.controller.cluster.datastore.identifiers.ShardTransactionIdentifier;
34 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardMBeanFactory;
35 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shard.ShardStats;
36 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
37 import org.opendaylight.controller.cluster.datastore.messages.CloseTransactionChain;
38 import org.opendaylight.controller.cluster.datastore.messages.CommitTransactionReply;
39 import org.opendaylight.controller.cluster.datastore.messages.CreateTransaction;
40 import org.opendaylight.controller.cluster.datastore.messages.CreateTransactionReply;
41 import org.opendaylight.controller.cluster.datastore.messages.EnableNotification;
42 import org.opendaylight.controller.cluster.datastore.messages.ForwardedCommitTransaction;
43 import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
44 import org.opendaylight.controller.cluster.datastore.messages.ReadData;
45 import org.opendaylight.controller.cluster.datastore.messages.ReadDataReply;
46 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListener;
47 import org.opendaylight.controller.cluster.datastore.messages.RegisterChangeListenerReply;
48 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
49 import org.opendaylight.controller.cluster.datastore.modification.Modification;
50 import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification;
51 import org.opendaylight.controller.cluster.datastore.node.NormalizedNodeToNodeCodec;
52 import org.opendaylight.controller.cluster.raft.RaftActor;
53 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
54 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
55 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.CompositeModificationPayload;
56 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
57 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
58 import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
59 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStore;
60 import org.opendaylight.controller.md.sal.dom.store.impl.InMemoryDOMDataStoreFactory;
61 import org.opendaylight.controller.protobuff.messages.common.NormalizedNodeMessages;
62 import org.opendaylight.controller.sal.core.spi.data.DOMStoreReadTransaction;
63 import org.opendaylight.controller.sal.core.spi.data.DOMStoreThreePhaseCommitCohort;
64 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionChain;
65 import org.opendaylight.controller.sal.core.spi.data.DOMStoreTransactionFactory;
66 import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction;
67 import org.opendaylight.yangtools.concepts.ListenerRegistration;
68 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
69 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
70 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
71 import java.util.ArrayList;
72 import java.util.Collection;
73 import java.util.HashMap;
74 import java.util.List;
75 import java.util.Map;
76 import java.util.concurrent.ExecutionException;
77
78 /**
79  * A Shard represents a portion of the logical data tree <br/>
80  * <p>
81  * Our Shard uses InMemoryDataStore as it's internal representation and delegates all requests it
82  * </p>
83  */
84 public class Shard extends RaftActor {
85
86     public static final String DEFAULT_NAME = "default";
87
88     // The state of this Shard
89     private final InMemoryDOMDataStore store;
90
91     private final Map<Object, DOMStoreThreePhaseCommitCohort>
92         modificationToCohort = new HashMap<>();
93
94     private final LoggingAdapter LOG =
95         Logging.getLogger(getContext().system(), this);
96
97     // By default persistent will be true and can be turned off using the system
98     // property shard.persistent
99     private final boolean persistent;
100
101     /// The name of this shard
102     private final ShardIdentifier name;
103
104     private final ShardStats shardMBean;
105
106     private final List<ActorSelection> dataChangeListeners = new ArrayList<>();
107
108     private final DatastoreContext datastoreContext;
109
110     private SchemaContext schemaContext;
111
112     private ActorRef createSnapshotTransaction;
113
114     /**
115      * Coordinates persistence recovery on startup.
116      */
117     private ShardRecoveryCoordinator recoveryCoordinator;
118     private List<Object> currentLogRecoveryBatch;
119
120     private final Map<String, DOMStoreTransactionChain> transactionChains = new HashMap<>();
121
122     protected Shard(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
123             DatastoreContext datastoreContext, SchemaContext schemaContext) {
124         super(name.toString(), mapPeerAddresses(peerAddresses),
125                 Optional.of(datastoreContext.getShardRaftConfig()));
126
127         this.name = name;
128         this.datastoreContext = datastoreContext;
129         this.schemaContext = schemaContext;
130
131         String setting = System.getProperty("shard.persistent");
132
133         this.persistent = !"false".equals(setting);
134
135         LOG.info("Shard created : {} persistent : {}", name, persistent);
136
137         store = InMemoryDOMDataStoreFactory.create(name.toString(), null,
138                 datastoreContext.getDataStoreProperties());
139
140         if(schemaContext != null) {
141             store.onGlobalContextUpdated(schemaContext);
142         }
143
144         shardMBean = ShardMBeanFactory.getShardStatsMBean(name.toString(),
145                 datastoreContext.getDataStoreMXBeanType());
146         shardMBean.setDataStoreExecutor(store.getDomStoreExecutor());
147         shardMBean.setNotificationManager(store.getDataChangeListenerNotificationManager());
148
149         if (isMetricsCaptureEnabled()) {
150             getContext().become(new MeteringBehavior(this));
151         }
152     }
153
154     private static Map<String, String> mapPeerAddresses(
155         Map<ShardIdentifier, String> peerAddresses) {
156         Map<String, String> map = new HashMap<>();
157
158         for (Map.Entry<ShardIdentifier, String> entry : peerAddresses
159             .entrySet()) {
160             map.put(entry.getKey().toString(), entry.getValue());
161         }
162
163         return map;
164     }
165
166     public static Props props(final ShardIdentifier name,
167         final Map<ShardIdentifier, String> peerAddresses,
168         DatastoreContext datastoreContext, SchemaContext schemaContext) {
169         Preconditions.checkNotNull(name, "name should not be null");
170         Preconditions.checkNotNull(peerAddresses, "peerAddresses should not be null");
171         Preconditions.checkNotNull(datastoreContext, "dataStoreContext should not be null");
172         Preconditions.checkNotNull(schemaContext, "schemaContext should not be null");
173
174         return Props.create(new ShardCreator(name, peerAddresses, datastoreContext, schemaContext));
175     }
176
177     @Override public void onReceiveRecover(Object message) {
178         if(LOG.isDebugEnabled()) {
179             LOG.debug("onReceiveRecover: Received message {} from {}",
180                 message.getClass().toString(),
181                 getSender());
182         }
183
184         if (message instanceof RecoveryFailure){
185             LOG.error(((RecoveryFailure) message).cause(), "Recovery failed because of this cause");
186         } else {
187             super.onReceiveRecover(message);
188         }
189     }
190
191     @Override public void onReceiveCommand(Object message) {
192         if(LOG.isDebugEnabled()) {
193             LOG.debug("onReceiveCommand: Received message {} from {}",
194                 message.getClass().toString(),
195                 getSender());
196         }
197
198         if(message.getClass().equals(ReadDataReply.SERIALIZABLE_CLASS)) {
199             // This must be for install snapshot. Don't want to open this up and trigger
200             // deSerialization
201             self()
202                 .tell(new CaptureSnapshotReply(ReadDataReply.getNormalizedNodeByteString(message)),
203                     self());
204
205             createSnapshotTransaction = null;
206             // Send a PoisonPill instead of sending close transaction because we do not really need
207             // a response
208             getSender().tell(PoisonPill.getInstance(), self());
209
210         } else if (message.getClass().equals(CloseTransactionChain.SERIALIZABLE_CLASS)){
211             closeTransactionChain(CloseTransactionChain.fromSerializable(message));
212         } else if (message instanceof RegisterChangeListener) {
213             registerChangeListener((RegisterChangeListener) message);
214         } else if (message instanceof UpdateSchemaContext) {
215             updateSchemaContext((UpdateSchemaContext) message);
216         } else if (message instanceof ForwardedCommitTransaction) {
217             handleForwardedCommit((ForwardedCommitTransaction) message);
218         } else if (message.getClass()
219             .equals(CreateTransaction.SERIALIZABLE_CLASS)) {
220             if (isLeader()) {
221                 createTransaction(CreateTransaction.fromSerializable(message));
222             } else if (getLeader() != null) {
223                 getLeader().forward(message, getContext());
224             } else {
225                 getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(
226                     "Could not find shard leader so transaction cannot be created. This typically happens" +
227                             " when system is coming up or recovering and a leader is being elected. Try again" +
228                             " later.")), getSelf());
229             }
230         } else if (message instanceof PeerAddressResolved) {
231             PeerAddressResolved resolved = (PeerAddressResolved) message;
232             setPeerAddress(resolved.getPeerId().toString(),
233                 resolved.getPeerAddress());
234         } else {
235             super.onReceiveCommand(message);
236         }
237     }
238
239     private void closeTransactionChain(CloseTransactionChain closeTransactionChain) {
240         DOMStoreTransactionChain chain =
241             transactionChains.remove(closeTransactionChain.getTransactionChainId());
242
243         if(chain != null) {
244             chain.close();
245         }
246     }
247
248     private ActorRef createTypedTransactionActor(
249         int transactionType,
250         ShardTransactionIdentifier transactionId,
251         String transactionChainId ) {
252
253         DOMStoreTransactionFactory factory = store;
254
255         if(!transactionChainId.isEmpty()) {
256             factory = transactionChains.get(transactionChainId);
257             if(factory == null){
258                 DOMStoreTransactionChain transactionChain = store.createTransactionChain();
259                 transactionChains.put(transactionChainId, transactionChain);
260                 factory = transactionChain;
261             }
262         }
263
264         if(this.schemaContext == null){
265             throw new NullPointerException("schemaContext should not be null");
266         }
267
268         if (transactionType
269             == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
270
271             shardMBean.incrementReadOnlyTransactionCount();
272
273             return getContext().actorOf(
274                 ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(),
275                         schemaContext,datastoreContext, shardMBean), transactionId.toString());
276
277         } else if (transactionType
278             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
279
280             shardMBean.incrementReadWriteTransactionCount();
281
282             return getContext().actorOf(
283                 ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(),
284                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
285
286
287         } else if (transactionType
288             == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
289
290             shardMBean.incrementWriteOnlyTransactionCount();
291
292             return getContext().actorOf(
293                 ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(),
294                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
295         } else {
296             throw new IllegalArgumentException(
297                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
298                     + transactionType);
299         }
300     }
301
302     private void createTransaction(CreateTransaction createTransaction) {
303         createTransaction(createTransaction.getTransactionType(),
304             createTransaction.getTransactionId(), createTransaction.getTransactionChainId());
305     }
306
307     private ActorRef createTransaction(int transactionType, String remoteTransactionId, String transactionChainId) {
308
309         ShardTransactionIdentifier transactionId =
310             ShardTransactionIdentifier.builder()
311                 .remoteTransactionId(remoteTransactionId)
312                 .build();
313         if(LOG.isDebugEnabled()) {
314             LOG.debug("Creating transaction : {} ", transactionId);
315         }
316         ActorRef transactionActor =
317             createTypedTransactionActor(transactionType, transactionId, transactionChainId);
318
319         getSender()
320             .tell(new CreateTransactionReply(
321                     Serialization.serializedActorPath(transactionActor),
322                     remoteTransactionId).toSerializable(),
323                 getSelf());
324
325         return transactionActor;
326     }
327
328     private void syncCommitTransaction(DOMStoreWriteTransaction transaction)
329         throws ExecutionException, InterruptedException {
330         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
331         commitCohort.preCommit().get();
332         commitCohort.commit().get();
333     }
334
335
336     private void commit(final ActorRef sender, Object serialized) {
337         Modification modification = MutableCompositeModification
338             .fromSerializable(serialized, schemaContext);
339         DOMStoreThreePhaseCommitCohort cohort =
340             modificationToCohort.remove(serialized);
341         if (cohort == null) {
342             // If there's no cached cohort then we must be applying replicated state.
343             commitWithNewTransaction(serialized);
344             return;
345         }
346
347         if(sender == null) {
348             LOG.error("Commit failed. Sender cannot be null");
349             return;
350         }
351
352         ListenableFuture<Void> future = cohort.commit();
353
354         Futures.addCallback(future, new FutureCallback<Void>() {
355             @Override
356             public void onSuccess(Void v) {
357                 sender.tell(new CommitTransactionReply().toSerializable(), getSelf());
358                 shardMBean.incrementCommittedTransactionCount();
359                 shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
360             }
361
362             @Override
363             public void onFailure(Throwable t) {
364                 LOG.error(t, "An exception happened during commit");
365                 shardMBean.incrementFailedTransactionsCount();
366                 sender.tell(new akka.actor.Status.Failure(t), getSelf());
367             }
368         });
369
370     }
371
372     private void commitWithNewTransaction(Object modification) {
373         DOMStoreWriteTransaction tx = store.newWriteOnlyTransaction();
374         MutableCompositeModification.fromSerializable(modification, schemaContext).apply(tx);
375         try {
376             syncCommitTransaction(tx);
377             shardMBean.incrementCommittedTransactionCount();
378         } catch (InterruptedException | ExecutionException e) {
379             shardMBean.incrementFailedTransactionsCount();
380             LOG.error(e, "Failed to commit");
381         }
382     }
383
384     private void handleForwardedCommit(ForwardedCommitTransaction message) {
385         Object serializedModification =
386             message.getModification().toSerializable();
387
388         modificationToCohort
389             .put(serializedModification, message.getCohort());
390
391         if (persistent) {
392             this.persistData(getSender(), "identifier",
393                 new CompositeModificationPayload(serializedModification));
394         } else {
395             this.commit(getSender(), serializedModification);
396         }
397     }
398
399     private void updateSchemaContext(UpdateSchemaContext message) {
400         this.schemaContext = message.getSchemaContext();
401         updateSchemaContext(message.getSchemaContext());
402         store.onGlobalContextUpdated(message.getSchemaContext());
403     }
404
405     @VisibleForTesting void updateSchemaContext(SchemaContext schemaContext) {
406         store.onGlobalContextUpdated(schemaContext);
407     }
408
409     private void registerChangeListener(
410         RegisterChangeListener registerChangeListener) {
411
412         if(LOG.isDebugEnabled()) {
413             LOG.debug("registerDataChangeListener for {}", registerChangeListener
414                 .getPath());
415         }
416
417
418         ActorSelection dataChangeListenerPath = getContext()
419             .system().actorSelection(
420                 registerChangeListener.getDataChangeListenerPath());
421
422
423         // Notify the listener if notifications should be enabled or not
424         // If this shard is the leader then it will enable notifications else
425         // it will not
426         dataChangeListenerPath
427             .tell(new EnableNotification(isLeader()), getSelf());
428
429         // Now store a reference to the data change listener so it can be notified
430         // at a later point if notifications should be enabled or disabled
431         dataChangeListeners.add(dataChangeListenerPath);
432
433         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
434             listener = new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
435
436         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
437             registration = store.registerChangeListener(registerChangeListener.getPath(),
438                 listener, registerChangeListener.getScope());
439         ActorRef listenerRegistration =
440             getContext().actorOf(
441                 DataChangeListenerRegistration.props(registration));
442
443         if(LOG.isDebugEnabled()) {
444             LOG.debug(
445                 "registerDataChangeListener sending reply, listenerRegistrationPath = {} "
446                 , listenerRegistration.path().toString());
447         }
448
449         getSender()
450             .tell(new RegisterChangeListenerReply(listenerRegistration.path()),
451                 getSelf());
452     }
453
454     private boolean isMetricsCaptureEnabled(){
455         CommonConfig config = new CommonConfig(getContext().system().settings().config());
456         return config.isMetricCaptureEnabled();
457     }
458
459     @Override
460     protected
461     void startLogRecoveryBatch(int maxBatchSize) {
462         currentLogRecoveryBatch = Lists.newArrayListWithCapacity(maxBatchSize);
463
464         if(LOG.isDebugEnabled()) {
465             LOG.debug("{} : starting log recovery batch with max size {}", persistenceId(), maxBatchSize);
466         }
467     }
468
469     @Override
470     protected void appendRecoveredLogEntry(Payload data) {
471         if (data instanceof CompositeModificationPayload) {
472             currentLogRecoveryBatch.add(((CompositeModificationPayload) data).getModification());
473         } else {
474             LOG.error("Unknown state received {} during recovery", data);
475         }
476     }
477
478     @Override
479     protected void applyRecoverySnapshot(ByteString snapshot) {
480         if(recoveryCoordinator == null) {
481             recoveryCoordinator = new ShardRecoveryCoordinator(persistenceId(), schemaContext);
482         }
483
484         recoveryCoordinator.submit(snapshot, store.newWriteOnlyTransaction());
485
486         if(LOG.isDebugEnabled()) {
487             LOG.debug("{} : submitted recovery sbapshot", persistenceId());
488         }
489     }
490
491     @Override
492     protected void applyCurrentLogRecoveryBatch() {
493         if(recoveryCoordinator == null) {
494             recoveryCoordinator = new ShardRecoveryCoordinator(persistenceId(), schemaContext);
495         }
496
497         recoveryCoordinator.submit(currentLogRecoveryBatch, store.newWriteOnlyTransaction());
498
499         if(LOG.isDebugEnabled()) {
500             LOG.debug("{} : submitted log recovery batch with size {}", persistenceId(),
501                     currentLogRecoveryBatch.size());
502         }
503     }
504
505     @Override
506     protected void onRecoveryComplete() {
507         if(recoveryCoordinator != null) {
508             Collection<DOMStoreWriteTransaction> txList = recoveryCoordinator.getTransactions();
509
510             if(LOG.isDebugEnabled()) {
511                 LOG.debug("{} : recovery complete - committing {} Tx's", persistenceId(), txList.size());
512             }
513
514             for(DOMStoreWriteTransaction tx: txList) {
515                 try {
516                     syncCommitTransaction(tx);
517                     shardMBean.incrementCommittedTransactionCount();
518                 } catch (InterruptedException | ExecutionException e) {
519                     shardMBean.incrementFailedTransactionsCount();
520                     LOG.error(e, "Failed to commit");
521                 }
522             }
523         }
524
525         recoveryCoordinator = null;
526         currentLogRecoveryBatch = null;
527         updateJournalStats();
528
529         //notify shard manager
530         getContext().parent().tell(new ActorInitialized(), getSelf());
531     }
532
533     @Override
534     protected void applyState(ActorRef clientActor, String identifier, Object data) {
535
536         if (data instanceof CompositeModificationPayload) {
537             Object modification = ((CompositeModificationPayload) data).getModification();
538
539             if (modification != null) {
540                 commit(clientActor, modification);
541             } else {
542                 LOG.error(
543                     "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
544                     identifier, clientActor != null ? clientActor.path().toString() : null);
545             }
546
547         } else {
548             LOG.error("Unknown state received {} Class loader = {} CompositeNodeMod.ClassLoader = {}",
549                     data, data.getClass().getClassLoader(),
550                     CompositeModificationPayload.class.getClassLoader());
551         }
552
553         updateJournalStats();
554
555     }
556
557     private void updateJournalStats() {
558         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
559
560         if (lastLogEntry != null) {
561             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
562             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
563         }
564
565         shardMBean.setCommitIndex(getCommitIndex());
566         shardMBean.setLastApplied(getLastApplied());
567     }
568
569     @Override
570     protected void createSnapshot() {
571         if (createSnapshotTransaction == null) {
572
573             // Create a transaction. We are really going to treat the transaction as a worker
574             // so that this actor does not get block building the snapshot
575             createSnapshotTransaction = createTransaction(
576                 TransactionProxy.TransactionType.READ_ONLY.ordinal(),
577                 "createSnapshot", "");
578
579             createSnapshotTransaction.tell(
580                 new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
581
582         }
583     }
584
585     @VisibleForTesting
586     @Override
587     protected void applySnapshot(ByteString snapshot) {
588         // Since this will be done only on Recovery or when this actor is a Follower
589         // we can safely commit everything in here. We not need to worry about event notifications
590         // as they would have already been disabled on the follower
591
592         LOG.info("Applying snapshot");
593         try {
594             DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
595             NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
596             NormalizedNode<?, ?> node = new NormalizedNodeToNodeCodec(schemaContext)
597                 .decode(serializedNode);
598
599             // delete everything first
600             transaction.delete(YangInstanceIdentifier.builder().build());
601
602             // Add everything from the remote node back
603             transaction.write(YangInstanceIdentifier.builder().build(), node);
604             syncCommitTransaction(transaction);
605         } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
606             LOG.error(e, "An exception occurred when applying snapshot");
607         } finally {
608             LOG.info("Done applying snapshot");
609         }
610     }
611
612     @Override protected void onStateChanged() {
613         for (ActorSelection dataChangeListener : dataChangeListeners) {
614             dataChangeListener
615                 .tell(new EnableNotification(isLeader()), getSelf());
616         }
617
618         shardMBean.setRaftState(getRaftState().name());
619         shardMBean.setCurrentTerm(getCurrentTerm());
620
621         // If this actor is no longer the leader close all the transaction chains
622         if(!isLeader()){
623             for(Map.Entry<String, DOMStoreTransactionChain> entry : transactionChains.entrySet()){
624                 if(LOG.isDebugEnabled()) {
625                     LOG.debug(
626                         "onStateChanged: Closing transaction chain {} because shard {} is no longer the leader",
627                         entry.getKey(), getId());
628                 }
629                 entry.getValue().close();
630             }
631
632             transactionChains.clear();
633         }
634     }
635
636     @Override protected void onLeaderChanged(String oldLeader, String newLeader) {
637         shardMBean.setLeader(newLeader);
638     }
639
640     @Override public String persistenceId() {
641         return this.name.toString();
642     }
643
644     private static class ShardCreator implements Creator<Shard> {
645
646         private static final long serialVersionUID = 1L;
647
648         final ShardIdentifier name;
649         final Map<ShardIdentifier, String> peerAddresses;
650         final DatastoreContext datastoreContext;
651         final SchemaContext schemaContext;
652
653         ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
654                 DatastoreContext datastoreContext, SchemaContext schemaContext) {
655             this.name = name;
656             this.peerAddresses = peerAddresses;
657             this.datastoreContext = datastoreContext;
658             this.schemaContext = schemaContext;
659         }
660
661         @Override
662         public Shard create() throws Exception {
663             return new Shard(name, peerAddresses, datastoreContext, schemaContext);
664         }
665     }
666
667     @VisibleForTesting
668     NormalizedNode<?,?> readStore(YangInstanceIdentifier id)
669             throws ExecutionException, InterruptedException {
670         DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
671
672         CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future =
673             transaction.read(id);
674
675         Optional<NormalizedNode<?, ?>> optional = future.get();
676         NormalizedNode<?, ?> node = optional.isPresent()? optional.get() : null;
677
678         transaction.close();
679
680         return node;
681     }
682
683     @VisibleForTesting
684     void writeToStore(YangInstanceIdentifier id, NormalizedNode<?,?> node)
685         throws ExecutionException, InterruptedException {
686         DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
687
688         transaction.write(id, node);
689
690         syncCommitTransaction(transaction);
691     }
692
693     @VisibleForTesting
694     ShardStats getShardMBean() {
695         return shardMBean;
696     }
697 }