From f0525c56b97cd8c69b25059070a19e3e7305db6b Mon Sep 17 00:00:00 2001 From: ajay_dp001 Date: Wed, 11 Sep 2019 00:29:09 +0100 Subject: [PATCH] Teach NETCONF about YANG 1.1 actions in cluster topology 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: Id7f5c9ef4a515c28d5da8bab29724bb353c10d13 Signed-off-by: ajay_dp001 --- .../singleton/impl/MasterSalFacade.java | 16 +-- .../impl/NetconfTopologyContext.java | 14 +- .../impl/NetconfTopologyManager.java | 19 ++- .../singleton/impl/ProxyDOMActionService.java | 127 ++++++++++++++++++ .../impl/RemoteDeviceConnectorImpl.java | 7 +- .../singleton/impl/SlaveSalFacade.java | 5 +- .../impl/actors/NetconfNodeActor.java | 59 +++++++- .../impl/utils/ClusteringActionException.java | 36 +++++ .../impl/utils/NetconfTopologySetup.java | 18 +++ .../messages/ContainerNodeMessage.java | 59 ++++++++ .../CreateInitialMasterActorData.java | 9 +- .../singleton/messages/SchemaPathMessage.java | 8 +- .../messages/action/InvokeActionMessage.java | 106 +++++++++++++++ .../action/InvokeActionMessageReply.java | 99 ++++++++++++++ .../blueprint/netconf-topology-singleton.xml | 6 + .../impl/MountPointEndToEndTest.java | 31 +++-- .../singleton/impl/NetconfNodeActorTest.java | 68 +++++++++- .../impl/NetconfNodeManagerTest.java | 6 +- .../impl/NetconfTopologyManagerTest.java | 11 +- .../impl/RemoteDeviceConnectorImplTest.java | 11 +- .../TestingRemoteDeviceConnectorImpl.java | 6 +- .../topology/AbstractNetconfTopology.java | 7 +- 22 files changed, 678 insertions(+), 50 deletions(-) create mode 100644 netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/ProxyDOMActionService.java create mode 100644 netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/ClusteringActionException.java create mode 100644 netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/ContainerNodeMessage.java create mode 100644 netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessage.java create mode 100644 netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessageReply.java diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java index 1b7068a442..e67bdc14d5 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/MasterSalFacade.java @@ -52,6 +52,7 @@ class MasterSalFacade implements AutoCloseable, RemoteDeviceHandler future = Patterns.ask(masterActorRef, new RefreshSetupMasterActorData( diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManager.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManager.java index 397b508701..f403540684 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManager.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManager.java @@ -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 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 index 0000000000..56a2c54a35 --- /dev/null +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/ProxyDOMActionService.java @@ -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 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 scalaFuture = Patterns.ask(masterActorRef, new InvokeActionMessage( + new SchemaPathMessage(type), containerNodeMessage, domDataTreeIdentifier), actorResponseWaitTime); + + final SettableFuture settableFuture = SettableFuture.create(); + + scalaFuture.onComplete(new OnComplete() { + @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 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 getExtensions() { + return ImmutableClassToInstanceMap.of(); + } +} diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImpl.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImpl.java index 8d32fb3014..955f465ee7 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImpl.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImpl.java @@ -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,15 @@ 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 +204,7 @@ public class RemoteDeviceConnectorImpl implements RemoteDeviceConnector { .setSchemaResourcesDTO(schemaResourcesDTO) .setGlobalProcessingExecutor(netconfTopologyDeviceSetup.getProcessingExecutor()) .setId(remoteDeviceId) + .setDeviceActionFactory(deviceActionFactory) .setSalFacade(salFacade) .build(); } diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/SlaveSalFacade.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/SlaveSalFacade.java index 4fbed61aed..118e8e0735 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/SlaveSalFacade.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/SlaveSalFacade.java @@ -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,7 +42,7 @@ 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; } @@ -52,7 +53,7 @@ public class SlaveSalFacade { new ProxyDOMDataBroker(id, masterActorRef, actorSystem.dispatcher(), actorResponseWaitTime); salProvider.getMountInstance().onTopologyDeviceConnected(remoteSchemaContext, netconfDeviceDataBroker, - deviceRpc, notificationService); + deviceRpc, notificationService, deviceAction); LOG.info("{}: Slave mount point registered.", id); } diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/actors/NetconfNodeActor.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/actors/NetconfNodeActor.java index 7f8ff7d836..c437220ce9 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/actors/NetconfNodeActor.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/actors/NetconfNodeActor.java @@ -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 sourceIdentifiers; private DOMRpcService deviceRpc; + private DOMActionService deviceAction; private SlaveSalFacade slaveSalManager; private DOMDataBroker deviceDataBroker; //readTxActor can be shared @@ -118,6 +126,7 @@ 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()); @@ -168,6 +177,11 @@ public class NetconfNodeActor extends AbstractUntypedActor { 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; sourceIdentifiers = registerMountPoint.getSourceIndentifiers(); @@ -259,6 +273,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 actionResult = deviceAction.invokeAction(schemaPath, + domDataTreeIdentifier, containerNodeMessage != null ? containerNodeMessage.getNode() : null); + + Futures.addCallback(actionResult, new FutureCallback() { + + @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(); @@ -271,6 +324,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); @@ -300,7 +357,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); } }); } 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 index 0000000000..6d348a39e5 --- /dev/null +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/ClusteringActionException.java @@ -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 diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/NetconfTopologySetup.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/NetconfTopologySetup.java index 7963d440f0..f6520665b4 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/NetconfTopologySetup.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/impl/utils/NetconfTopologySetup.java @@ -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 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 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; } 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 index 0000000000..5091c59785 --- /dev/null +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/ContainerNodeMessage.java @@ -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 + "]"; + } +} diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/CreateInitialMasterActorData.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/CreateInitialMasterActorData.java index 509ecb0e55..a47a2c6d42 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/CreateInitialMasterActorData.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/CreateInitialMasterActorData.java @@ -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 allSourceIdentifiers; private final DOMRpcService deviceRpc; + private final DOMActionService deviceAction; public CreateInitialMasterActorData(final DOMDataBroker deviceDataBroker, final List 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; + } } diff --git a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/SchemaPathMessage.java b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/SchemaPathMessage.java index 2e7e0619b9..5f170036c5 100644 --- a/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/SchemaPathMessage.java +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/SchemaPathMessage.java @@ -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 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 index 0000000000..58a44044fa --- /dev/null +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessage.java @@ -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 index 0000000000..07ddfe9dee --- /dev/null +++ b/netconf/netconf-topology-singleton/src/main/java/org/opendaylight/netconf/topology/singleton/messages/action/InvokeActionMessageReply.java @@ -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 rpcErrors; + private final ContainerNodeMessage containerNodeMessage; + + /** + * Constructor for {@code InvokeActionMessage}. + * + * @param containerNodeMessage ContainerNodeMessage + * @param rpcErrors RpcError + */ + public InvokeActionMessageReply(final @Nullable ContainerNodeMessage containerNodeMessage, + final @NonNull Collection rpcErrors) { + this.containerNodeMessage = requireNonNull(containerNodeMessage); + this.rpcErrors = requireNonNull(rpcErrors); + } + + public @Nullable ContainerNodeMessage getContainerNodeMessage() { + return containerNodeMessage; + } + + public @NonNull Collection 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 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; + } + } +} diff --git a/netconf/netconf-topology-singleton/src/main/resources/OSGI-INF/blueprint/netconf-topology-singleton.xml b/netconf/netconf-topology-singleton/src/main/resources/OSGI-INF/blueprint/netconf-topology-singleton.xml index 4138656a67..91ef4f842b 100644 --- a/netconf/netconf-topology-singleton/src/main/resources/OSGI-INF/blueprint/netconf-topology-singleton.xml +++ b/netconf/netconf-topology-singleton/src/main/resources/OSGI-INF/blueprint/netconf-topology-singleton.xml @@ -16,6 +16,8 @@ and is available at http://www.eclipse.org/legal/epl-v10.html interface="org.opendaylight.mdsal.binding.api.DataBroker"/> + + + @@ -69,6 +74,7 @@ and is available at http://www.eclipse.org/legal/epl-v10.html + diff --git a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/MountPointEndToEndTest.java b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/MountPointEndToEndTest.java index 2713850ddf..c7df728406 100644 --- a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/MountPointEndToEndTest.java +++ b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/MountPointEndToEndTest.java @@ -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,15 @@ 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 +308,15 @@ public class MountPointEndToEndTest { .registerClusterSingletonService(any()); slaveNetconfTopologyManager = new NetconfTopologyManager(slaveDataBroker, mockRpcProviderRegistry, - mockSlaveClusterSingletonServiceProvider, mockKeepaliveExecutor, mockThreadPool, - mockSlaveActorSystemProvider, eventExecutor, mockClientDispatcher, TOPOLOGY_ID, config, - slaveMountPointService, mockEncryptionService) { + mockActionProviderRegistry, mockSlaveClusterSingletonServiceProvider, mockKeepaliveExecutor, + mockThreadPool, mockSlaveActorSystemProvider, eventExecutor, mockClientDispatcher, TOPOLOGY_ID, config, + 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 deviceActionFact) { + NetconfTopologyContext spiedContext = spy(super.newNetconfTopologyContext(setup, serviceGroupIdent, + actorResponseWaitTime, deviceActionFact)); slaveNetconfTopologyContextFuture.set(spiedContext); return spiedContext; } @@ -651,6 +658,10 @@ public class MountPointEndToEndTest { return getMountPointService(mountPoint, DOMRpcService.class); } + private static DOMActionService getDomActionService(final DOMMountPoint mountPoint) { + return getMountPointService(mountPoint, DOMActionService.class); + } + private static T getMountPointService(final DOMMountPoint mountPoint, final Class serviceClass) { final Optional maybeService = mountPoint.getService(serviceClass); diff --git a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeActorTest.java b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeActorTest.java index 145140011e..d879a62c4d 100644 --- a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeActorTest.java +++ b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeActorTest.java @@ -59,7 +59,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; @@ -70,8 +75,10 @@ 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.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; @@ -88,6 +95,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; @@ -129,6 +137,9 @@ public class NetconfNodeActorTest { @Mock private DOMRpcService mockDOMRpcService; + @Mock + private DOMActionService mockDOMActionService; + @Mock private DOMMountPointService mockMountPointService; @@ -293,6 +304,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)); @@ -495,6 +507,60 @@ public class NetconfNodeActorTest { } } + @Test + @SuppressWarnings({"checkstyle:AvoidHidingCauseException", "checkstyle:IllegalThrows"}) + public void testSlaveInvokeAction() throws Throwable { + final List sourceIdentifiers = Lists + .newArrayList(RevisionSourceIdentifier.create("testActionID")); + initializeMaster(sourceIdentifiers); + registerSlaveMountPoint(); + + ArgumentCaptor 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() { @@ -545,7 +611,7 @@ public class NetconfNodeActorTest { private void initializeMaster(final List sourceIdentifiers) { masterRef.tell(new CreateInitialMasterActorData(mockDOMDataBroker, sourceIdentifiers, - mockDOMRpcService), testKit.getRef()); + mockDOMRpcService, mockDOMActionService), testKit.getRef()); testKit.expectMsgClass(MasterActorDataInitialized.class); } diff --git a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeManagerTest.java b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeManagerTest.java index d7bd0de49e..e270ea6f8d 100644 --- a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeManagerTest.java +++ b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfNodeManagerTest.java @@ -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); } diff --git a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManagerTest.java b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManagerTest.java index f88314ec31..737ec7c963 100644 --- a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManagerTest.java +++ b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/NetconfTopologyManagerTest.java @@ -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; @@ -120,15 +122,18 @@ 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 DOMActionProviderService actionProviderRegistry = mock(DOMActionProviderService.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); diff --git a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImplTest.java b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImplTest.java index cdb25907ca..37ff7c18b3 100644 --- a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImplTest.java +++ b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/RemoteDeviceConnectorImplTest.java @@ -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; @@ -153,7 +157,8 @@ public class RemoteDeviceConnectorImplTest { final RemoteDeviceHandler salFacade = mock(RemoteDeviceHandler.class); final TestingRemoteDeviceConnectorImpl remoteDeviceConnection = - new TestingRemoteDeviceConnectorImpl(builder.build(), remoteDeviceId, communicator); + new TestingRemoteDeviceConnectorImpl(builder.build(), remoteDeviceId, communicator, + deviceActionFactory); remoteDeviceConnection.startRemoteDeviceConnection(salFacade); @@ -185,7 +190,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 salFacade = mock(RemoteDeviceHandler.class); @@ -212,7 +217,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); diff --git a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/TestingRemoteDeviceConnectorImpl.java b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/TestingRemoteDeviceConnectorImpl.java index db23123968..4ea9940bf6 100644 --- a/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/TestingRemoteDeviceConnectorImpl.java +++ b/netconf/netconf-topology-singleton/src/test/java/org/opendaylight/netconf/topology/singleton/impl/TestingRemoteDeviceConnectorImpl.java @@ -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,9 @@ 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; } diff --git a/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/AbstractNetconfTopology.java b/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/AbstractNetconfTopology.java index e559b6b68a..5c73e643a0 100644 --- a/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/AbstractNetconfTopology.java +++ b/netconf/netconf-topology/src/main/java/org/opendaylight/netconf/topology/AbstractNetconfTopology.java @@ -381,10 +381,9 @@ 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(); } -- 2.36.6