Eliminate LockChangeListener 44/103844/2
authorRobert Varga <robert.varga@pantheon.tech>
Sat, 31 Dec 2022 17:45:16 +0000 (18:45 +0100)
committerRobert Varga <robert.varga@pantheon.tech>
Sun, 1 Jan 2023 13:40:07 +0000 (14:40 +0100)
Configuration of netconf-sal-provider should not directly depend on the
datastore. Move the single leaf from netconf-node-optional into
odl-netconf-device:datastore-access grouping and let users propagate it
via normal configuration updates.

This removes a race as well as explicitly deals with lifecycle of the
device when the configuration is changed.

JIRA: NETCONF-921
Change-Id: I07ba946bdef56dfc9ba958384188d93a32e2d974
Signed-off-by: Robert Varga <robert.varga@pantheon.tech>
13 files changed:
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyContext.java
netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/spi/AbstractNetconfTopology.java
netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/AbstractNetconfDataTreeService.java
netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/LockChangeListener.java [deleted file]
netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBroker.java
netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacade.java
netconf/sal-netconf-connector/src/main/yang/netconf-node-optional.yang
netconf/sal-netconf-connector/src/main/yang/netconf-node-topology.yang
netconf/sal-netconf-connector/src/main/yang/odl-netconf-device.yang
netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDataTreeServiceImplTest.java
netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBrokerTest.java
netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacadeTest.java

index f33a2ba4a0a2d0e7a955741cb1d6e88092404789..e747f2a8f17e13ebb91afc8f66e80dd8672a5390 100644 (file)
@@ -46,6 +46,7 @@ class MasterSalFacade implements RemoteDeviceHandler, AutoCloseable {
     private final NetconfDeviceSalProvider salProvider;
     private final ActorRef masterActorRef;
     private final ActorSystem actorSystem;
+    private final boolean lockDatastore;
 
     private NetconfDeviceSchema currentSchema = null;
     private NetconfSessionPreferences netconfSessionPreferences = null;
@@ -58,12 +59,14 @@ class MasterSalFacade implements RemoteDeviceHandler, AutoCloseable {
                     final ActorRef masterActorRef,
                     final Timeout actorResponseWaitTime,
                     final DOMMountPointService mountService,
-                    final DataBroker dataBroker) {
+                    final DataBroker dataBroker,
+                    final boolean lockDatastore) {
         this.id = id;
         salProvider = new NetconfDeviceSalProvider(id, mountService, dataBroker);
         this.actorSystem = actorSystem;
         this.masterActorRef = masterActorRef;
         this.actorResponseWaitTime = actorResponseWaitTime;
+        this.lockDatastore = lockDatastore;
     }
 
     @Override
@@ -142,12 +145,12 @@ class MasterSalFacade implements RemoteDeviceHandler, AutoCloseable {
 
     protected DOMDataBroker newDeviceDataBroker(final MountPointContext mountContext,
             final NetconfSessionPreferences preferences) {
-        return new NetconfDeviceDataBroker(id, mountContext, deviceServices.rpcs(), preferences);
+        return new NetconfDeviceDataBroker(id, mountContext, deviceServices.rpcs(), preferences, lockDatastore);
     }
 
     protected NetconfDataTreeService newNetconfDataTreeService(final MountPointContext mountContext,
             final NetconfSessionPreferences preferences) {
-        return AbstractNetconfDataTreeService.of(id, mountContext, deviceServices.rpcs(), preferences);
+        return AbstractNetconfDataTreeService.of(id, mountContext, deviceServices.rpcs(), preferences, lockDatastore);
     }
 
     private Future<Object> sendInitialDataToActor() {
index 4c92f3c0e3655633f2f502244b53c202e5bb0655..6b070d08a7e0ea4d2eb6b39ec9cf619bfc509c10 100644 (file)
@@ -7,6 +7,7 @@
  */
 package org.opendaylight.netconf.topology.singleton.impl;
 
+import static com.google.common.base.Verify.verifyNotNull;
 import static java.util.Objects.requireNonNull;
 
 import akka.actor.ActorRef;
@@ -34,19 +35,20 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
-
     private static final Logger LOG = LoggerFactory.getLogger(NetconfTopologyContext.class);
 
     private final @NonNull ServiceGroupIdentifier serviceGroupIdent;
     private final Timeout actorResponseWaitTime;
     private final DOMMountPointService mountService;
     private final DeviceActionFactory deviceActionFactory;
+    private final boolean lockDatastore;
 
     private NetconfTopologySetup netconfTopologyDeviceSetup;
     private RemoteDeviceId remoteDeviceId;
     private RemoteDeviceConnector remoteDeviceConnector;
     private NetconfNodeManager netconfNodeManager;
     private ActorRef masterActorRef;
+
     private final AtomicBoolean closed = new AtomicBoolean(false);
     private final AtomicBoolean stopped = new AtomicBoolean(false);
     private volatile boolean isMaster;
@@ -61,10 +63,12 @@ class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
         this.deviceActionFactory = deviceActionFactory;
 
         final var node = netconfTopologyDeviceSetup.getNode();
-        remoteDeviceId = NetconfNodeUtils.toRemoteDeviceId(node.getNodeId(), node.augmentation(NetconfNode.class));
+        final var netconfNode = verifyNotNull(node.augmentation(NetconfNode.class));
+        remoteDeviceId = NetconfNodeUtils.toRemoteDeviceId(node.getNodeId(), netconfNode);
         remoteDeviceConnector = new RemoteDeviceConnectorImpl(netconfTopologyDeviceSetup, remoteDeviceId,
             deviceActionFactory);
         netconfNodeManager = createNodeDeviceManager();
+        lockDatastore = netconfNode.requireLockDatastore();
     }
 
     @Override
@@ -179,6 +183,6 @@ class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
 
     protected MasterSalFacade newMasterSalFacade() {
         return new MasterSalFacade(remoteDeviceId, netconfTopologyDeviceSetup.getActorSystem(), masterActorRef,
-                actorResponseWaitTime, mountService, netconfTopologyDeviceSetup.getDataBroker());
+                actorResponseWaitTime, mountService, netconfTopologyDeviceSetup.getDataBroker(), lockDatastore);
     }
 }
\ No newline at end of file
index 1b881c409e7af4f16a9ebf4310cb9ac38ad71731..039f222693dd3fddc7f2a8ba38fb4339989439ea 100644 (file)
@@ -62,7 +62,7 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.cr
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.credentials.KeyAuth;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.credentials.LoginPw;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.credentials.LoginPwUnencrypted;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.NetconfNodeAugmentedOptional;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev221225.NetconfNodeAugmentedOptional;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev221225.NetconfNode;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NetworkTopology;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
@@ -216,7 +216,8 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
         final var deviceId = NetconfNodeUtils.toRemoteDeviceId(nodeId, node);
         final long keepaliveDelay = node.requireKeepaliveDelay().toJava();
 
-        final var deviceSalFacade = new NetconfDeviceSalFacade(deviceId, mountPointService, dataBroker, topologyId);
+        final var deviceSalFacade = new NetconfDeviceSalFacade(deviceId, mountPointService, dataBroker,
+            node.requireLockDatastore());
         // The facade we are going it present to NetconfDevice
         RemoteDeviceHandler salFacade;
         final KeepaliveSalFacade keepAliveFacade;
index 90d95d5e274135451eefcd5403117cbee745d712..e7c5c21d36c60104dcd86fa712890f3de57bd031 100644 (file)
@@ -39,10 +39,11 @@ import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeService {
+public abstract sealed class AbstractNetconfDataTreeService implements NetconfDataTreeService {
     private static final class Candidate extends AbstractNetconfDataTreeService {
-        Candidate(final RemoteDeviceId id, final NetconfBaseOps netconfOps, final boolean rollbackSupport) {
-            super(id, netconfOps, rollbackSupport);
+        Candidate(final RemoteDeviceId id, final NetconfBaseOps netconfOps, final boolean rollbackSupport,
+                final boolean lockDatastore) {
+            super(id, netconfOps, rollbackSupport, lockDatastore);
         }
 
         /**
@@ -74,8 +75,9 @@ public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeS
     }
 
     private static final class Running extends AbstractNetconfDataTreeService {
-        Running(final RemoteDeviceId id, final NetconfBaseOps netconfOps, final boolean rollbackSupport) {
-            super(id, netconfOps, rollbackSupport);
+        Running(final RemoteDeviceId id, final NetconfBaseOps netconfOps, final boolean rollbackSupport,
+                final boolean lockDatastore) {
+            super(id, netconfOps, rollbackSupport, lockDatastore);
         }
 
         @Override
@@ -114,10 +116,10 @@ public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeS
         private final Running running;
 
         CandidateWithRunning(final RemoteDeviceId id, final NetconfBaseOps netconfOps,
-                final boolean rollbackSupport) {
-            super(id, netconfOps, rollbackSupport);
-            candidate = new Candidate(id, netconfOps, rollbackSupport);
-            running = new Running(id, netconfOps, rollbackSupport);
+                final boolean rollbackSupport, final boolean lockDatastore) {
+            super(id, netconfOps, rollbackSupport, lockDatastore);
+            candidate = new Candidate(id, netconfOps, rollbackSupport, lockDatastore);
+            running = new Running(id, netconfOps, rollbackSupport, lockDatastore);
         }
 
         @Override
@@ -155,29 +157,29 @@ public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeS
     final NetconfBaseOps netconfOps;
     final boolean rollbackSupport;
 
-    // FIXME: what do we do with locks acquired before this got flipped?
-    private volatile boolean isLockAllowed = true;
+    private final boolean lockDatastore;
 
     AbstractNetconfDataTreeService(final RemoteDeviceId id, final NetconfBaseOps netconfOps,
-            final boolean rollbackSupport) {
+            final boolean rollbackSupport, final boolean lockDatastore) {
         this.id = requireNonNull(id);
         this.netconfOps = requireNonNull(netconfOps);
         this.rollbackSupport = rollbackSupport;
+        this.lockDatastore = lockDatastore;
     }
 
     public static @NonNull AbstractNetconfDataTreeService of(final RemoteDeviceId id,
             final MountPointContext mountContext, final Rpcs rpcs,
-            final NetconfSessionPreferences sessionPreferences) {
+            final NetconfSessionPreferences sessionPreferences, final boolean lockDatastore) {
         final var netconfOps = new NetconfBaseOps(rpcs, mountContext);
         final boolean rollbackSupport = sessionPreferences.isRollbackSupported();
 
         // Examine preferences and decide which implementation to use
         if (sessionPreferences.isCandidateSupported()) {
             return sessionPreferences.isRunningWritable()
-                ? new CandidateWithRunning(id, netconfOps, rollbackSupport)
-                    : new Candidate(id, netconfOps, rollbackSupport);
+                ? new CandidateWithRunning(id, netconfOps, rollbackSupport, lockDatastore)
+                    : new Candidate(id, netconfOps, rollbackSupport, lockDatastore);
         } else if (sessionPreferences.isRunningWritable()) {
-            return new Running(id, netconfOps, rollbackSupport);
+            return new Running(id, netconfOps, rollbackSupport, lockDatastore);
         } else {
             throw new IllegalArgumentException("Device " + id.getName() + " has advertised neither :writable-running "
                 + "nor :candidate capability. Failed to establish session, as at least one of these must be "
@@ -187,7 +189,7 @@ public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeS
 
     @Override
     public synchronized ListenableFuture<DOMRpcResult> lock() {
-        if (!isLockAllowed) {
+        if (!lockDatastore) {
             LOG.trace("Lock is not allowed by device configuration, ignoring lock results: {}", id);
             return RPC_SUCCESS;
         }
@@ -226,8 +228,7 @@ public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeS
 
     @Override
     public synchronized ListenableFuture<DOMRpcResult> unlock() {
-        // FIXME: deal with lock with lifecycle?
-        if (!isLockAllowed) {
+        if (!lockDatastore) {
             LOG.trace("Unlock is not allowed: {}", id);
             return RPC_SUCCESS;
         }
@@ -337,10 +338,6 @@ public abstract class AbstractNetconfDataTreeService implements NetconfDataTreeS
         return id;
     }
 
-    final void setLockAllowed(final boolean isLockAllowedOrig) {
-        isLockAllowed = isLockAllowedOrig;
-    }
-
     abstract ListenableFuture<? extends DOMRpcResult> editConfig(DataContainerChild editStructure,
         @Nullable ModifyAction defaultOperation);
 
diff --git a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/LockChangeListener.java b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/LockChangeListener.java
deleted file mode 100644 (file)
index 4cbdb42..0000000
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright (c) 2019 Lumina Networks, Inc. All rights reserved.
- *
- * This program and the accompanying materials are made available under the
- * terms of the Eclipse Public License v1.0 which accompanies this distribution,
- * and is available at http://www.eclipse.org/legal/epl-v10.html
- */
-package org.opendaylight.netconf.sal.connect.netconf.sal;
-
-import java.util.Collection;
-import org.opendaylight.mdsal.binding.api.DataObjectModification;
-import org.opendaylight.mdsal.binding.api.DataTreeChangeListener;
-import org.opendaylight.mdsal.binding.api.DataTreeModification;
-import org.opendaylight.mdsal.dom.api.DOMDataBroker;
-import org.opendaylight.netconf.dom.api.NetconfDataTreeService;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.netconf.node.fields.optional.topology.node.DatastoreLock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-final class LockChangeListener implements DataTreeChangeListener<DatastoreLock> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(LockChangeListener.class);
-
-    private final NetconfDeviceDataBroker netconfDeviceDataBroker;
-    private final AbstractNetconfDataTreeService netconfDataTreeService;
-
-    LockChangeListener(final DOMDataBroker netconfDeviceDataBrokder,
-                       final NetconfDataTreeService netconfDataTreeService) {
-        this.netconfDeviceDataBroker = (NetconfDeviceDataBroker)netconfDeviceDataBrokder;
-        this.netconfDataTreeService = (AbstractNetconfDataTreeService) netconfDataTreeService;
-    }
-
-    @Override
-    public void onDataTreeChanged(final Collection<DataTreeModification<DatastoreLock>> changes) {
-        for (final DataTreeModification<DatastoreLock> change : changes) {
-            final DataObjectModification<DatastoreLock> rootNode = change.getRootNode();
-            switch (rootNode.getModificationType()) {
-                case SUBTREE_MODIFIED:
-                case WRITE:
-                    if (!rootNode.getDataAfter().getDatastoreLockAllowed()) {
-                        LOG.warn("With blocking the lock/unlock operations, the user is coming to "
-                                 + "operate in a manner which is not supported. Concurrent access to "
-                                 + "the data store may interfere with data consistency.");
-                    }
-                    netconfDeviceDataBroker.setLockAllowed(rootNode.getDataAfter().getDatastoreLockAllowed());
-                    netconfDataTreeService.setLockAllowed(rootNode.getDataAfter().getDatastoreLockAllowed());
-                    break;
-                case DELETE:
-                    netconfDeviceDataBroker.setLockAllowed(true);
-                    netconfDataTreeService.setLockAllowed(true);
-                    break;
-                default:
-                    LOG.debug("Unsupported modification type: {}.", rootNode.getModificationType());
-            }
-        }
-    }
-}
index 84b383292ff74932353c8f08c1a8a7c4292d94eb..69d2c858b0702a10f1a86bbe83c8df6def3b3083 100644 (file)
@@ -7,7 +7,8 @@
  */
 package org.opendaylight.netconf.sal.connect.netconf.sal;
 
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.common.collect.ClassToInstanceMap;
 import com.google.common.collect.ImmutableClassToInstanceMap;
 import org.opendaylight.mdsal.dom.api.DOMDataBrokerExtension;
@@ -37,19 +38,16 @@ import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
 import org.opendaylight.yangtools.rfc8528.data.api.MountPointContext;
 
 public final class NetconfDeviceDataBroker implements PingPongMergingDOMDataBroker {
-
     private final NetconfDOMDataBrokerFieldsExtension fieldsExtension = new NetconfDOMDataBrokerFieldsExtensionImpl();
-
     private final RemoteDeviceId id;
     private final NetconfBaseOps netconfOps;
     private final boolean rollbackSupport;
     private final boolean candidateSupported;
     private final boolean runningWritable;
-
-    private boolean isLockAllowed = true;
+    private final boolean lockDatastore;
 
     public NetconfDeviceDataBroker(final RemoteDeviceId id, final MountPointContext mountContext, final Rpcs rpcs,
-            final NetconfSessionPreferences netconfSessionPreferences) {
+            final NetconfSessionPreferences netconfSessionPreferences, final boolean lockDatastore) {
         this.id = id;
         netconfOps = new NetconfBaseOps(rpcs, mountContext);
         // get specific attributes from netconf preferences and get rid of it
@@ -57,9 +55,10 @@ public final class NetconfDeviceDataBroker implements PingPongMergingDOMDataBrok
         candidateSupported = netconfSessionPreferences.isCandidateSupported();
         runningWritable = netconfSessionPreferences.isRunningWritable();
         rollbackSupport = netconfSessionPreferences.isRollbackSupported();
-        Preconditions.checkArgument(candidateSupported || runningWritable,
-            "Device %s has advertised neither :writable-running nor :candidate capability."
-                    + "At least one of these should be advertised. Failed to establish a session.", id.getName());
+        checkArgument(candidateSupported || runningWritable,
+            "Device %s has advertised neither :writable-running nor :candidate capability. At least one of these "
+                + "should be advertised. Failed to establish a session.", id.getName());
+        this.lockDatastore = lockDatastore;
     }
 
     @Override
@@ -76,12 +75,12 @@ public final class NetconfDeviceDataBroker implements PingPongMergingDOMDataBrok
     public DOMDataTreeWriteTransaction newWriteOnlyTransaction() {
         if (candidateSupported) {
             if (runningWritable) {
-                return new WriteCandidateRunningTx(id, netconfOps, rollbackSupport, isLockAllowed);
+                return new WriteCandidateRunningTx(id, netconfOps, rollbackSupport, lockDatastore);
             } else {
-                return new WriteCandidateTx(id, netconfOps, rollbackSupport, isLockAllowed);
+                return new WriteCandidateTx(id, netconfOps, rollbackSupport, lockDatastore);
             }
         } else {
-            return new WriteRunningTx(id, netconfOps, rollbackSupport, isLockAllowed);
+            return new WriteRunningTx(id, netconfOps, rollbackSupport, lockDatastore);
         }
     }
 
@@ -95,10 +94,6 @@ public final class NetconfDeviceDataBroker implements PingPongMergingDOMDataBrok
         return ImmutableClassToInstanceMap.of(NetconfDOMDataBrokerFieldsExtension.class, fieldsExtension);
     }
 
-    void setLockAllowed(final boolean isLockAllowedOrig) {
-        isLockAllowed = isLockAllowedOrig;
-    }
-
     private final class NetconfDOMDataBrokerFieldsExtensionImpl implements NetconfDOMDataBrokerFieldsExtension {
         @Override
         public NetconfDOMFieldsReadTransaction newReadOnlyTransaction() {
index 8703b14ed40a85a298015ab3025b214b7870ea99..a5dd2231a22a358018adbacac774a019388a0e79 100644 (file)
@@ -9,27 +9,14 @@ package org.opendaylight.netconf.sal.connect.netconf.sal;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.opendaylight.mdsal.binding.api.DataBroker;
-import org.opendaylight.mdsal.binding.api.DataTreeIdentifier;
-import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMNotification;
-import org.opendaylight.netconf.dom.api.NetconfDataTreeService;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceServices;
 import org.opendaylight.netconf.sal.connect.netconf.NetconfDeviceSchema;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCapabilities;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.NetconfNodeFieldsOptional;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.netconf.node.fields.optional.Topology;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.netconf.node.fields.optional.TopologyKey;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.netconf.node.fields.optional.topology.Node;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.netconf.node.fields.optional.topology.NodeKey;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.netconf.node.fields.optional.topology.node.DatastoreLock;
-import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
-import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.TopologyId;
-import org.opendaylight.yangtools.concepts.ListenerRegistration;
-import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,23 +25,19 @@ public final class NetconfDeviceSalFacade implements RemoteDeviceHandler, AutoCl
 
     private final RemoteDeviceId id;
     private final NetconfDeviceSalProvider salProvider;
-    private final DataBroker dataBroker;
-    private final String topologyId;
-
-    private ListenerRegistration<LockChangeListener> listenerRegistration = null;
+    private final boolean lockDatastore;
 
     public NetconfDeviceSalFacade(final RemoteDeviceId id, final DOMMountPointService mountPointService,
-            final DataBroker dataBroker, final String topologyId) {
-        this(id, new NetconfDeviceSalProvider(id, mountPointService, dataBroker), dataBroker, topologyId);
+            final DataBroker dataBroker, final boolean lockDatastore) {
+        this(id, new NetconfDeviceSalProvider(id, mountPointService, dataBroker), lockDatastore);
     }
 
     @VisibleForTesting
     NetconfDeviceSalFacade(final RemoteDeviceId id, final NetconfDeviceSalProvider salProvider,
-            final DataBroker dataBroker, final String topologyId) {
+            final boolean lockDatastore) {
         this.id = id;
         this.salProvider = salProvider;
-        this.dataBroker = dataBroker;
-        this.topologyId = topologyId;
+        this.lockDatastore = lockDatastore;
     }
 
     @Override
@@ -69,9 +52,11 @@ public final class NetconfDeviceSalFacade implements RemoteDeviceHandler, AutoCl
         final var modelContext = mountContext.getEffectiveModelContext();
 
         final var deviceRpc = services.rpcs();
-        final var netconfDataTree = AbstractNetconfDataTreeService.of(id, mountContext, deviceRpc, sessionPreferences);
-        final var netconfDataBroker = new NetconfDeviceDataBroker(id, mountContext, deviceRpc, sessionPreferences);
-        registerLockListener(netconfDataBroker, netconfDataTree);
+
+        final var netconfDataTree = AbstractNetconfDataTreeService.of(id, mountContext, deviceRpc, sessionPreferences,
+            lockDatastore);
+        final var netconfDataBroker = new NetconfDeviceDataBroker(id, mountContext, deviceRpc, sessionPreferences,
+            lockDatastore);
 
         salProvider.getMountInstance().onTopologyDeviceConnected(modelContext, services, netconfDataBroker,
             netconfDataTree);
@@ -82,20 +67,17 @@ public final class NetconfDeviceSalFacade implements RemoteDeviceHandler, AutoCl
     public synchronized void onDeviceDisconnected() {
         salProvider.getTopologyDatastoreAdapter().updateDeviceData(false, NetconfDeviceCapabilities.empty());
         salProvider.getMountInstance().onTopologyDeviceDisconnected();
-        closeLockChangeListener();
     }
 
     @Override
     public synchronized void onDeviceFailed(final Throwable throwable) {
         salProvider.getTopologyDatastoreAdapter().setDeviceAsFailed(throwable);
         salProvider.getMountInstance().onTopologyDeviceDisconnected();
-        closeLockChangeListener();
     }
 
     @Override
     public synchronized void close() {
         closeGracefully(salProvider);
-        closeLockChangeListener();
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
@@ -108,21 +90,4 @@ public final class NetconfDeviceSalFacade implements RemoteDeviceHandler, AutoCl
             }
         }
     }
-
-    private void closeLockChangeListener() {
-        if (listenerRegistration != null) {
-            listenerRegistration.close();
-        }
-    }
-
-    private void registerLockListener(final NetconfDeviceDataBroker netconfDeviceDataBroker,
-                                      final NetconfDataTreeService netconfDataTreeService) {
-        listenerRegistration = dataBroker.registerDataTreeChangeListener(
-            DataTreeIdentifier.create(LogicalDatastoreType.CONFIGURATION,
-                InstanceIdentifier.create(NetconfNodeFieldsOptional.class)
-                .child(Topology.class, new TopologyKey(new TopologyId(topologyId)))
-                .child(Node.class, new NodeKey(new NodeId(id.getName())))
-                .child(DatastoreLock.class)),
-            new LockChangeListener(netconfDeviceDataBroker, netconfDataTreeService));
-    }
 }
index 9ecbe6e4b74e4077e59292193c58ccd5015c5e3c..815b6e9d1553acd0e8ea5fbc39befafee6a7a064 100644 (file)
@@ -5,6 +5,10 @@ module netconf-node-optional {
     import network-topology { prefix nt; revision-date 2013-10-21; }
     import yang-ext { prefix ext; revision-date "2013-07-09";}
 
+    revision 2022-12-25 {
+        description "Datastore locking moved to odl-netconf-device.";
+    }
+
     revision "2019-06-14" {
         description "Initial revision of Node Optional model";
     }
@@ -27,40 +31,6 @@ module netconf-node-optional {
         }
     }
 
-    container netconf-node-fields-optional {
-        description "Allows to create node's optional value with the path mapping according to
-            the network-topology -> topology -> node";
-        list topology {
-            key topology-id;
-            leaf topology-id {
-                type nt:topology-id;
-                description "The name of node's topology";
-            }
-            list node {
-                key node-id;
-                leaf node-id {
-                    type nt:node-id;
-                    description "The identifier of a node in the topology";
-                }
-                // Containers allow to create specific data-change-listener directly on a node's optional value.
-                // In the future, it'll be easy to extend the node by optional node fields in this way. Do not create
-                // direct leafs here, please.
-                container datastore-lock {
-                    description "Allows to ignore lock/unlock node's datastare.";
-                    leaf datastore-lock-allowed {
-                        type boolean;
-                        default true;
-                        description "The operation allows the client to lock the entire configuration datastore
-                            system of a device.
-                            WARNING - With blocking the lock/unlock operations, the user is coming to operate
-                            in a manner which is not supported. Concurrent access to the data store may interfere
-                            with data consistency.";
-                    }
-                }
-            }
-        }
-    }
-
     augment "/nt:network-topology/nt:topology/nt:node/" {
         when "../../nt:topology-types/topology-netconf";
         ext:augment-identifier "netconf-node-augmented-optional";
index 4614290a734f206a39fd1f8cac2d1a274625f036..6582b7eda60cd59f19a958240c769377346edb7e 100644 (file)
@@ -4,7 +4,7 @@ module netconf-node-topology {
 
     import network-topology { prefix nt; revision-date 2013-10-21; }
     import yang-ext { prefix ext; revision-date "2013-07-09";}
-    import netconf-node-optional { prefix nno; revision-date "2019-06-14"; }
+    import netconf-node-optional { prefix nno; }
     import odl-netconf-device { prefix ond; }
 
     revision 2022-12-25 {
@@ -19,6 +19,7 @@ module netconf-node-topology {
       uses ond:credentials;
       uses ond:connection-parameters;
       uses ond:netconf-schema-storage;
+      uses ond:datastore-access;
     }
 
     rpc create-device {
index 268a227715ce2c989894ff1bf0bb944610301561..4c676548f794e23c62d132b11b28aa844683c8f9 100644 (file)
@@ -324,4 +324,16 @@ module odl-netconf-device {
       }
     }
   }
+
+  grouping datastore-access {
+    leaf lock-datastore {
+      type boolean;
+      default true;
+      description "The operation allows the client to lock the entire configuration datastore
+                   system of a device.
+                   WARNING - With blocking the lock/unlock operations, the user is coming to operate
+                   in a manner which is not supported. Concurrent access to the data store may interfere
+                   with data consistency.";
+    }
+  }
 }
index 89f54e85d8a7b0c9113320069749e6cf94991581..5f45176f1876692d2c2a98df7809d454de94c240 100644 (file)
@@ -162,6 +162,7 @@ public class NetconfDataTreeServiceImplTest extends AbstractTestModelTest {
                 List.of(NetconfMessageTransformUtil.NETCONF_CANDIDATE_URI.toString()));
         final RemoteDeviceId id =
                 new RemoteDeviceId("device-1", InetSocketAddress.createUnresolved("localhost", 17830));
-        return AbstractNetconfDataTreeService.of(id, new EmptyMountPointContext(SCHEMA_CONTEXT), rpcService, prefs);
+        return AbstractNetconfDataTreeService.of(id, new EmptyMountPointContext(SCHEMA_CONTEXT), rpcService, prefs,
+            true);
     }
 }
index b7a91d2884cda16a8ff1a457d45ddca403bdedcd..7340598a2f32739d572a01ba61c0083db5dc1d64 100644 (file)
@@ -133,7 +133,6 @@ public class NetconfDeviceDataBrokerTest {
         NetconfSessionPreferences prefs = NetconfSessionPreferences.fromStrings(List.of(caps));
         final RemoteDeviceId id =
                 new RemoteDeviceId("device-1", InetSocketAddress.createUnresolved("localhost", 17830));
-        return new NetconfDeviceDataBroker(id, new EmptyMountPointContext(SCHEMA_CONTEXT), rpcService, prefs);
+        return new NetconfDeviceDataBroker(id, new EmptyMountPointContext(SCHEMA_CONTEXT), rpcService, prefs, true);
     }
-
 }
index 3106a3eedca8073bc96f97c690ed502d56d1d80c..91cb8a7f619b509be134c7053c74325f7773f670 100644 (file)
@@ -75,7 +75,7 @@ public class NetconfDeviceSalFacadeTest {
 
         final NetconfDeviceTopologyAdapter adapter = new NetconfDeviceTopologyAdapter(dataBroker, remoteDeviceId);
 
-        deviceFacade = new NetconfDeviceSalFacade(remoteDeviceId, salProvider, dataBroker, "mockTopo");
+        deviceFacade = new NetconfDeviceSalFacade(remoteDeviceId, salProvider, true);
 
         doReturn(adapter).when(salProvider).getTopologyDatastoreAdapter();