Merge "Bug 1804: the condition was not working correctly to report leader"
[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             createSnapshotTransaction = null;
197             // Send a PoisonPill instead of sending close transaction because we do not really need
198             // a response
199             getSender().tell(PoisonPill.getInstance(), self());
200
201         } else if (message.getClass().equals(CloseTransactionChain.SERIALIZABLE_CLASS)){
202             closeTransactionChain(CloseTransactionChain.fromSerializable(message));
203         } else if (message instanceof RegisterChangeListener) {
204             registerChangeListener((RegisterChangeListener) message);
205         } else if (message instanceof UpdateSchemaContext) {
206             updateSchemaContext((UpdateSchemaContext) message);
207         } else if (message instanceof ForwardedCommitTransaction) {
208             handleForwardedCommit((ForwardedCommitTransaction) message);
209         } else if (message.getClass()
210             .equals(CreateTransaction.SERIALIZABLE_CLASS)) {
211             if (isLeader()) {
212                 createTransaction(CreateTransaction.fromSerializable(message));
213             } else if (getLeader() != null) {
214                 getLeader().forward(message, getContext());
215             } else {
216                 getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(
217                     "Could not find leader so transaction cannot be created")), getSelf());
218             }
219         } else if (message instanceof PeerAddressResolved) {
220             PeerAddressResolved resolved = (PeerAddressResolved) message;
221             setPeerAddress(resolved.getPeerId().toString(),
222                 resolved.getPeerAddress());
223         } else {
224             super.onReceiveCommand(message);
225         }
226     }
227
228     private void closeTransactionChain(CloseTransactionChain closeTransactionChain) {
229         DOMStoreTransactionChain chain =
230             transactionChains.remove(closeTransactionChain.getTransactionChainId());
231
232         if(chain != null) {
233             chain.close();
234         }
235     }
236
237     private ActorRef createTypedTransactionActor(
238         int transactionType,
239         ShardTransactionIdentifier transactionId,
240         String transactionChainId ) {
241
242         DOMStoreTransactionFactory factory = store;
243
244         if(!transactionChainId.isEmpty()) {
245             factory = transactionChains.get(transactionChainId);
246             if(factory == null){
247                 DOMStoreTransactionChain transactionChain = store.createTransactionChain();
248                 transactionChains.put(transactionChainId, transactionChain);
249                 factory = transactionChain;
250             }
251         }
252
253         if(this.schemaContext == null){
254             throw new NullPointerException("schemaContext should not be null");
255         }
256
257         if (transactionType
258             == TransactionProxy.TransactionType.READ_ONLY.ordinal()) {
259
260             shardMBean.incrementReadOnlyTransactionCount();
261
262             return getContext().actorOf(
263                 ShardTransaction.props(factory.newReadOnlyTransaction(), getSelf(),
264                         schemaContext,datastoreContext, shardMBean), transactionId.toString());
265
266         } else if (transactionType
267             == TransactionProxy.TransactionType.READ_WRITE.ordinal()) {
268
269             shardMBean.incrementReadWriteTransactionCount();
270
271             return getContext().actorOf(
272                 ShardTransaction.props(factory.newReadWriteTransaction(), getSelf(),
273                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
274
275
276         } else if (transactionType
277             == TransactionProxy.TransactionType.WRITE_ONLY.ordinal()) {
278
279             shardMBean.incrementWriteOnlyTransactionCount();
280
281             return getContext().actorOf(
282                 ShardTransaction.props(factory.newWriteOnlyTransaction(), getSelf(),
283                         schemaContext, datastoreContext, shardMBean), transactionId.toString());
284         } else {
285             throw new IllegalArgumentException(
286                 "Shard="+name + ":CreateTransaction message has unidentified transaction type="
287                     + transactionType);
288         }
289     }
290
291     private void createTransaction(CreateTransaction createTransaction) {
292         createTransaction(createTransaction.getTransactionType(),
293             createTransaction.getTransactionId(), createTransaction.getTransactionChainId());
294     }
295
296     private ActorRef createTransaction(int transactionType, String remoteTransactionId, String transactionChainId) {
297
298         ShardTransactionIdentifier transactionId =
299             ShardTransactionIdentifier.builder()
300                 .remoteTransactionId(remoteTransactionId)
301                 .build();
302         LOG.debug("Creating transaction : {} ", transactionId);
303         ActorRef transactionActor =
304             createTypedTransactionActor(transactionType, transactionId, transactionChainId);
305
306         getSender()
307             .tell(new CreateTransactionReply(
308                     Serialization.serializedActorPath(transactionActor),
309                     remoteTransactionId).toSerializable(),
310                 getSelf());
311
312         return transactionActor;
313     }
314
315     private void syncCommitTransaction(DOMStoreWriteTransaction transaction)
316         throws ExecutionException, InterruptedException {
317         DOMStoreThreePhaseCommitCohort commitCohort = transaction.ready();
318         commitCohort.preCommit().get();
319         commitCohort.commit().get();
320     }
321
322
323     private void commit(final ActorRef sender, Object serialized) {
324         Modification modification = MutableCompositeModification
325             .fromSerializable(serialized, schemaContext);
326         DOMStoreThreePhaseCommitCohort cohort =
327             modificationToCohort.remove(serialized);
328         if (cohort == null) {
329             LOG.debug(
330                 "Could not find cohort for modification : {}. Writing modification using a new transaction",
331                 modification);
332             DOMStoreWriteTransaction transaction =
333                 store.newWriteOnlyTransaction();
334
335             LOG.debug("Created new transaction {}", transaction.getIdentifier().toString());
336
337             modification.apply(transaction);
338             try {
339                 syncCommitTransaction(transaction);
340             } catch (InterruptedException | ExecutionException e) {
341                 shardMBean.incrementFailedTransactionsCount();
342                 LOG.error("Failed to commit", e);
343                 return;
344             }
345             //we want to just apply the recovery commit and return
346             shardMBean.incrementCommittedTransactionCount();
347             return;
348         }
349
350
351         if(sender == null){
352             LOG.error("Commit failed. Sender cannot be null");
353             return;
354         }
355
356         final ListenableFuture<Void> future = cohort.commit();
357         final ActorRef self = getSelf();
358
359         Futures.addCallback(future, new FutureCallback<Void>() {
360             @Override
361             public void onSuccess(Void v) {
362                 sender.tell(new CommitTransactionReply().toSerializable(), self);
363                 shardMBean.incrementCommittedTransactionCount();
364                 shardMBean.setLastCommittedTransactionTime(System.currentTimeMillis());
365             }
366
367             @Override
368             public void onFailure(Throwable t) {
369                 LOG.error(t, "An exception happened during commit");
370                 shardMBean.incrementFailedTransactionsCount();
371                 sender.tell(new akka.actor.Status.Failure(t), self);
372             }
373         });
374
375     }
376
377     private void handleForwardedCommit(ForwardedCommitTransaction message) {
378         Object serializedModification =
379             message.getModification().toSerializable();
380
381         modificationToCohort
382             .put(serializedModification, message.getCohort());
383
384         if (persistent) {
385             this.persistData(getSender(), "identifier",
386                 new CompositeModificationPayload(serializedModification));
387         } else {
388             this.commit(getSender(), serializedModification);
389         }
390     }
391
392     private void updateSchemaContext(UpdateSchemaContext message) {
393         this.schemaContext = message.getSchemaContext();
394         updateSchemaContext(message.getSchemaContext());
395         store.onGlobalContextUpdated(message.getSchemaContext());
396     }
397
398     @VisibleForTesting void updateSchemaContext(SchemaContext schemaContext) {
399         store.onGlobalContextUpdated(schemaContext);
400     }
401
402     private void registerChangeListener(
403         RegisterChangeListener registerChangeListener) {
404
405         LOG.debug("registerDataChangeListener for {}", registerChangeListener
406             .getPath());
407
408
409         ActorSelection dataChangeListenerPath = getContext()
410             .system().actorSelection(
411                 registerChangeListener.getDataChangeListenerPath());
412
413
414         // Notify the listener if notifications should be enabled or not
415         // If this shard is the leader then it will enable notifications else
416         // it will not
417         dataChangeListenerPath
418             .tell(new EnableNotification(isLeader()), getSelf());
419
420         // Now store a reference to the data change listener so it can be notified
421         // at a later point if notifications should be enabled or disabled
422         dataChangeListeners.add(dataChangeListenerPath);
423
424         AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>
425             listener = new DataChangeListenerProxy(schemaContext, dataChangeListenerPath);
426
427         ListenerRegistration<AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
428             registration = store.registerChangeListener(registerChangeListener.getPath(),
429                 listener, registerChangeListener.getScope());
430         ActorRef listenerRegistration =
431             getContext().actorOf(
432                 DataChangeListenerRegistration.props(registration));
433
434         LOG.debug(
435             "registerDataChangeListener sending reply, listenerRegistrationPath = {} "
436             , listenerRegistration.path().toString());
437
438         getSender()
439             .tell(new RegisterChangeListenerReply(listenerRegistration.path()),
440                 getSelf());
441     }
442
443     private void createTransactionChain() {
444         DOMStoreTransactionChain chain = store.createTransactionChain();
445         ActorRef transactionChain = getContext().actorOf(
446                 ShardTransactionChain.props(chain, schemaContext, datastoreContext, shardMBean));
447         getSender().tell(new CreateTransactionChainReply(transactionChain.path()).toSerializable(),
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 protected void applyState(ActorRef clientActor, String identifier,
457         Object data) {
458
459         if (data instanceof CompositeModificationPayload) {
460             Object modification =
461                 ((CompositeModificationPayload) data).getModification();
462
463             if (modification != null) {
464                 commit(clientActor, modification);
465             } else {
466                 LOG.error(
467                     "modification is null - this is very unexpected, clientActor = {}, identifier = {}",
468                     identifier, clientActor.path().toString());
469             }
470
471         } else {
472             LOG.error("Unknown state received {} Class loader = {} CompositeNodeMod.ClassLoader = {}", data, data.getClass().getClassLoader(), CompositeModificationPayload.class.getClassLoader());
473         }
474
475         // Update stats
476         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
477
478         if (lastLogEntry != null) {
479             shardMBean.setLastLogIndex(lastLogEntry.getIndex());
480             shardMBean.setLastLogTerm(lastLogEntry.getTerm());
481         }
482
483         shardMBean.setCommitIndex(getCommitIndex());
484         shardMBean.setLastApplied(getLastApplied());
485
486     }
487
488     @Override protected void createSnapshot() {
489         if (createSnapshotTransaction == null) {
490
491             // Create a transaction. We are really going to treat the transaction as a worker
492             // so that this actor does not get block building the snapshot
493             createSnapshotTransaction = createTransaction(
494                 TransactionProxy.TransactionType.READ_ONLY.ordinal(),
495                 "createSnapshot", "");
496
497             createSnapshotTransaction.tell(
498                 new ReadData(YangInstanceIdentifier.builder().build()).toSerializable(), self());
499
500         }
501     }
502
503     @VisibleForTesting @Override protected void applySnapshot(ByteString snapshot) {
504         // Since this will be done only on Recovery or when this actor is a Follower
505         // we can safely commit everything in here. We not need to worry about event notifications
506         // as they would have already been disabled on the follower
507
508         LOG.info("Applying snapshot");
509         try {
510             DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
511             NormalizedNodeMessages.Node serializedNode = NormalizedNodeMessages.Node.parseFrom(snapshot);
512             NormalizedNode<?, ?> node = new NormalizedNodeToNodeCodec(schemaContext)
513                 .decode(YangInstanceIdentifier.builder().build(), serializedNode);
514
515             // delete everything first
516             transaction.delete(YangInstanceIdentifier.builder().build());
517
518             // Add everything from the remote node back
519             transaction.write(YangInstanceIdentifier.builder().build(), node);
520             syncCommitTransaction(transaction);
521         } catch (InvalidProtocolBufferException | InterruptedException | ExecutionException e) {
522             LOG.error(e, "An exception occurred when applying snapshot");
523         } finally {
524             LOG.info("Done applying snapshot");
525         }
526     }
527
528     @Override protected void onStateChanged() {
529         for (ActorSelection dataChangeListener : dataChangeListeners) {
530             dataChangeListener
531                 .tell(new EnableNotification(isLeader()), getSelf());
532         }
533
534
535         shardMBean.setRaftState(getRaftState().name());
536         shardMBean.setCurrentTerm(getCurrentTerm());
537
538         // If this actor is no longer the leader close all the transaction chains
539         if(!isLeader()){
540             for(Map.Entry<String, DOMStoreTransactionChain> entry : transactionChains.entrySet()){
541                 LOG.debug("onStateChanged: Closing transaction chain {} because shard {} is no longer the leader", entry.getKey(), getId());
542                 entry.getValue().close();
543             }
544
545             transactionChains.clear();
546         }
547     }
548
549     @Override protected void onLeaderChanged(String oldLeader, String newLeader) {
550         shardMBean.setLeader(newLeader);
551     }
552
553     @Override public String persistenceId() {
554         return this.name.toString();
555     }
556
557
558     private static class ShardConfigParams extends DefaultConfigParamsImpl {
559         public static final FiniteDuration HEART_BEAT_INTERVAL =
560             new FiniteDuration(500, TimeUnit.MILLISECONDS);
561
562         @Override public FiniteDuration getHeartBeatInterval() {
563             return HEART_BEAT_INTERVAL;
564         }
565     }
566
567     private static class ShardCreator implements Creator<Shard> {
568
569         private static final long serialVersionUID = 1L;
570
571         final ShardIdentifier name;
572         final Map<ShardIdentifier, String> peerAddresses;
573         final DatastoreContext datastoreContext;
574         final SchemaContext schemaContext;
575
576         ShardCreator(ShardIdentifier name, Map<ShardIdentifier, String> peerAddresses,
577                 DatastoreContext datastoreContext, SchemaContext schemaContext) {
578             this.name = name;
579             this.peerAddresses = peerAddresses;
580             this.datastoreContext = datastoreContext;
581             this.schemaContext = schemaContext;
582         }
583
584         @Override
585         public Shard create() throws Exception {
586             return new Shard(name, peerAddresses, datastoreContext, schemaContext);
587         }
588     }
589
590     @VisibleForTesting NormalizedNode readStore() throws ExecutionException, InterruptedException {
591         DOMStoreReadTransaction transaction = store.newReadOnlyTransaction();
592
593         CheckedFuture<Optional<NormalizedNode<?, ?>>, ReadFailedException> future =
594             transaction.read(YangInstanceIdentifier.builder().build());
595
596         NormalizedNode<?, ?> node = future.get().get();
597
598         transaction.close();
599
600         return node;
601     }
602
603     @VisibleForTesting void writeToStore(YangInstanceIdentifier id, NormalizedNode node)
604         throws ExecutionException, InterruptedException {
605         DOMStoreWriteTransaction transaction = store.newWriteOnlyTransaction();
606
607         transaction.write(id, node);
608
609         syncCommitTransaction(transaction);
610     }
611
612 }