Teach NETCONF about YANG 1.1 actions in cluster topology 93/83693/11
authorajay.dp001 <ajay.deep.singh@ericsson.com>
Fri, 16 Aug 2019 11:38:39 +0000 (12:38 +0100)
committerRobert Varga <nite@hq.sk>
Wed, 28 Aug 2019 08:53:38 +0000 (08:53 +0000)
NETCONF allows invocation of actions but only in Non-cluster
topology which works absolutly fine but when invoked in cluster
topology it failed, usecase was not supported.

This patch provides support for Yang 1.1 action in cluster topology
NETCONF code base is updated to support action, can be invoked in
cluster topology.

JIRA: NETCONF-635
Change-Id: Ic7345f02224b098eafb0bd7c42f1d619309fe12d
Signed-off-by: ajay.dp001 <ajay.deep.singh@ericsson.com>
Signed-off-by: Robert Varga <robert.varga@pantheon.tech>
22 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-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManager.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/ProxyDOMActionService.java [new file with mode: 0644]
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImpl.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/SlaveSalFacade.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/actors/NetconfNodeActor.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/ClusteringActionException.java [new file with mode: 0644]
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/NetconfTopologySetup.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/ContainerNodeMessage.java [new file with mode: 0644]
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/CreateInitialMasterActorData.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/SchemaPathMessage.java
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessage.java [new file with mode: 0644]
netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessageReply.java [new file with mode: 0644]
netconf/netconf-topology-singleton/src/main/resources/OSGI-INF/blueprint/netconf-topology-singleton.xml
netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/MountPointEndToEndTest.java
netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeActorTest.java
netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeManagerTest.java
netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManagerTest.java
netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImplTest.java
netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/TestingRemoteDeviceConnectorImpl.java
netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/AbstractNetconfTopology.java

index 1b7068a442fa4aa2a780966a93c0f1656ed833a4..e0c5e062cbb26ed1396944fe31c19ec18f531956 100644 (file)
@@ -52,6 +52,7 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
     private NetconfSessionPreferences netconfSessionPreferences = null;
     private DOMRpcService deviceRpc = null;
     private DOMDataBroker deviceDataBroker = null;
+    private DOMActionService deviceAction = null;
 
     MasterSalFacade(final RemoteDeviceId id,
                     final ActorSystem actorSystem,
@@ -70,12 +71,9 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
     public void onDeviceConnected(final SchemaContext remoteSchemaContext,
                                   final NetconfSessionPreferences sessionPreferences,
                                   final DOMRpcService domRpcService, final DOMActionService domActionService) {
-        // We do not support actions in clustered topology yet
-        if (domActionService != null) {
-            LOG.warn("{}: YANG 1.1 actions are not supported in clustered netconf topology, "
-                    + "DOMActionService will not be exposed for the device", id);
-        }
-
+        this.deviceAction = domActionService;
+        LOG.debug("{}: YANG 1.1 actions are supported in clustered netconf topology, "
+            + "DOMActionService exposed for the device", id);
         onDeviceConnected(remoteSchemaContext, sessionPreferences, domRpcService);
     }
 
@@ -137,15 +135,14 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
                 "Device has no capabilities yet. Probably not fully connected.");
 
         final NetconfDeviceNotificationService notificationService = new NetconfDeviceNotificationService();
-
         deviceDataBroker = newDeviceDataBroker();
 
         // We need to create ProxyDOMDataBroker so accessing mountpoint
         // on leader node would be same as on follower node
-        final ProxyDOMDataBroker proxyDataBroker =
-                new ProxyDOMDataBroker(id, masterActorRef, actorSystem.dispatcher(), actorResponseWaitTime);
-        salProvider.getMountInstance()
-                .onTopologyDeviceConnected(currentSchemaContext, proxyDataBroker, deviceRpc, notificationService);
+        final ProxyDOMDataBroker proxyDataBroker = new ProxyDOMDataBroker(id, masterActorRef, actorSystem.dispatcher(),
+            actorResponseWaitTime);
+        salProvider.getMountInstance().onTopologyDeviceConnected(currentSchemaContext,
+            proxyDataBroker, deviceRpc, notificationService, deviceAction);
     }
 
     protected DOMDataBroker newDeviceDataBroker() {
@@ -158,12 +155,12 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
                 .map(mi -> RevisionSourceIdentifier.create(mi.getName(), mi.getRevision()))
                 .collect(Collectors.toList());
 
-        LOG.debug("{}: Sending CreateInitialMasterActorData with sourceIdentifiers {} to {}",
-                id, sourceIdentifiers, masterActorRef);
+        LOG.debug("{}: Sending CreateInitialMasterActorData with sourceIdentifiers {} to {}", id, sourceIdentifiers,
+            masterActorRef);
 
         // send initial data to master actor
         return Patterns.ask(masterActorRef, new CreateInitialMasterActorData(deviceDataBroker, sourceIdentifiers,
-                deviceRpc), actorResponseWaitTime);
+                deviceRpc, deviceAction), actorResponseWaitTime);
     }
 
     private void updateDeviceData() {
@@ -187,5 +184,4 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler<NetconfSessi
             }
         }
     }
-
-}
+}
\ No newline at end of file
index 673e8a82988f0ad47a574afc4e78b6a1b264b9cd..56a2ae5c8352fe9b2c581ad4c96627deef87d29b 100644 (file)
@@ -20,6 +20,7 @@ import org.eclipse.jdt.annotation.NonNull;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonService;
 import org.opendaylight.mdsal.singleton.common.api.ServiceGroupIdentifier;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
 import org.opendaylight.netconf.topology.singleton.api.RemoteDeviceConnector;
 import org.opendaylight.netconf.topology.singleton.impl.actors.NetconfNodeActor;
@@ -39,6 +40,7 @@ class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
     private final ServiceGroupIdentifier serviceGroupIdent;
     private final Timeout actorResponseWaitTime;
     private final DOMMountPointService mountService;
+    private final DeviceActionFactory deviceActionFactory;
 
     private NetconfTopologySetup netconfTopologyDeviceSetup;
     private RemoteDeviceId remoteDeviceId;
@@ -50,18 +52,18 @@ class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
     private volatile boolean isMaster;
 
     NetconfTopologyContext(final NetconfTopologySetup netconfTopologyDeviceSetup,
-                           final ServiceGroupIdentifier serviceGroupIdent,
-                           final Timeout actorResponseWaitTime, final DOMMountPointService mountService) {
+            final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime,
+            final DOMMountPointService mountService, final DeviceActionFactory deviceActionFactory) {
         this.netconfTopologyDeviceSetup = requireNonNull(netconfTopologyDeviceSetup);
         this.serviceGroupIdent = serviceGroupIdent;
         this.actorResponseWaitTime = actorResponseWaitTime;
         this.mountService = mountService;
+        this.deviceActionFactory = deviceActionFactory;
 
         remoteDeviceId = NetconfTopologyUtils.createRemoteDeviceId(netconfTopologyDeviceSetup.getNode().getNodeId(),
-                netconfTopologyDeviceSetup.getNode().augmentation(NetconfNode.class));
-
-        remoteDeviceConnector = new RemoteDeviceConnectorImpl(netconfTopologyDeviceSetup, remoteDeviceId);
-
+            netconfTopologyDeviceSetup.getNode().augmentation(NetconfNode.class));
+        remoteDeviceConnector = new RemoteDeviceConnectorImpl(netconfTopologyDeviceSetup, remoteDeviceId,
+            deviceActionFactory);
         netconfNodeManager = createNodeDeviceManager();
     }
 
@@ -144,12 +146,12 @@ class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
         if (!isMaster) {
             netconfNodeManager.refreshDevice(netconfTopologyDeviceSetup, remoteDeviceId);
         }
-        remoteDeviceConnector = new RemoteDeviceConnectorImpl(netconfTopologyDeviceSetup, remoteDeviceId);
+        remoteDeviceConnector = new RemoteDeviceConnectorImpl(netconfTopologyDeviceSetup, remoteDeviceId,
+            deviceActionFactory);
 
         if (isMaster) {
             final Future<Object> future = Patterns.ask(masterActorRef, new RefreshSetupMasterActorData(
-                    netconfTopologyDeviceSetup, remoteDeviceId), actorResponseWaitTime);
-
+                netconfTopologyDeviceSetup, remoteDeviceId), actorResponseWaitTime);
             future.onComplete(new OnComplete<Object>() {
                 @Override
                 public void onComplete(final Throwable failure, final Object success) {
@@ -181,4 +183,4 @@ class NetconfTopologyContext implements ClusterSingletonService, AutoCloseable {
         return new MasterSalFacade(remoteDeviceId, netconfTopologyDeviceSetup.getActorSystem(), masterActorRef,
                 actorResponseWaitTime, mountService, netconfTopologyDeviceSetup.getDataBroker());
     }
-}
+}
\ No newline at end of file
index 397b5087010403261a227b80f591edf1d11f6fcf..dfe0687e0938ccceea7be1c1eb6441b7846660d2 100644 (file)
@@ -33,12 +33,14 @@ import org.opendaylight.mdsal.binding.api.DataTreeModification;
 import org.opendaylight.mdsal.binding.api.WriteTransaction;
 import org.opendaylight.mdsal.common.api.CommitInfo;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.DOMActionProviderService;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMRpcProviderService;
 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegistration;
 import org.opendaylight.mdsal.singleton.common.api.ServiceGroupIdentifier;
 import org.opendaylight.netconf.client.NetconfClientDispatcher;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.topology.singleton.api.NetconfTopologySingletonService;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup.NetconfTopologySetupBuilder;
@@ -70,6 +72,7 @@ public class NetconfTopologyManager
 
     private final DataBroker dataBroker;
     private final DOMRpcProviderService rpcProviderRegistry;
+    private final DOMActionProviderService actionProviderRegistry;
     private final ClusterSingletonServiceProvider clusterSingletonServiceProvider;
     private final ScheduledExecutorService keepaliveExecutor;
     private final ListeningExecutorService processingExecutor;
@@ -80,21 +83,25 @@ public class NetconfTopologyManager
     private final Duration writeTxIdleTimeout;
     private final DOMMountPointService mountPointService;
     private final AAAEncryptionService encryptionService;
+    private final DeviceActionFactory deviceActionFactory;
     private ListenerRegistration<NetconfTopologyManager> dataChangeListenerRegistration;
     private String privateKeyPath;
     private String privateKeyPassphrase;
 
     public NetconfTopologyManager(final DataBroker dataBroker, final DOMRpcProviderService rpcProviderRegistry,
+                                  final DOMActionProviderService actionProviderService,
                                   final ClusterSingletonServiceProvider clusterSingletonServiceProvider,
                                   final ScheduledThreadPool keepaliveExecutor, final ThreadPool processingExecutor,
                                   final ActorSystemProvider actorSystemProvider,
                                   final EventExecutor eventExecutor, final NetconfClientDispatcher clientDispatcher,
                                   final String topologyId, final Config config,
                                   final DOMMountPointService mountPointService,
-                                  final AAAEncryptionService encryptionService) {
+                                  final AAAEncryptionService encryptionService,
+                                  final DeviceActionFactory deviceActionFactory) {
 
         this.dataBroker = requireNonNull(dataBroker);
         this.rpcProviderRegistry = requireNonNull(rpcProviderRegistry);
+        this.actionProviderRegistry = requireNonNull(actionProviderService);
         this.clusterSingletonServiceProvider = requireNonNull(clusterSingletonServiceProvider);
         this.keepaliveExecutor = keepaliveExecutor.getExecutor();
         this.processingExecutor = MoreExecutors.listeningDecorator(processingExecutor.getExecutor());
@@ -105,6 +112,7 @@ public class NetconfTopologyManager
         this.writeTxIdleTimeout = Duration.apply(config.getWriteTransactionIdleTimeout(), TimeUnit.SECONDS);
         this.mountPointService = mountPointService;
         this.encryptionService = requireNonNull(encryptionService);
+        this.deviceActionFactory = requireNonNull(deviceActionFactory);
 
     }
 
@@ -166,7 +174,7 @@ public class NetconfTopologyManager
                 ServiceGroupIdentifier.create(instanceIdentifier.toString());
 
         final NetconfTopologyContext newNetconfTopologyContext = newNetconfTopologyContext(
-                createSetup(instanceIdentifier, node), serviceGroupIdent, actorResponseWaitTime);
+            createSetup(instanceIdentifier, node), serviceGroupIdent, actorResponseWaitTime, deviceActionFactory);
 
         int tries = 3;
         while (true) {
@@ -199,8 +207,10 @@ public class NetconfTopologyManager
 
     @VisibleForTesting
     protected NetconfTopologyContext newNetconfTopologyContext(final NetconfTopologySetup setup,
-            final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime) {
-        return new NetconfTopologyContext(setup, serviceGroupIdent, actorResponseWaitTime, mountPointService);
+            final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime,
+            final DeviceActionFactory deviceActionFact) {
+        return new NetconfTopologyContext(setup, serviceGroupIdent, actorResponseWaitTime, mountPointService,
+            deviceActionFact);
     }
 
     @Override
@@ -277,6 +287,7 @@ public class NetconfTopologyManager
                 .setDataBroker(dataBroker)
                 .setInstanceIdentifier(instanceIdentifier)
                 .setRpcProviderRegistry(rpcProviderRegistry)
+                .setActionProviderRegistry(actionProviderRegistry)
                 .setNode(node)
                 .setActorSystem(actorSystem)
                 .setEventExecutor(eventExecutor)
diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/ProxyDOMActionService.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/ProxyDOMActionService.java
new file mode 100644 (file)
index 0000000..c3f61b6
--- /dev/null
@@ -0,0 +1,127 @@
+/*
+ * Copyright (C) 2019 Ericsson Software Technology AB. 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.topology.singleton.impl;
+
+import static java.util.Objects.requireNonNull;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.dispatch.OnComplete;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import com.google.common.collect.ClassToInstanceMap;
+import com.google.common.collect.ImmutableClassToInstanceMap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.FluentFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import java.util.Collection;
+import org.opendaylight.mdsal.dom.api.DOMActionResult;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
+import org.opendaylight.mdsal.dom.api.DOMActionServiceExtension;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
+import org.opendaylight.mdsal.dom.spi.SimpleDOMActionResult;
+import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
+import org.opendaylight.netconf.topology.singleton.impl.utils.ClusteringActionException;
+import org.opendaylight.netconf.topology.singleton.messages.ContainerNodeMessage;
+import org.opendaylight.netconf.topology.singleton.messages.SchemaPathMessage;
+import org.opendaylight.netconf.topology.singleton.messages.action.InvokeActionMessage;
+import org.opendaylight.netconf.topology.singleton.messages.action.InvokeActionMessageReply;
+import org.opendaylight.netconf.topology.singleton.messages.transactions.EmptyResultResponse;
+import org.opendaylight.yangtools.yang.common.RpcError;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.model.api.SchemaPath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+
+/**
+ * Implementation of {@link DOMActionService} provided by device in Odl-Cluster environment to invoke action.
+ * Communicates action message {@link InvokeActionMessage} to {@link ActorSystem} using {@link ActorRef} and transforms
+ * replied NETCONF message to action result, using {@link SimpleDOMActionResult}.
+ */
+public class ProxyDOMActionService implements DOMActionService {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ProxyDOMActionService.class);
+
+    private final RemoteDeviceId id;
+    private final ActorRef masterActorRef;
+    private final ActorSystem actorSystem;
+    private final Timeout actorResponseWaitTime;
+
+    /**
+     * Constructor for {@code ProxyDOMActionService}.
+     *
+     * @param actorSystem ActorSystem
+     * @param masterActorRef ActorRef
+     * @param remoteDeviceId {@link org.opendaylight.netconf.sal.connect.util.RemoteDeviceId} ref
+     * @param actorResponseWaitTime Timeout
+     */
+    public ProxyDOMActionService(final ActorSystem actorSystem, final ActorRef masterActorRef,
+        final RemoteDeviceId remoteDeviceId, final Timeout actorResponseWaitTime) {
+        id = remoteDeviceId;
+        this.actorSystem = requireNonNull(actorSystem);
+        this.masterActorRef = requireNonNull(masterActorRef);
+        this.actorResponseWaitTime = requireNonNull(actorResponseWaitTime);
+    }
+
+    @Override
+    public FluentFuture<DOMActionResult> invokeAction(final SchemaPath type,
+        final DOMDataTreeIdentifier domDataTreeIdentifier, final ContainerNode input) {
+        requireNonNull(type);
+        requireNonNull(input);
+        requireNonNull(domDataTreeIdentifier);
+
+        LOG.info("{}: Action Operation invoked with schema type: {} and node: {}.", id, type, input);
+        final ContainerNodeMessage containerNodeMessage = new ContainerNodeMessage(input);
+
+        final Future<Object> scalaFuture = Patterns.ask(masterActorRef, new InvokeActionMessage(
+            new SchemaPathMessage(type), containerNodeMessage, domDataTreeIdentifier), actorResponseWaitTime);
+
+        final SettableFuture<DOMActionResult> settableFuture = SettableFuture.create();
+
+        scalaFuture.onComplete(new OnComplete<Object>() {
+            @Override
+            public void onComplete(final Throwable failure, final Object response) {
+                if (failure != null) {
+                    if (failure instanceof ClusteringActionException) {
+                        settableFuture.setException(failure);
+                    } else {
+                        settableFuture.setException(new ClusteringActionException(
+                            id + ": Exception during remote Action invocation.", failure));
+                    }
+                    return;
+                }
+
+                if (response instanceof EmptyResultResponse) {
+                    settableFuture.set(null);
+                    return;
+                }
+                final Collection<? extends RpcError> errors = ((InvokeActionMessageReply) response).getRpcErrors();
+
+                final ContainerNodeMessage containerNodeMessage =
+                    ((InvokeActionMessageReply) response).getContainerNodeMessage();
+
+                final DOMActionResult result;
+
+                if (containerNodeMessage == null) {
+                    result = new SimpleDOMActionResult(ImmutableList.copyOf(errors));
+                } else {
+                    result = new SimpleDOMActionResult(containerNodeMessage.getNode(), ImmutableList.copyOf(errors));
+                }
+                settableFuture.set(result);
+            }
+        }, actorSystem.dispatcher());
+
+        return FluentFuture.from(settableFuture);
+    }
+
+    @Override
+    public ClassToInstanceMap<DOMActionServiceExtension> getExtensions() {
+        return ImmutableClassToInstanceMap.of();
+    }
+}
index 8d32fb3014f1f62f26aaa735a048dd3a789f4ee2..907a67167c946066f6068bed243b89ebba8b0921 100644 (file)
@@ -33,6 +33,7 @@ import org.opendaylight.netconf.nettyutil.ReconnectStrategyFactory;
 import org.opendaylight.netconf.nettyutil.TimedReconnectStrategyFactory;
 import org.opendaylight.netconf.nettyutil.handler.ssh.authentication.AuthenticationHandler;
 import org.opendaylight.netconf.nettyutil.handler.ssh.authentication.LoginPasswordHandler;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.sal.connect.api.RemoteDevice;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
 import org.opendaylight.netconf.sal.connect.netconf.LibraryModulesSchemas;
@@ -89,12 +90,13 @@ public class RemoteDeviceConnectorImpl implements RemoteDeviceConnector {
     private final AAAEncryptionService encryptionService;
     private NetconfConnectorDTO deviceCommunicatorDTO;
     private final NetconfKeystoreAdapter keystoreAdapter;
+    private final DeviceActionFactory deviceActionFactory;
 
     public RemoteDeviceConnectorImpl(final NetconfTopologySetup netconfTopologyDeviceSetup,
-                                     final RemoteDeviceId remoteDeviceId) {
-
+            final RemoteDeviceId remoteDeviceId, final DeviceActionFactory deviceActionFactory) {
         this.netconfTopologyDeviceSetup = requireNonNull(netconfTopologyDeviceSetup);
         this.remoteDeviceId = remoteDeviceId;
+        this.deviceActionFactory = requireNonNull(deviceActionFactory);
         this.privateKeyPath = netconfTopologyDeviceSetup.getPrivateKeyPath();
         this.privateKeyPassphrase = netconfTopologyDeviceSetup.getPrivateKeyPassphrase();
         this.encryptionService = netconfTopologyDeviceSetup.getEncryptionService();
@@ -200,6 +202,7 @@ public class RemoteDeviceConnectorImpl implements RemoteDeviceConnector {
                     .setSchemaResourcesDTO(schemaResourcesDTO)
                     .setGlobalProcessingExecutor(netconfTopologyDeviceSetup.getProcessingExecutor())
                     .setId(remoteDeviceId)
+                    .setDeviceActionFactory(deviceActionFactory)
                     .setSalFacade(salFacade)
                     .build();
         }
index 4fbed61aed444feb64864887a3c357d8a5d50c58..c70de74b8861480c6d7dae242c1336901271c49a 100644 (file)
@@ -11,6 +11,7 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfDeviceNotificationService;
@@ -41,18 +42,17 @@ public class SlaveSalFacade {
     }
 
     public void registerSlaveMountPoint(final SchemaContext remoteSchemaContext, final DOMRpcService deviceRpc,
-                                        final ActorRef masterActorRef) {
+            final DOMActionService deviceAction, final ActorRef masterActorRef) {
         if (!registered.compareAndSet(false, true)) {
             return;
         }
 
         final NetconfDeviceNotificationService notificationService = new NetconfDeviceNotificationService();
-
-        final ProxyDOMDataBroker netconfDeviceDataBroker =
-                new ProxyDOMDataBroker(id, masterActorRef, actorSystem.dispatcher(), actorResponseWaitTime);
+        final ProxyDOMDataBroker netconfDeviceDataBroker = new ProxyDOMDataBroker(id, masterActorRef,
+            actorSystem.dispatcher(), actorResponseWaitTime);
 
         salProvider.getMountInstance().onTopologyDeviceConnected(remoteSchemaContext, netconfDeviceDataBroker,
-                deviceRpc, notificationService);
+            deviceRpc, notificationService, deviceAction);
 
         LOG.info("{}: Slave mount point registered.", id);
     }
index a3f73e1446d045c3a00d0320d859b67a4a442a14..e99834d07a74dacdd7bf4c41df0e8e3d6bc33e4a 100644 (file)
@@ -25,7 +25,10 @@ import org.opendaylight.controller.cluster.common.actor.AbstractUntypedActor;
 import org.opendaylight.controller.cluster.schema.provider.RemoteYangTextSourceProvider;
 import org.opendaylight.controller.cluster.schema.provider.impl.RemoteSchemaProvider;
 import org.opendaylight.controller.cluster.schema.provider.impl.YangTextSchemaSourceSerializationProxy;
+import org.opendaylight.mdsal.dom.api.DOMActionResult;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
@@ -33,11 +36,13 @@ import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMRpcResult;
 import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
+import org.opendaylight.netconf.topology.singleton.impl.ProxyDOMActionService;
 import org.opendaylight.netconf.topology.singleton.impl.ProxyDOMRpcService;
 import org.opendaylight.netconf.topology.singleton.impl.ProxyYangTextSourceProvider;
 import org.opendaylight.netconf.topology.singleton.impl.SlaveSalFacade;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup;
 import org.opendaylight.netconf.topology.singleton.messages.AskForMasterMountPoint;
+import org.opendaylight.netconf.topology.singleton.messages.ContainerNodeMessage;
 import org.opendaylight.netconf.topology.singleton.messages.CreateInitialMasterActorData;
 import org.opendaylight.netconf.topology.singleton.messages.MasterActorDataInitialized;
 import org.opendaylight.netconf.topology.singleton.messages.NormalizedNodeMessage;
@@ -47,6 +52,8 @@ import org.opendaylight.netconf.topology.singleton.messages.RefreshSlaveActor;
 import org.opendaylight.netconf.topology.singleton.messages.RegisterMountPoint;
 import org.opendaylight.netconf.topology.singleton.messages.UnregisterSlaveMountPoint;
 import org.opendaylight.netconf.topology.singleton.messages.YangTextSchemaSourceRequest;
+import org.opendaylight.netconf.topology.singleton.messages.action.InvokeActionMessage;
+import org.opendaylight.netconf.topology.singleton.messages.action.InvokeActionMessageReply;
 import org.opendaylight.netconf.topology.singleton.messages.rpc.InvokeRpcMessage;
 import org.opendaylight.netconf.topology.singleton.messages.rpc.InvokeRpcMessageReply;
 import org.opendaylight.netconf.topology.singleton.messages.transactions.EmptyResultResponse;
@@ -77,6 +84,7 @@ public class NetconfNodeActor extends AbstractUntypedActor {
     private NetconfTopologySetup setup;
     private List<SourceIdentifier> sourceIdentifiers;
     private DOMRpcService deviceRpc;
+    private DOMActionService deviceAction;
     private SlaveSalFacade slaveSalManager;
     private DOMDataBroker deviceDataBroker;
     //readTxActor can be shared
@@ -114,23 +122,24 @@ public class NetconfNodeActor extends AbstractUntypedActor {
             final DOMDataTreeReadTransaction tx = deviceDataBroker.newReadOnlyTransaction();
             readTxActor = context().actorOf(ReadTransactionActor.props(tx));
             this.deviceRpc = masterActorData.getDeviceRpc();
+            this.deviceAction = masterActorData.getDeviceAction();
 
             sender().tell(new MasterActorDataInitialized(), self());
 
             LOG.debug("{}: Master is ready.", id);
 
-        } else if (message instanceof  RefreshSetupMasterActorData) {
+        } else if (message instanceof RefreshSetupMasterActorData) {
             setup = ((RefreshSetupMasterActorData) message).getNetconfTopologyDeviceSetup();
             id = ((RefreshSetupMasterActorData) message).getRemoteDeviceId();
             sender().tell(new MasterActorDataInitialized(), self());
         } else if (message instanceof AskForMasterMountPoint) { // master
-            AskForMasterMountPoint askForMasterMountPoint = (AskForMasterMountPoint)message;
+            AskForMasterMountPoint askForMasterMountPoint = (AskForMasterMountPoint) message;
 
             // only master contains reference to deviceDataBroker
             if (deviceDataBroker != null) {
                 LOG.debug("{}: Sending RegisterMountPoint reply to {}", id, askForMasterMountPoint.getSlaveActorRef());
                 askForMasterMountPoint.getSlaveActorRef().tell(new RegisterMountPoint(sourceIdentifiers, self()),
-                        sender());
+                    sender());
             } else {
                 LOG.warn("{}: Received {} but we don't appear to be the master", id, askForMasterMountPoint);
                 sender().tell(new Failure(new NotMasterException(self())), self());
@@ -163,9 +172,13 @@ public class NetconfNodeActor extends AbstractUntypedActor {
         } else if (message instanceof InvokeRpcMessage) { // master
             final InvokeRpcMessage invokeRpcMessage = (InvokeRpcMessage) message;
             invokeSlaveRpc(invokeRpcMessage.getSchemaPath(), invokeRpcMessage.getNormalizedNodeMessage(), sender());
-
+        } else if (message instanceof InvokeActionMessage) { // master
+            final InvokeActionMessage invokeActionMessage = (InvokeActionMessage) message;
+            LOG.info("InvokeActionMessage Details : {}", invokeActionMessage.toString());
+            invokeSlaveAction(invokeActionMessage.getSchemaPath(), invokeActionMessage.getContainerNodeMessage(),
+                invokeActionMessage.getDOMDataTreeIdentifier(), sender());
         } else if (message instanceof RegisterMountPoint) { //slaves
-            RegisterMountPoint registerMountPoint = (RegisterMountPoint)message;
+            RegisterMountPoint registerMountPoint = (RegisterMountPoint) message;
             sourceIdentifiers = registerMountPoint.getSourceIndentifiers();
             registerSlaveMountPoint(registerMountPoint.getMasterActorRef());
             sender().tell(new Success(null), self());
@@ -178,7 +191,6 @@ public class NetconfNodeActor extends AbstractUntypedActor {
             setup = ((RefreshSlaveActor) message).getSetup();
             schemaRepository = ((RefreshSlaveActor) message).getSchemaRepository();
         }
-
     }
 
     @Override
@@ -255,6 +267,45 @@ public class NetconfNodeActor extends AbstractUntypedActor {
         }, MoreExecutors.directExecutor());
     }
 
+    /**
+     * Invoking Action on Slave Node in Odl Cluster Environment.
+     *
+     * @param schemaPath {@link SchemaPath}
+     * @param containerNodeMessage {@link ContainerNodeMessage}
+     * @param domDataTreeIdentifier {@link DOMDataTreeIdentifier}
+     * @param recipient {@link ActorRef}
+     */
+    private void invokeSlaveAction(final SchemaPath schemaPath, final ContainerNodeMessage containerNodeMessage,
+        final DOMDataTreeIdentifier domDataTreeIdentifier, final ActorRef recipient) {
+        LOG.info("{}: invokeSlaveAction for {}, input: {}, identifier: {} on action service {}", id, schemaPath,
+            containerNodeMessage, domDataTreeIdentifier, deviceAction);
+
+        final ListenableFuture<? extends DOMActionResult> actionResult = deviceAction.invokeAction(schemaPath,
+            domDataTreeIdentifier, containerNodeMessage != null ? containerNodeMessage.getNode() : null);
+
+        Futures.addCallback(actionResult, new FutureCallback<DOMActionResult>() {
+
+            @Override
+            public void onSuccess(final DOMActionResult domActionResult) {
+                LOG.debug("{}: invokeSlaveAction for {}, domActionResult: {}", id, schemaPath, domActionResult);
+                if (domActionResult == null) {
+                    recipient.tell(new EmptyResultResponse(), getSender());
+                    return;
+                }
+
+                //Check DomActionResult containing Ok onSuccess pass empty nodeMessageReply
+                ContainerNodeMessage nodeMessageReply = domActionResult.getOutput().map(ContainerNodeMessage::new)
+                        .orElse(null);
+                recipient.tell(new InvokeActionMessageReply(nodeMessageReply, domActionResult.getErrors()), getSelf());
+            }
+
+            @Override
+            public void onFailure(final Throwable throwable) {
+                recipient.tell(new Failure(throwable), getSelf());
+            }
+        }, MoreExecutors.directExecutor());
+    }
+
     private void registerSlaveMountPoint(final ActorRef masterReference) {
         unregisterSlaveMountPoint();
 
@@ -267,6 +318,10 @@ public class NetconfNodeActor extends AbstractUntypedActor {
         return new ProxyDOMRpcService(setup.getActorSystem(), masterReference, id, actorResponseWaitTime);
     }
 
+    private DOMActionService getDOMActionService(final ActorRef masterReference) {
+        return new ProxyDOMActionService(setup.getActorSystem(), masterReference, id, actorResponseWaitTime);
+    }
+
     private EffectiveModelContextFactory createSchemaContextFactory(final ActorRef masterReference) {
         final RemoteYangTextSourceProvider remoteYangTextSourceProvider =
                 new ProxyYangTextSourceProvider(masterReference, getContext().dispatcher(), actorResponseWaitTime);
@@ -296,7 +351,7 @@ public class NetconfNodeActor extends AbstractUntypedActor {
                         LOG.info("{}: Schema context resolved: {} - registering slave mount point",
                                 id, result.getModules());
                         slaveSalManager.registerSlaveMountPoint(result, getDOMRpcService(masterReference),
-                                masterReference);
+                            getDOMActionService(masterReference), masterReference);
                     }
                 });
             }
@@ -330,4 +385,4 @@ public class NetconfNodeActor extends AbstractUntypedActor {
             registeredSchemas = null;
         }
     }
-}
+}
\ No newline at end of file
diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/ClusteringActionException.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/ClusteringActionException.java
new file mode 100644 (file)
index 0000000..6d348a3
--- /dev/null
@@ -0,0 +1,36 @@
+/*
+ * Copyright (C) 2019 Ericsson Software Technology AB. 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.topology.singleton.impl.utils;
+
+import org.opendaylight.mdsal.dom.api.DOMActionException;
+
+/**
+ * Exception thrown during remote action invocation in Odl-Cluster environment.
+ */
+public class ClusteringActionException extends DOMActionException {
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Constructor for {@code ClusteringActionException}.
+     *
+     * @param message String
+     */
+    public ClusteringActionException(final String message) {
+        super(message);
+    }
+
+    /**
+     * Constructor for {@code ClusteringActionException}.
+     *
+     * @param message String
+     * @param cause Throwable
+     */
+    public ClusteringActionException(final String message, final Throwable cause) {
+        super(message, cause);
+    }
+}
\ No newline at end of file
index 7963d440f0793c5c237cff45ca91af4e6ebe6f20..4224d1eb5179896871d8c53f3c6eac3338831b2e 100644 (file)
@@ -13,6 +13,7 @@ import io.netty.util.concurrent.EventExecutor;
 import java.util.concurrent.ScheduledExecutorService;
 import org.opendaylight.aaa.encrypt.AAAEncryptionService;
 import org.opendaylight.mdsal.binding.api.DataBroker;
+import org.opendaylight.mdsal.dom.api.DOMActionProviderService;
 import org.opendaylight.mdsal.dom.api.DOMRpcProviderService;
 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
 import org.opendaylight.netconf.client.NetconfClientDispatcher;
@@ -25,6 +26,7 @@ public class NetconfTopologySetup {
 
     private final ClusterSingletonServiceProvider clusterSingletonServiceProvider;
     private final DOMRpcProviderService rpcProviderRegistry;
+    private final DOMActionProviderService actionProviderRegistry;
     private final DataBroker dataBroker;
     private final InstanceIdentifier<Node> instanceIdentifier;
     private final Node node;
@@ -43,6 +45,7 @@ public class NetconfTopologySetup {
     NetconfTopologySetup(final NetconfTopologySetupBuilder builder) {
         this.clusterSingletonServiceProvider = builder.getClusterSingletonServiceProvider();
         this.rpcProviderRegistry = builder.getRpcProviderRegistry();
+        this.actionProviderRegistry = builder.getActionProviderRegistry();
         this.dataBroker = builder.getDataBroker();
         this.instanceIdentifier = builder.getInstanceIdentifier();
         this.node = builder.getNode();
@@ -67,6 +70,10 @@ public class NetconfTopologySetup {
         return rpcProviderRegistry;
     }
 
+    public DOMActionProviderService getActionProviderRegistry() {
+        return actionProviderRegistry;
+    }
+
     public DataBroker getDataBroker() {
         return dataBroker;
     }
@@ -127,6 +134,7 @@ public class NetconfTopologySetup {
 
         private ClusterSingletonServiceProvider clusterSingletonServiceProvider;
         private DOMRpcProviderService rpcProviderRegistry;
+        private DOMActionProviderService actionProviderRegistry;
         private DataBroker dataBroker;
         private InstanceIdentifier<Node> instanceIdentifier;
         private Node node;
@@ -164,6 +172,16 @@ public class NetconfTopologySetup {
             return this;
         }
 
+        private DOMActionProviderService getActionProviderRegistry() {
+            return actionProviderRegistry;
+        }
+
+        public NetconfTopologySetupBuilder setActionProviderRegistry(
+            final DOMActionProviderService actionProviderRegistry) {
+            this.actionProviderRegistry = actionProviderRegistry;
+            return this;
+        }
+
         private DataBroker getDataBroker() {
             return dataBroker;
         }
@@ -299,6 +317,4 @@ public class NetconfTopologySetup {
             return new NetconfTopologySetupBuilder();
         }
     }
-
-
-}
+}
\ No newline at end of file
diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/ContainerNodeMessage.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/ContainerNodeMessage.java
new file mode 100644 (file)
index 0000000..5091c59
--- /dev/null
@@ -0,0 +1,59 @@
+/*
+ * Copyright (C) 2019 Ericsson Software Technology AB. 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.topology.singleton.messages;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.opendaylight.controller.cluster.datastore.node.utils.stream.SerializationUtils;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+
+/**
+ * Message container which holds node data {@link ContainerNode}, prepared to send between remote hosts with
+ * serialization when remote action is invoked.
+ */
+public class ContainerNodeMessage implements Externalizable {
+    private static final long serialVersionUID = 1L;
+
+    private ContainerNode node;
+
+    public ContainerNodeMessage() {
+        // Empty Constructor Needed for Externalizable
+    }
+
+    /**
+     * Constructor for {@code ContainerNodeMessage}.
+     *
+     * @param node ContainerNode
+     */
+    public ContainerNodeMessage(final ContainerNode node) {
+        this.node = requireNonNull(node);
+    }
+
+    public ContainerNode getNode() {
+        return node;
+    }
+
+    @Override
+    public void writeExternal(final ObjectOutput out) throws IOException {
+        SerializationUtils.writeNormalizedNode(out, node);
+    }
+
+    @Override
+    public void readExternal(final ObjectInput in) throws IOException {
+        node = (ContainerNode) SerializationUtils.readNormalizedNode(in).get();
+    }
+
+    @Override
+    public String toString() {
+        return "ContainerNodeMessage [node=" + node + "]";
+    }
+}
index 509ecb0e55ab28b1ce2d2af7a0475ae5ea08ed3d..8c6bc6b0e94637fc4be5404911d28afe5b7db8c9 100644 (file)
@@ -8,6 +8,7 @@
 package org.opendaylight.netconf.topology.singleton.messages;
 
 import java.util.List;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
 import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
@@ -19,13 +20,15 @@ public class CreateInitialMasterActorData {
     private final DOMDataBroker deviceDataBroker;
     private final List<SourceIdentifier> allSourceIdentifiers;
     private final DOMRpcService deviceRpc;
+    private final DOMActionService deviceAction;
 
     public CreateInitialMasterActorData(final DOMDataBroker deviceDataBroker,
                                         final List<SourceIdentifier> allSourceIdentifiers,
-                                        final DOMRpcService deviceRpc) {
+                                        final DOMRpcService deviceRpc,final DOMActionService deviceAction) {
         this.deviceDataBroker = deviceDataBroker;
         this.allSourceIdentifiers = allSourceIdentifiers;
         this.deviceRpc = deviceRpc;
+        this.deviceAction = deviceAction;
     }
 
     public DOMDataBroker getDeviceDataBroker() {
@@ -39,4 +42,8 @@ public class CreateInitialMasterActorData {
     public DOMRpcService getDeviceRpc() {
         return deviceRpc;
     }
+
+    public DOMActionService getDeviceAction() {
+        return deviceAction;
+    }
 }
index 2e7e0619b9b578f465a83787e6252b033f41d9ec..5f170036c5456cca2eeb275f491229f8cc3de79d 100644 (file)
@@ -57,9 +57,10 @@ public class SchemaPathMessage implements Serializable {
 
         @Override
         public void writeExternal(final ObjectOutput out) throws IOException {
-            out.writeInt(Iterables.size(schemaPathMessage.getSchemaPath().getPathTowardsRoot()));
-
-            for (final QName qualifiedName : schemaPathMessage.getSchemaPath().getPathTowardsRoot()) {
+            final Iterable<QName> path = schemaPathMessage.getSchemaPath().getPathFromRoot();
+            out.writeInt(Iterables.size(path));
+            for (final QName qualifiedName : path) {
+                // FIXME: switch to QName.writeTo() or a sal-clustering-commons stream
                 out.writeObject(qualifiedName);
             }
 
@@ -71,6 +72,7 @@ public class SchemaPathMessage implements Serializable {
             final int sizePath = in.readInt();
             final QName[] paths = new QName[sizePath];
             for (int i = 0; i < sizePath; i++) {
+                // FIXME: switch to QName.readFrom() or a sal-clustering-commons stream
                 paths[i] = (QName) in.readObject();
             }
             final boolean absolute = in.readBoolean();
diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessage.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessage.java
new file mode 100644 (file)
index 0000000..58a4404
--- /dev/null
@@ -0,0 +1,106 @@
+/*
+ * Copyright (C) 2019 Ericsson Software Technology AB. 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.topology.singleton.messages.action;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import org.eclipse.jdt.annotation.Nullable;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
+import org.opendaylight.netconf.topology.singleton.messages.ContainerNodeMessage;
+import org.opendaylight.netconf.topology.singleton.messages.SchemaPathMessage;
+import org.opendaylight.yangtools.yang.model.api.SchemaPath;
+
+/**
+ * Message container which holds node data in {@link SchemaPathMessage}, {@link ContainerNodeMessage} and
+ * {@link DOMDataTreeIdentifier} prepared to send between remote hosts with serialization when action operation is
+ * invoked.
+ */
+public class InvokeActionMessage implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final SchemaPathMessage schemaPathMessage;
+    private final ContainerNodeMessage containerNodeMessage;
+    private final DOMDataTreeIdentifier domDataTreeIdentifier;
+
+    /**
+     * Constructor for {@code InvokeActionMessage}.
+     *
+     * @param schemaPathMessage SchemaPathMessage
+     * @param containerNodeMessage ContainerNodeMessage
+     * @param domDataTreeIdentifier DOMDataTreeIdentifier
+     */
+    public InvokeActionMessage(final SchemaPathMessage schemaPathMessage,
+        final @Nullable ContainerNodeMessage containerNodeMessage, final DOMDataTreeIdentifier domDataTreeIdentifier) {
+        this.schemaPathMessage = requireNonNull(schemaPathMessage);
+        this.containerNodeMessage = requireNonNull(containerNodeMessage);
+        this.domDataTreeIdentifier = requireNonNull(domDataTreeIdentifier);
+    }
+
+    public SchemaPath getSchemaPath() {
+        return schemaPathMessage.getSchemaPath();
+    }
+
+    private SchemaPathMessage getSchemaPathMessage() {
+        return schemaPathMessage;
+    }
+
+    public @Nullable ContainerNodeMessage getContainerNodeMessage() {
+        return containerNodeMessage;
+    }
+
+    public DOMDataTreeIdentifier getDOMDataTreeIdentifier() {
+        return domDataTreeIdentifier;
+    }
+
+    private Object writeReplace() {
+        return new Proxy(this);
+    }
+
+    @Override
+    public String toString() {
+        return "InvokeActionMessage [schemaPathMessage=" + schemaPathMessage + ", containerNodeMessage="
+            + containerNodeMessage + ", domDataTreeIdentifier=" + domDataTreeIdentifier + "]";
+    }
+
+    private static class Proxy implements Externalizable {
+        private static final long serialVersionUID = 2L;
+
+        private InvokeActionMessage invokeActionMessage;
+
+        @SuppressWarnings("checkstyle:RedundantModifier")
+        public Proxy() {
+            //Due to Externalizable
+        }
+
+        Proxy(final InvokeActionMessage invokeActionMessage) {
+            this.invokeActionMessage = invokeActionMessage;
+        }
+
+        @Override
+        public void writeExternal(final ObjectOutput out) throws IOException {
+            out.writeObject(invokeActionMessage.getSchemaPathMessage());
+            out.writeObject(invokeActionMessage.getContainerNodeMessage());
+            out.writeObject(invokeActionMessage.getDOMDataTreeIdentifier());
+        }
+
+        @Override
+        public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException {
+            invokeActionMessage = new InvokeActionMessage((SchemaPathMessage) in.readObject(),
+                (ContainerNodeMessage) in.readObject(), (DOMDataTreeIdentifier) in.readObject());
+        }
+
+        private Object readResolve() {
+            return invokeActionMessage;
+        }
+    }
+}
diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessageReply.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessageReply.java
new file mode 100644 (file)
index 0000000..07ddfe9
--- /dev/null
@@ -0,0 +1,99 @@
+/*
+ * Copyright (C) 2019 Ericsson Software Technology AB. 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.topology.singleton.messages.action;
+
+import static java.util.Objects.requireNonNull;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import org.eclipse.jdt.annotation.NonNull;
+import org.eclipse.jdt.annotation.Nullable;
+import org.opendaylight.netconf.topology.singleton.messages.ContainerNodeMessage;
+import org.opendaylight.yangtools.yang.common.RpcError;
+
+/**
+ * Message container which holds node reply in {@link ContainerNodeMessage}, {@link RpcError} prepared to send between
+ * remote hosts with serialization when action operation is invoked.
+ */
+public class InvokeActionMessageReply implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    @SuppressFBWarnings("SE_BAD_FIELD")
+    private final Collection<? extends RpcError> rpcErrors;
+    private final ContainerNodeMessage containerNodeMessage;
+
+    /**
+     * Constructor for {@code InvokeActionMessage}.
+     *
+     * @param containerNodeMessage ContainerNodeMessage
+     * @param rpcErrors RpcError
+     */
+    public InvokeActionMessageReply(final @Nullable ContainerNodeMessage containerNodeMessage,
+        final @NonNull Collection<? extends RpcError> rpcErrors) {
+        this.containerNodeMessage = requireNonNull(containerNodeMessage);
+        this.rpcErrors = requireNonNull(rpcErrors);
+    }
+
+    public @Nullable ContainerNodeMessage getContainerNodeMessage() {
+        return containerNodeMessage;
+    }
+
+    public @NonNull Collection<? extends RpcError> getRpcErrors() {
+        return rpcErrors;
+    }
+
+    private Object writeReplace() {
+        return new Proxy(this);
+    }
+
+    private static class Proxy implements Externalizable {
+        private static final long serialVersionUID = 2L;
+
+        private InvokeActionMessageReply invokeActionMessageReply;
+
+        @SuppressWarnings("checkstyle:RedundantModifier")
+        public Proxy() {
+            //due to Externalizable
+        }
+
+        Proxy(final InvokeActionMessageReply invokeActionMessageReply) {
+            this.invokeActionMessageReply = invokeActionMessageReply;
+        }
+
+        @Override
+        public void writeExternal(final ObjectOutput out) throws IOException {
+            out.writeInt(invokeActionMessageReply.getRpcErrors().size());
+            for (final RpcError rpcError : invokeActionMessageReply.getRpcErrors()) {
+                out.writeObject(rpcError);
+            }
+            out.writeObject(invokeActionMessageReply.getContainerNodeMessage());
+        }
+
+        @Override
+        public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException {
+            final int size = in.readInt();
+            final Collection<RpcError> rpcErrors = new ArrayList<>(size);
+            for (int i = 0; i < size; i++) {
+                rpcErrors.add((RpcError) in.readObject());
+            }
+
+            final ContainerNodeMessage containerNodeMessage = (ContainerNodeMessage) in.readObject();
+            invokeActionMessageReply = new InvokeActionMessageReply(containerNodeMessage, rpcErrors);
+        }
+
+        private Object readResolve() {
+            return invokeActionMessageReply;
+        }
+    }
+}
index 4138656a67f3d6704d4b02819bdc22e124ca3cab..91ef4f842b33c4b8d23ee731381c8f8ce01fa1c9 100644 (file)
@@ -16,6 +16,8 @@ and is available at http://www.eclipse.org/legal/epl-v10.html
                interface="org.opendaylight.mdsal.binding.api.DataBroker"/>
     <reference id="rpcRegistry"
                interface="org.opendaylight.mdsal.dom.api.DOMRpcProviderService"/>
+    <reference id="actionRegistry"
+               interface="org.opendaylight.mdsal.dom.api.DOMActionProviderService"/>
     <reference id="clusterSingletonService"
                interface="org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider"/>
     <reference id="keepAliveExecutor"
@@ -35,6 +37,8 @@ and is available at http://www.eclipse.org/legal/epl-v10.html
     <reference id="mountPointService"
                interface="org.opendaylight.mdsal.dom.api.DOMMountPointService"
                odl:type="default"/>
+    <reference id="deviceActionFactory"
+               interface="org.opendaylight.netconf.sal.connect.api.DeviceActionFactory"/>
     <odl:clustered-app-config
             id="singletonConfig"
             binding-class="org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.topology.singleton.config.rev170419.Config"
@@ -57,6 +61,7 @@ and is available at http://www.eclipse.org/legal/epl-v10.html
                                update-strategy="container-managed"/>
         <argument ref="dataBroker"/>
         <argument ref="rpcRegistry"/>
+        <argument ref="actionRegistry"/>
         <argument ref="clusterSingletonService"/>
         <argument ref="keepAliveExecutor"/>
         <argument ref="processingExecutor"/>
@@ -69,6 +74,7 @@ and is available at http://www.eclipse.org/legal/epl-v10.html
         <property name="privateKeyPath" value="${private-key-path}"/>
         <property name="privateKeyPassphrase" value="${private-key-passphrase}"/>
         <argument ref="encryptionService" />
+        <argument ref="deviceActionFactory"/>
     </bean>
     <service ref="netconfTopologyManager"
              interface="org.opendaylight.netconf.topology.singleton.api.NetconfTopologySingletonService"/>
index 2713850ddf43180af62861286cac24ab07ddfc86..be2e580ff1602a523135f46a0e00abf2e890ab1d 100644 (file)
@@ -71,6 +71,8 @@ import org.opendaylight.mdsal.binding.dom.codec.api.BindingNormalizedNodeSeriali
 import org.opendaylight.mdsal.binding.generator.impl.ModuleInfoBackedContext;
 import org.opendaylight.mdsal.binding.spec.reflect.BindingReflections;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.DOMActionProviderService;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadOperations;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
@@ -93,6 +95,7 @@ import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegist
 import org.opendaylight.mdsal.singleton.common.api.ServiceGroupIdentifier;
 import org.opendaylight.mdsal.singleton.dom.impl.DOMClusterSingletonServiceProviderImpl;
 import org.opendaylight.netconf.client.NetconfClientDispatcher;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
 import org.opendaylight.netconf.topology.singleton.impl.utils.ClusteringRpcException;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup;
@@ -161,10 +164,12 @@ public class MountPointEndToEndTest {
             new NodeKey(NODE_ID), TOPOLOGY_ID);
 
     @Mock private DOMRpcProviderService mockRpcProviderRegistry;
+    @Mock private DOMActionProviderService mockActionProviderRegistry;
     @Mock private NetconfClientDispatcher mockClientDispatcher;
     @Mock private AAAEncryptionService mockEncryptionService;
     @Mock private ThreadPool mockThreadPool;
     @Mock private ScheduledThreadPool mockKeepaliveExecutor;
+    @Mock private DeviceActionFactory deviceActionFactory;
 
     @Mock private ActorSystemProvider mockMasterActorSystemProvider;
     @Mock private DOMMountPointListener masterMountPointListener;
@@ -265,14 +270,16 @@ public class MountPointEndToEndTest {
                     topModuleInfo.getName().getRevision()), YangTextSchemaSource.class, 1));
 
         masterNetconfTopologyManager = new NetconfTopologyManager(masterDataBroker, mockRpcProviderRegistry,
-                masterClusterSingletonServiceProvider, mockKeepaliveExecutor, mockThreadPool,
+            mockActionProviderRegistry, masterClusterSingletonServiceProvider, mockKeepaliveExecutor, mockThreadPool,
                 mockMasterActorSystemProvider, eventExecutor, mockClientDispatcher, TOPOLOGY_ID, config,
-                masterMountPointService, mockEncryptionService) {
+                masterMountPointService, mockEncryptionService, deviceActionFactory) {
             @Override
             protected NetconfTopologyContext newNetconfTopologyContext(final NetconfTopologySetup setup,
-                    final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime) {
+                final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime,
+                final DeviceActionFactory deviceActionFact) {
                 NetconfTopologyContext context =
-                        super.newNetconfTopologyContext(setup, serviceGroupIdent, actorResponseWaitTime);
+                    super.newNetconfTopologyContext(setup, serviceGroupIdent, actorResponseWaitTime, deviceActionFact);
+
                 NetconfTopologyContext spiedContext = spy(context);
                 doAnswer(invocation -> {
                     final MasterSalFacade spiedFacade = (MasterSalFacade) spy(invocation.callRealMethod());
@@ -302,14 +309,16 @@ public class MountPointEndToEndTest {
                 .registerClusterSingletonService(any());
 
         slaveNetconfTopologyManager = new NetconfTopologyManager(slaveDataBroker, mockRpcProviderRegistry,
-                mockSlaveClusterSingletonServiceProvider, mockKeepaliveExecutor, mockThreadPool,
+            mockActionProviderRegistry, mockSlaveClusterSingletonServiceProvider, mockKeepaliveExecutor, mockThreadPool,
                 mockSlaveActorSystemProvider, eventExecutor, mockClientDispatcher, TOPOLOGY_ID, config,
-                slaveMountPointService, mockEncryptionService) {
+                slaveMountPointService, mockEncryptionService, deviceActionFactory) {
             @Override
             protected NetconfTopologyContext newNetconfTopologyContext(final NetconfTopologySetup setup,
-                    final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime) {
-                NetconfTopologyContext spiedContext =
-                        spy(super.newNetconfTopologyContext(setup, serviceGroupIdent, actorResponseWaitTime));
+                final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime,
+                final DeviceActionFactory actionFactory) {
+                NetconfTopologyContext spiedContext = spy(super.newNetconfTopologyContext(setup, serviceGroupIdent,
+                    actorResponseWaitTime, actionFactory));
+
                 slaveNetconfTopologyContextFuture.set(spiedContext);
                 return spiedContext;
             }
@@ -651,6 +660,10 @@ public class MountPointEndToEndTest {
         return getMountPointService(mountPoint, DOMRpcService.class);
     }
 
+    private static DOMActionService getDomActionService(final DOMMountPoint mountPoint) {
+        return getMountPointService(mountPoint, DOMActionService.class);
+    }
+
     private static <T extends DOMService> T getMountPointService(final DOMMountPoint mountPoint,
             final Class<T> serviceClass) {
         final Optional<T> maybeService = mountPoint.getService(serviceClass);
index b1b887299cca010385c2a364abda29c0616931f5..6e6d3d6f5ec11288fa33e97c2200089eceb5cdee 100644 (file)
@@ -60,7 +60,12 @@ import org.junit.rules.ExpectedException;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mock;
 import org.opendaylight.controller.cluster.schema.provider.impl.YangTextSchemaSourceSerializationProxy;
+import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.DOMActionException;
+import org.opendaylight.mdsal.dom.api.DOMActionResult;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeReadWriteTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
@@ -71,9 +76,11 @@ import org.opendaylight.mdsal.dom.api.DOMRpcException;
 import org.opendaylight.mdsal.dom.api.DOMRpcResult;
 import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.mdsal.dom.spi.DefaultDOMRpcResult;
+import org.opendaylight.mdsal.dom.spi.SimpleDOMActionResult;
 import org.opendaylight.netconf.sal.connect.netconf.NetconfDevice.SchemaResourcesDTO;
 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
 import org.opendaylight.netconf.topology.singleton.impl.actors.NetconfNodeActor;
+import org.opendaylight.netconf.topology.singleton.impl.utils.ClusteringActionException;
 import org.opendaylight.netconf.topology.singleton.impl.utils.ClusteringRpcException;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup.NetconfTopologySetupBuilder;
@@ -90,6 +97,7 @@ import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.common.RpcError;
 import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import org.opendaylight.yangtools.yang.data.impl.schema.builder.impl.ImmutableContainerNodeBuilder;
@@ -131,6 +139,9 @@ public class NetconfNodeActorTest {
     @Mock
     private DOMRpcService mockDOMRpcService;
 
+    @Mock
+    private DOMActionService mockDOMActionService;
+
     @Mock
     private DOMMountPointService mockMountPointService;
 
@@ -300,6 +311,7 @@ public class NetconfNodeActorTest {
         verify(mockMountPointBuilder, after(500)).addInitialSchemaContext(mockSchemaContext);
         verify(mockMountPointBuilder).addService(eq(DOMDataBroker.class), any());
         verify(mockMountPointBuilder).addService(eq(DOMRpcService.class), any());
+        verify(mockMountPointBuilder).addService(eq(DOMActionService.class), any());
         verify(mockMountPointBuilder).addService(eq(DOMNotificationService.class), any());
         verify(mockSchemaSourceReg1).close();
         verify(mockRegistry, times(2)).registerSchemaSource(any(), withSourceId(SOURCE_IDENTIFIER1));
@@ -526,6 +538,60 @@ public class NetconfNodeActorTest {
         }
     }
 
+    @Test
+    @SuppressWarnings({"checkstyle:AvoidHidingCauseException", "checkstyle:IllegalThrows"})
+    public void testSlaveInvokeAction() throws Throwable {
+        final List<SourceIdentifier> sourceIdentifiers = Lists
+            .newArrayList(RevisionSourceIdentifier.create("testActionID"));
+        initializeMaster(sourceIdentifiers);
+        registerSlaveMountPoint();
+
+        ArgumentCaptor<DOMActionService> domActionServiceCaptor = ArgumentCaptor.forClass(DOMActionService.class);
+        verify(mockMountPointBuilder).addService(eq(DOMActionService.class), domActionServiceCaptor.capture());
+
+        final DOMActionService slaveDomActionService = domActionServiceCaptor.getValue();
+        assertTrue(slaveDomActionService instanceof ProxyDOMActionService);
+
+        final QName testQName = QName.create("test", "2019-08-16", "TestActionQname");
+        final SchemaPath schemaPath = SchemaPath.create(true, testQName);
+
+        final YangInstanceIdentifier yangIIdPath = YangInstanceIdentifier
+            .create(new YangInstanceIdentifier.NodeIdentifier(testQName));
+
+        final DOMDataTreeIdentifier domDataTreeIdentifier = new DOMDataTreeIdentifier(LogicalDatastoreType.OPERATIONAL,
+            yangIIdPath);
+
+        final ContainerNode outputNode = ImmutableContainerNodeBuilder.create()
+            .withNodeIdentifier(new YangInstanceIdentifier.NodeIdentifier(testQName))
+            .withChild(ImmutableNodes.leafNode(testQName, "foo")).build();
+
+        // Action with no response output.
+        doReturn(FluentFutures.immediateNullFluentFuture()).when(mockDOMActionService)
+            .invokeAction(any(), any(), any());
+        DOMActionResult result = slaveDomActionService.invokeAction(schemaPath, domDataTreeIdentifier, outputNode)
+            .get(2, TimeUnit.SECONDS);
+        assertEquals(null, result);
+
+        // Action with response output.
+        doReturn(FluentFutures.immediateFluentFuture(new SimpleDOMActionResult(outputNode))).when(mockDOMActionService)
+            .invokeAction(any(), any(), any());
+        result = slaveDomActionService.invokeAction(schemaPath, domDataTreeIdentifier, outputNode)
+            .get(2, TimeUnit.SECONDS);
+
+        assertEquals(outputNode, result.getOutput().get());
+        assertTrue(result.getErrors().isEmpty());
+
+        // Action failure.
+        exception.expect(DOMActionException.class);
+        doReturn(FluentFutures.immediateFailedFluentFuture(new ClusteringActionException("mock")))
+            .when(mockDOMActionService).invokeAction(any(), any(), any());
+        try {
+            slaveDomActionService.invokeAction(schemaPath, domDataTreeIdentifier, outputNode).get(2, TimeUnit.SECONDS);
+        } catch (ExecutionException e) {
+            throw e.getCause();
+        }
+    }
+
     @Test
     public void testSlaveNewTransactionRequests() {
 
@@ -579,7 +645,7 @@ public class NetconfNodeActorTest {
 
     private void initializeMaster(final List<SourceIdentifier> sourceIdentifiers) {
         masterRef.tell(new CreateInitialMasterActorData(mockDOMDataBroker, sourceIdentifiers,
-                mockDOMRpcService), testKit.getRef());
+                mockDOMRpcService, mockDOMActionService), testKit.getRef());
 
         testKit.expectMsgClass(MasterActorDataInitialized.class);
     }
index d0219758db455dcc91db983d1c4b2a2dd14d30b7..1a1fa38fb295a82c6f554bd8a5fc192a1724d6b3 100644 (file)
@@ -53,6 +53,7 @@ import org.opendaylight.mdsal.binding.api.DataBroker;
 import org.opendaylight.mdsal.binding.api.DataObjectModification;
 import org.opendaylight.mdsal.binding.api.DataTreeIdentifier;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
 import org.opendaylight.mdsal.dom.api.DOMMountPoint;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
@@ -120,6 +121,9 @@ public class NetconfNodeManagerTest {
     @Mock
     private DOMRpcService mockRpcService;
 
+    @Mock
+    private DOMActionService mockActionService;
+
     @Mock
     private NetconfDeviceSchemasResolver mockSchemasResolver;
 
@@ -382,7 +386,7 @@ public class NetconfNodeManagerTest {
         TestKit kit = new TestKit(masterSystem);
 
         testMasterActorRef.tell(new CreateInitialMasterActorData(mockDeviceDataBroker, SOURCE_IDENTIFIERS,
-                mockRpcService), kit.getRef());
+                mockRpcService, mockActionService), kit.getRef());
 
         kit.expectMsgClass(MasterActorDataInitialized.class);
     }
index f88314ec319d41d6292cc914d5eb5a716e384ecd..721795f8b1fbe65ef7f94c9fd4c253ac8436976b 100644 (file)
@@ -52,12 +52,14 @@ import org.opendaylight.mdsal.binding.api.ReadTransaction;
 import org.opendaylight.mdsal.binding.dom.adapter.test.AbstractDataBrokerTest;
 import org.opendaylight.mdsal.binding.spec.reflect.BindingReflections;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.DOMActionProviderService;
 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMRpcProviderService;
 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceProvider;
 import org.opendaylight.mdsal.singleton.common.api.ClusterSingletonServiceRegistration;
 import org.opendaylight.mdsal.singleton.common.api.ServiceGroupIdentifier;
 import org.opendaylight.netconf.client.NetconfClientDispatcher;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologySetup;
 import org.opendaylight.netconf.topology.singleton.impl.utils.NetconfTopologyUtils;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.Host;
@@ -112,6 +114,7 @@ public class NetconfTopologyManagerTest {
 
         final DOMRpcProviderService rpcProviderRegistry = mock(DOMRpcProviderService.class);
         final ScheduledThreadPool keepaliveExecutor = mock(ScheduledThreadPool.class);
+        final DOMActionProviderService actionProviderRegistry = mock(DOMActionProviderService.class);
         final ThreadPool processingThreadPool = mock(ThreadPool.class);
         final ExecutorService processingService = mock(ExecutorService.class);
         doReturn(processingService).when(processingThreadPool).getExecutor();
@@ -120,15 +123,17 @@ public class NetconfTopologyManagerTest {
         final NetconfClientDispatcher clientDispatcher = mock(NetconfClientDispatcher.class);
         final DOMMountPointService mountPointService = mock(DOMMountPointService.class);
         final AAAEncryptionService encryptionService = mock(AAAEncryptionService.class);
+        final DeviceActionFactory deviceActionFactory = mock(DeviceActionFactory.class);
 
         final Config config = new ConfigBuilder().setWriteTransactionIdleTimeout(0).build();
-        netconfTopologyManager = new NetconfTopologyManager(dataBroker, rpcProviderRegistry,
+        netconfTopologyManager = new NetconfTopologyManager(dataBroker, rpcProviderRegistry, actionProviderRegistry,
                 clusterSingletonServiceProvider, keepaliveExecutor, processingThreadPool,
                 actorSystemProvider, eventExecutor, clientDispatcher, TOPOLOGY_ID, config,
-                mountPointService, encryptionService) {
+                mountPointService, encryptionService, deviceActionFactory) {
             @Override
             protected NetconfTopologyContext newNetconfTopologyContext(final NetconfTopologySetup setup,
-                    final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime) {
+                final ServiceGroupIdentifier serviceGroupIdent, final Timeout actorResponseWaitTime,
+                final DeviceActionFactory deviceActionFactory) {
                 assertEquals(ACTOR_RESPONSE_WAIT_TIME, actorResponseWaitTime.duration().toSeconds());
                 return Objects.requireNonNull(mockContextMap.get(setup.getInstanceIdentifier()),
                         "No mock context for " + setup.getInstanceIdentifier()).apply(setup);
index cdb25907caec89c7120f143ca5e50a191daeacbf..5ecfe0cb60a756adb610b7ad5038ffe276788b0c 100644 (file)
@@ -42,6 +42,7 @@ import org.opendaylight.netconf.client.NetconfClientDispatcher;
 import org.opendaylight.netconf.client.NetconfClientSessionListener;
 import org.opendaylight.netconf.client.conf.NetconfClientConfiguration;
 import org.opendaylight.netconf.client.conf.NetconfReconnectingClientConfiguration;
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
@@ -102,6 +103,9 @@ public class RemoteDeviceConnectorImplTest {
     @Mock
     private WriteTransaction writeTx;
 
+    @Mock
+    private DeviceActionFactory deviceActionFactory;
+
     private NetconfTopologySetup.NetconfTopologySetupBuilder builder;
     private RemoteDeviceId remoteDeviceId;
 
@@ -152,8 +156,8 @@ public class RemoteDeviceConnectorImplTest {
         final NetconfDeviceCommunicator communicator = mock(NetconfDeviceCommunicator.class);
         final RemoteDeviceHandler<NetconfSessionPreferences> salFacade = mock(RemoteDeviceHandler.class);
 
-        final TestingRemoteDeviceConnectorImpl remoteDeviceConnection =
-                new TestingRemoteDeviceConnectorImpl(builder.build(), remoteDeviceId, communicator);
+        final TestingRemoteDeviceConnectorImpl remoteDeviceConnection = new TestingRemoteDeviceConnectorImpl(
+            builder.build(), remoteDeviceId, communicator, deviceActionFactory);
 
         remoteDeviceConnection.startRemoteDeviceConnection(salFacade);
 
@@ -185,7 +189,7 @@ public class RemoteDeviceConnectorImplTest {
         builder.setSchemaResourceDTO(NetconfTopologyUtils.setupSchemaCacheDTO(node));
 
         final RemoteDeviceConnectorImpl remoteDeviceConnection =
-                new RemoteDeviceConnectorImpl(builder.build(), remoteDeviceId);
+                new RemoteDeviceConnectorImpl(builder.build(), remoteDeviceId, deviceActionFactory);
 
         final RemoteDeviceHandler<NetconfSessionPreferences> salFacade = mock(RemoteDeviceHandler.class);
 
@@ -212,7 +216,7 @@ public class RemoteDeviceConnectorImplTest {
                 .build();
 
         final RemoteDeviceConnectorImpl remoteDeviceConnection =
-                new RemoteDeviceConnectorImpl(builder.build(), remoteDeviceId);
+                new RemoteDeviceConnectorImpl(builder.build(), remoteDeviceId, deviceActionFactory);
 
         final NetconfReconnectingClientConfiguration defaultClientConfig =
                 remoteDeviceConnection.getClientConfig(listener, testingNode);
index db23123968d02453bb7b4ed7c4fdf0271c64326c..8b630829ec9b547b944b0f356e3302891d23c3b2 100644 (file)
@@ -11,6 +11,7 @@ package org.opendaylight.netconf.topology.singleton.impl;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doReturn;
 
+import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
@@ -27,8 +28,10 @@ class TestingRemoteDeviceConnectorImpl extends RemoteDeviceConnectorImpl {
 
     TestingRemoteDeviceConnectorImpl(final NetconfTopologySetup netconfTopologyDeviceSetup,
                                      final RemoteDeviceId remoteDeviceId,
-                                     final NetconfDeviceCommunicator communicator) {
-        super(netconfTopologyDeviceSetup, remoteDeviceId);
+                                     final NetconfDeviceCommunicator communicator,
+                                     final DeviceActionFactory deviceActionFactory) {
+        super(netconfTopologyDeviceSetup, remoteDeviceId, deviceActionFactory);
+
         this.communicator = communicator;
     }
 
@@ -39,5 +42,4 @@ class TestingRemoteDeviceConnectorImpl extends RemoteDeviceConnectorImpl {
         doReturn(FluentFutures.immediateNullFluentFuture()).when(communicator).initializeRemoteConnection(any(), any());
         return connectorDTO;
     }
-
 }
index e559b6b68a2dffc03db4a524d28466874781294b..691707aa8495f9fbdaaece60543ac8cdc478a185 100644 (file)
@@ -381,10 +381,8 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
                     .setSalFacade(salFacade)
                     .setNode(node)
                     .setEventExecutor(eventExecutor)
-                    .setNodeOptional(nodeOptional);
-            if (this.deviceActionFactory != null) {
-                netconfDeviceBuilder.setDeviceActionFactory(this.deviceActionFactory);
-            }
+                    .setNodeOptional(nodeOptional)
+                    .setDeviceActionFactory(deviceActionFactory);
             device = netconfDeviceBuilder.build();
         }