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