Move RemoteDeviceId
[netconf.git] / netconf / netconf-topology-singleton / src / main / java / org / opendaylight / netconf / topology / singleton / impl / MasterSalFacade.java
index 44727a0c6ee4c801a47be4c07905e6001bc94262..55f661d513640261f813f852cde4c7803eb03854 100644 (file)
@@ -16,78 +16,73 @@ import akka.dispatch.OnComplete;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
 import java.util.List;
-import java.util.stream.Collectors;
 import org.opendaylight.mdsal.binding.api.DataBroker;
-import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMNotification;
-import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.netconf.dom.api.NetconfDataTreeService;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
+import org.opendaylight.netconf.sal.connect.api.RemoteDeviceId;
+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.netconf.sal.AbstractNetconfDataTreeService;
 import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceDataBroker;
-import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceNotificationService;
-import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceSalProvider;
-import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
+import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceMount;
 import org.opendaylight.netconf.topology.singleton.messages.CreateInitialMasterActorData;
+import org.opendaylight.netconf.topology.spi.NetconfDeviceTopologyAdapter;
 import org.opendaylight.yangtools.rfc8528.data.api.MountPointContext;
-import org.opendaylight.yangtools.yang.model.repo.api.RevisionSourceIdentifier;
 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
 import org.opendaylight.yangtools.yang.model.util.SchemaContextUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
 
-class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessionPreferences> {
-
+class MasterSalFacade implements RemoteDeviceHandler, AutoCloseable {
     private static final Logger LOG = LoggerFactory.getLogger(MasterSalFacade.class);
 
     private final RemoteDeviceId id;
     private final Timeout actorResponseWaitTime;
-    private final NetconfDeviceSalProvider salProvider;
     private final ActorRef masterActorRef;
     private final ActorSystem actorSystem;
+    private final NetconfDeviceTopologyAdapter datastoreAdapter;
+    private final NetconfDeviceMount mount;
+    private final boolean lockDatastore;
 
-    private MountPointContext currentMountContext = null;
+    private NetconfDeviceSchema currentSchema = null;
     private NetconfSessionPreferences netconfSessionPreferences = null;
-    private DOMRpcService deviceRpc = null;
+    private RemoteDeviceServices deviceServices = null;
     private DOMDataBroker deviceDataBroker = null;
     private NetconfDataTreeService netconfService = null;
-    private DOMActionService deviceAction = null;
 
     MasterSalFacade(final RemoteDeviceId id,
                     final ActorSystem actorSystem,
                     final ActorRef masterActorRef,
                     final Timeout actorResponseWaitTime,
                     final DOMMountPointService mountService,
-                    final DataBroker dataBroker) {
+                    final DataBroker dataBroker,
+                    final boolean lockDatastore) {
         this.id = id;
-        this.salProvider = new NetconfDeviceSalProvider(id, mountService, dataBroker);
+        mount = new NetconfDeviceMount(id, mountService, NetconfDeviceMount.defaultTopologyMountPath(id));
         this.actorSystem = actorSystem;
         this.masterActorRef = masterActorRef;
         this.actorResponseWaitTime = actorResponseWaitTime;
-    }
+        this.lockDatastore = lockDatastore;
 
-    @Override
-    public void onDeviceConnected(final MountPointContext mountContext,
-                                  final NetconfSessionPreferences sessionPreferences,
-                                  final DOMRpcService domRpcService, final DOMActionService domActionService) {
-        this.deviceAction = domActionService;
-        LOG.debug("{}: YANG 1.1 actions are supported in clustered netconf topology, "
-            + "DOMActionService exposed for the device", id);
-        onDeviceConnected(mountContext, sessionPreferences, domRpcService);
+        datastoreAdapter = new NetconfDeviceTopologyAdapter(dataBroker, RemoteDeviceId.DEFAULT_TOPOLOGY_IID, id);
     }
 
     @Override
-    public void onDeviceConnected(final MountPointContext mountContext,
-                                  final NetconfSessionPreferences sessionPreferences,
-                                  final DOMRpcService domRpcService) {
-        this.currentMountContext = mountContext;
-        this.netconfSessionPreferences = sessionPreferences;
-        this.deviceRpc = domRpcService;
+    public void onDeviceConnected(final NetconfDeviceSchema deviceSchema,
+            final NetconfSessionPreferences sessionPreferences, final RemoteDeviceServices services) {
+        currentSchema = requireNonNull(deviceSchema);
+        netconfSessionPreferences = requireNonNull(sessionPreferences);
+        deviceServices = requireNonNull(services);
+        if (services.actions() != null) {
+            LOG.debug("{}: YANG 1.1 actions are supported in clustered netconf topology, DOMActionService exposed for "
+                + "the device", id);
+        }
 
         LOG.info("Device {} connected - registering master mount point", id);
 
@@ -104,41 +99,43 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
                 LOG.error("{}: CreateInitialMasterActorData to {} failed", id, masterActorRef, failure);
             }
         }, actorSystem.dispatcher());
-
     }
 
     @Override
     public void onDeviceDisconnected() {
         LOG.info("Device {} disconnected - unregistering master mount point", id);
-        salProvider.getTopologyDatastoreAdapter().updateDeviceData(false, new NetconfDeviceCapabilities());
-        unregisterMasterMountPoint();
+        datastoreAdapter.updateDeviceData(false, NetconfDeviceCapabilities.empty());
+        mount.onDeviceDisconnected();
     }
 
     @Override
     public void onDeviceFailed(final Throwable throwable) {
-        salProvider.getTopologyDatastoreAdapter().setDeviceAsFailed(throwable);
-        unregisterMasterMountPoint();
+        datastoreAdapter.setDeviceAsFailed(throwable);
+        mount.onDeviceDisconnected();
     }
 
     @Override
     public void onNotification(final DOMNotification domNotification) {
-        salProvider.getMountInstance().publish(domNotification);
+        mount.publish(domNotification);
     }
 
     @Override
     public void close() {
-        unregisterMasterMountPoint();
-        closeGracefully(salProvider);
+        datastoreAdapter.close();
+        mount.close();
     }
 
     private void registerMasterMountPoint() {
         requireNonNull(id);
-        requireNonNull(currentMountContext, "Device has no remote schema context yet. Probably not fully connected.");
-        requireNonNull(netconfSessionPreferences, "Device has no capabilities yet. Probably not fully connected.");
 
-        final NetconfDeviceNotificationService notificationService = new NetconfDeviceNotificationService();
-        deviceDataBroker = newDeviceDataBroker();
-        netconfService = newNetconfDataTreeService();
+        final var mountContext = requireNonNull(currentSchema,
+            "Device has no remote schema context yet. Probably not fully connected.")
+            .mountContext();
+        final var preferences = requireNonNull(netconfSessionPreferences,
+            "Device has no capabilities yet. Probably not fully connected.");
+
+        deviceDataBroker = newDeviceDataBroker(mountContext, preferences);
+        netconfService = newNetconfDataTreeService(mountContext, preferences);
 
         // We need to create ProxyDOMDataBroker so accessing mountpoint
         // on leader node would be same as on follower node
@@ -146,51 +143,35 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
             actorResponseWaitTime);
         final NetconfDataTreeService proxyNetconfService = new ProxyNetconfDataTreeService(id, masterActorRef,
             actorSystem.dispatcher(), actorResponseWaitTime);
-        salProvider.getMountInstance().onTopologyDeviceConnected(currentMountContext.getEffectiveModelContext(),
-            proxyDataBroker, proxyNetconfService, deviceRpc, notificationService, deviceAction);
+        mount.onDeviceConnected(mountContext.getEffectiveModelContext(), deviceServices,
+            proxyDataBroker, proxyNetconfService);
     }
 
-    protected DOMDataBroker newDeviceDataBroker() {
-        return new NetconfDeviceDataBroker(id, currentMountContext, deviceRpc, netconfSessionPreferences);
+    protected DOMDataBroker newDeviceDataBroker(final MountPointContext mountContext,
+            final NetconfSessionPreferences preferences) {
+        return new NetconfDeviceDataBroker(id, mountContext, deviceServices.rpcs(), preferences, lockDatastore);
     }
 
-    protected NetconfDataTreeService newNetconfDataTreeService() {
-        return AbstractNetconfDataTreeService.of(id, currentMountContext, deviceRpc, netconfSessionPreferences);
+    protected NetconfDataTreeService newNetconfDataTreeService(final MountPointContext mountContext,
+            final NetconfSessionPreferences preferences) {
+        return AbstractNetconfDataTreeService.of(id, mountContext, deviceServices.rpcs(), preferences, lockDatastore);
     }
 
     private Future<Object> sendInitialDataToActor() {
-        final List<SourceIdentifier> sourceIdentifiers = SchemaContextUtil.getConstituentModuleIdentifiers(
-            currentMountContext.getEffectiveModelContext()).stream()
-                .map(mi -> RevisionSourceIdentifier.create(mi.getName(), mi.getRevision()))
-                .collect(Collectors.toList());
+        final List<SourceIdentifier> sourceIdentifiers = List.copyOf(SchemaContextUtil.getConstituentModuleIdentifiers(
+            currentSchema.mountContext().getEffectiveModelContext()));
 
         LOG.debug("{}: Sending CreateInitialMasterActorData with sourceIdentifiers {} to {}", id, sourceIdentifiers,
             masterActorRef);
 
         // send initial data to master actor
         return Patterns.ask(masterActorRef, new CreateInitialMasterActorData(deviceDataBroker, netconfService,
-            sourceIdentifiers, deviceRpc, deviceAction), actorResponseWaitTime);
+            sourceIdentifiers, deviceServices), actorResponseWaitTime);
     }
 
     private void updateDeviceData() {
         final String masterAddress = Cluster.get(actorSystem).selfAddress().toString();
         LOG.debug("{}: updateDeviceData with master address {}", id, masterAddress);
-        salProvider.getTopologyDatastoreAdapter().updateClusteredDeviceData(true, masterAddress,
-                netconfSessionPreferences.getNetconfDeviceCapabilities());
-    }
-
-    private void unregisterMasterMountPoint() {
-        salProvider.getMountInstance().onTopologyDeviceDisconnected();
-    }
-
-    @SuppressWarnings("checkstyle:IllegalCatch")
-    private void closeGracefully(final AutoCloseable resource) {
-        if (resource != null) {
-            try {
-                resource.close();
-            } catch (final Exception e) {
-                LOG.error("{}: Ignoring exception while closing {}", id, resource, e);
-            }
-        }
+        datastoreAdapter.updateClusteredDeviceData(true, masterAddress, currentSchema.capabilities());
     }
 }