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