From 22d8a6b30c7678b605a384f46a930a8b543de34a Mon Sep 17 00:00:00 2001 From: Robert Varga Date: Sat, 31 Dec 2022 18:45:16 +0100 Subject: [PATCH] Eliminate LockChangeListener 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 --- .../singleton/impl/MasterSalFacade.java | 9 ++- .../impl/NetconfTopologyContext.java | 10 +++- .../topology/spi/AbstractNetconfTopology.java | 5 +- .../sal/AbstractNetconfDataTreeService.java | 43 +++++++------- .../netconf/sal/LockChangeListener.java | 57 ------------------- .../netconf/sal/NetconfDeviceDataBroker.java | 27 ++++----- .../netconf/sal/NetconfDeviceSalFacade.java | 55 ++++-------------- .../src/main/yang/netconf-node-optional.yang | 38 ++----------- .../src/main/yang/netconf-node-topology.yang | 3 +- .../src/main/yang/odl-netconf-device.yang | 12 ++++ .../sal/NetconfDataTreeServiceImplTest.java | 3 +- .../sal/NetconfDeviceDataBrokerTest.java | 3 +- .../sal/NetconfDeviceSalFacadeTest.java | 2 +- 13 files changed, 79 insertions(+), 188 deletions(-) delete mode 100644 netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/LockChangeListener.java diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java index f33a2ba4a0..e747f2a8f1 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java @@ -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 sendInitialDataToActor() { diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyContext.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyContext.java index 4c92f3c0e3..6b070d08a7 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyContext.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyContext.java @@ -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 diff --git a/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/spi/AbstractNetconfTopology.java b/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/spi/AbstractNetconfTopology.java index 1b881c409e..039f222693 100644 --- a/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/spi/AbstractNetconfTopology.java +++ b/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/spi/AbstractNetconfTopology.java @@ -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; diff --git a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/AbstractNetconfDataTreeService.java b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/AbstractNetconfDataTreeService.java index 90d95d5e27..e7c5c21d36 100644 --- a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/AbstractNetconfDataTreeService.java +++ b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/AbstractNetconfDataTreeService.java @@ -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 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 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 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 index 4cbdb4298e..0000000000 --- a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/LockChangeListener.java +++ /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 { - - 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> changes) { - for (final DataTreeModification change : changes) { - final DataObjectModification 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()); - } - } - } -} diff --git a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBroker.java b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBroker.java index 84b383292f..69d2c858b0 100644 --- a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBroker.java +++ b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBroker.java @@ -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() { diff --git a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacade.java b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacade.java index 8703b14ed4..a5dd2231a2 100644 --- a/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacade.java +++ b/netconf/sal-netconf-connector/src/main/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacade.java @@ -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 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)); - } } diff --git a/netconf/sal-netconf-connector/src/main/yang/netconf-node-optional.yang b/netconf/sal-netconf-connector/src/main/yang/netconf-node-optional.yang index 9ecbe6e4b7..815b6e9d15 100644 --- a/netconf/sal-netconf-connector/src/main/yang/netconf-node-optional.yang +++ b/netconf/sal-netconf-connector/src/main/yang/netconf-node-optional.yang @@ -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"; diff --git a/netconf/sal-netconf-connector/src/main/yang/netconf-node-topology.yang b/netconf/sal-netconf-connector/src/main/yang/netconf-node-topology.yang index 4614290a73..6582b7eda6 100644 --- a/netconf/sal-netconf-connector/src/main/yang/netconf-node-topology.yang +++ b/netconf/sal-netconf-connector/src/main/yang/netconf-node-topology.yang @@ -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 { diff --git a/netconf/sal-netconf-connector/src/main/yang/odl-netconf-device.yang b/netconf/sal-netconf-connector/src/main/yang/odl-netconf-device.yang index 268a227715..4c676548f7 100644 --- a/netconf/sal-netconf-connector/src/main/yang/odl-netconf-device.yang +++ b/netconf/sal-netconf-connector/src/main/yang/odl-netconf-device.yang @@ -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."; + } + } } diff --git a/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDataTreeServiceImplTest.java b/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDataTreeServiceImplTest.java index 89f54e85d8..5f45176f18 100644 --- a/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDataTreeServiceImplTest.java +++ b/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDataTreeServiceImplTest.java @@ -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); } } diff --git a/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBrokerTest.java b/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBrokerTest.java index b7a91d2884..7340598a2f 100644 --- a/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBrokerTest.java +++ b/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceDataBrokerTest.java @@ -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); } - } diff --git a/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacadeTest.java b/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacadeTest.java index 3106a3eedc..91cb8a7f61 100644 --- a/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacadeTest.java +++ b/netconf/sal-netconf-connector/src/test/java/org/opendaylight/netconf/sal/connect/netconf/sal/NetconfDeviceSalFacadeTest.java @@ -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(); -- 2.36.6