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