Remove unused method createDeviceCommunicator
[netconf.git] / apps / netconf-topology / src / main / java / org / opendaylight / netconf / topology / spi / AbstractNetconfTopology.java
1 /*
2  * Copyright (c) 2015 Cisco Systems, Inc. and others. All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.netconf.topology.spi;
9
10 import static java.util.Objects.requireNonNull;
11
12 import com.google.common.annotations.VisibleForTesting;
13 import com.google.common.collect.Lists;
14 import com.google.common.util.concurrent.FutureCallback;
15 import com.google.common.util.concurrent.Futures;
16 import com.google.common.util.concurrent.ListenableFuture;
17 import com.google.common.util.concurrent.ListeningExecutorService;
18 import com.google.common.util.concurrent.MoreExecutors;
19 import io.netty.util.concurrent.EventExecutor;
20 import java.net.URL;
21 import java.util.ArrayList;
22 import java.util.HashMap;
23 import java.util.List;
24 import java.util.Map;
25 import java.util.concurrent.ExecutionException;
26 import org.opendaylight.aaa.encrypt.AAAEncryptionService;
27 import org.opendaylight.controller.config.threadpool.ScheduledThreadPool;
28 import org.opendaylight.controller.config.threadpool.ThreadPool;
29 import org.opendaylight.mdsal.binding.api.DataBroker;
30 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
31 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
32 import org.opendaylight.netconf.client.NetconfClientDispatcher;
33 import org.opendaylight.netconf.client.NetconfClientSessionListener;
34 import org.opendaylight.netconf.client.conf.NetconfClientConfiguration;
35 import org.opendaylight.netconf.client.conf.NetconfReconnectingClientConfiguration;
36 import org.opendaylight.netconf.client.conf.NetconfReconnectingClientConfigurationBuilder;
37 import org.opendaylight.netconf.nettyutil.ReconnectStrategyFactory;
38 import org.opendaylight.netconf.nettyutil.TimedReconnectStrategyFactory;
39 import org.opendaylight.netconf.nettyutil.handler.ssh.authentication.AuthenticationHandler;
40 import org.opendaylight.netconf.nettyutil.handler.ssh.authentication.LoginPasswordHandler;
41 import org.opendaylight.netconf.sal.connect.api.DeviceActionFactory;
42 import org.opendaylight.netconf.sal.connect.api.RemoteDevice;
43 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
44 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceId;
45 import org.opendaylight.netconf.sal.connect.api.SchemaResourceManager;
46 import org.opendaylight.netconf.sal.connect.netconf.LibraryModulesSchemas;
47 import org.opendaylight.netconf.sal.connect.netconf.NetconfDevice.SchemaResourcesDTO;
48 import org.opendaylight.netconf.sal.connect.netconf.NetconfDeviceBuilder;
49 import org.opendaylight.netconf.sal.connect.netconf.SchemalessNetconfDevice;
50 import org.opendaylight.netconf.sal.connect.netconf.auth.DatastoreBackedPublicKeyAuth;
51 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
52 import org.opendaylight.netconf.sal.connect.netconf.sal.KeepaliveSalFacade;
53 import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfKeystoreAdapter;
54 import org.opendaylight.netconf.sal.connect.netconf.schema.YangLibrarySchemaYangSourceProvider;
55 import org.opendaylight.netconf.sal.connect.netconf.schema.mapping.BaseNetconfSchemas;
56 import org.opendaylight.netconf.sal.connect.util.SslHandlerFactoryImpl;
57 import org.opendaylight.netconf.topology.api.NetconfTopology;
58 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.Uri;
59 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.connection.parameters.Protocol.Name;
60 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.Credentials;
61 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.credentials.KeyAuth;
62 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.credentials.LoginPw;
63 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225.credentials.credentials.LoginPwUnencrypted;
64 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev221225.NetconfNodeAugmentedOptional;
65 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev221225.NetconfNode;
66 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NetworkTopology;
67 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
68 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.TopologyId;
69 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.Topology;
70 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.TopologyBuilder;
71 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.TopologyKey;
72 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
73 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
74 import org.opendaylight.yangtools.yang.common.Empty;
75 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
76 import org.opendaylight.yangtools.yang.model.repo.api.YangTextSchemaSource;
77 import org.opendaylight.yangtools.yang.model.repo.spi.PotentialSchemaSource;
78 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistration;
79 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistry;
80 import org.slf4j.Logger;
81 import org.slf4j.LoggerFactory;
82
83 public abstract class AbstractNetconfTopology implements NetconfTopology {
84     private static final Logger LOG = LoggerFactory.getLogger(AbstractNetconfTopology.class);
85
86     private final NetconfClientDispatcher clientDispatcher;
87     private final EventExecutor eventExecutor;
88     private final DeviceActionFactory deviceActionFactory;
89     private final NetconfKeystoreAdapter keystoreAdapter;
90     private final SchemaResourceManager schemaManager;
91     private final BaseNetconfSchemas baseSchemas;
92
93     protected final ScheduledThreadPool keepaliveExecutor;
94     protected final ListeningExecutorService processingExecutor;
95     protected final DataBroker dataBroker;
96     protected final DOMMountPointService mountPointService;
97     protected final String topologyId;
98     protected String privateKeyPath;
99     protected String privateKeyPassphrase;
100     protected final AAAEncryptionService encryptionService;
101     protected final HashMap<NodeId, NetconfConnectorDTO> activeConnectors = new HashMap<>();
102
103     protected AbstractNetconfTopology(final String topologyId, final NetconfClientDispatcher clientDispatcher,
104                                       final EventExecutor eventExecutor, final ScheduledThreadPool keepaliveExecutor,
105                                       final ThreadPool processingExecutor, final SchemaResourceManager schemaManager,
106                                       final DataBroker dataBroker, final DOMMountPointService mountPointService,
107                                       final AAAEncryptionService encryptionService,
108                                       final DeviceActionFactory deviceActionFactory,
109                                       final BaseNetconfSchemas baseSchemas) {
110         this.topologyId = requireNonNull(topologyId);
111         this.clientDispatcher = clientDispatcher;
112         this.eventExecutor = eventExecutor;
113         this.keepaliveExecutor = keepaliveExecutor;
114         this.processingExecutor = MoreExecutors.listeningDecorator(processingExecutor.getExecutor());
115         this.schemaManager = requireNonNull(schemaManager);
116         this.deviceActionFactory = deviceActionFactory;
117         this.dataBroker = requireNonNull(dataBroker);
118         this.mountPointService = mountPointService;
119         this.encryptionService = encryptionService;
120         this.baseSchemas = requireNonNull(baseSchemas);
121
122         keystoreAdapter = new NetconfKeystoreAdapter(dataBroker);
123
124         // FIXME: this should be a put(), as we are initializing and will be re-populating the datastore with all the
125         //        devices. Whatever has been there before should be nuked to properly re-align lifecycle.
126         final var wtx = dataBroker.newWriteOnlyTransaction();
127         wtx.merge(LogicalDatastoreType.OPERATIONAL, InstanceIdentifier.builder(NetworkTopology.class)
128             .child(Topology.class, new TopologyKey(new TopologyId(topologyId)))
129             .build(), new TopologyBuilder().setTopologyId(new TopologyId(topologyId)).build());
130         final var future = wtx.commit();
131         try {
132             future.get();
133         } catch (InterruptedException | ExecutionException e) {
134             LOG.error("Unable to initialize topology {}", topologyId, e);
135             throw new IllegalStateException(e);
136         }
137
138         LOG.debug("Topology {} initialized", topologyId);
139     }
140
141     @Override
142     public ListenableFuture<Empty> connectNode(final NodeId nodeId, final Node configNode) {
143         LOG.info("Connecting RemoteDevice{{}} , with config {}", nodeId, hideCredentials(configNode));
144         return setupConnection(nodeId, configNode);
145     }
146
147     /**
148      * Hiding of private credentials from node configuration (credentials data is replaced by asterisks).
149      *
150      * @param nodeConfiguration Node configuration container.
151      * @return String representation of node configuration with credentials replaced by asterisks.
152      */
153     @VisibleForTesting
154     public static String hideCredentials(final Node nodeConfiguration) {
155         final NetconfNode netconfNodeAugmentation = nodeConfiguration.augmentation(NetconfNode.class);
156         final String nodeCredentials = netconfNodeAugmentation.getCredentials().toString();
157         final String nodeConfigurationString = nodeConfiguration.toString();
158         return nodeConfigurationString.replace(nodeCredentials, "***");
159     }
160
161     @Override
162     public ListenableFuture<Empty> disconnectNode(final NodeId nodeId) {
163         final var nodeName = nodeId.getValue();
164         LOG.debug("Disconnecting RemoteDevice{{}}", nodeName);
165
166         final NetconfConnectorDTO connectorDTO = activeConnectors.remove(nodeId);
167         if (connectorDTO == null) {
168             return Futures.immediateFailedFuture(
169                 new IllegalStateException("Cannot disconnect " + nodeName + " as it is not connected"));
170         }
171
172         connectorDTO.close();
173         return Futures.immediateFuture(Empty.value());
174     }
175
176     protected ListenableFuture<Empty> setupConnection(final NodeId nodeId, final Node configNode) {
177         final NetconfNode netconfNode = configNode.augmentation(NetconfNode.class);
178         final NetconfNodeAugmentedOptional nodeOptional = configNode.augmentation(NetconfNodeAugmentedOptional.class);
179
180         requireNonNull(netconfNode.getHost());
181         requireNonNull(netconfNode.getPort());
182
183         final NetconfConnectorDTO deviceCommunicatorDTO = createDeviceCommunicator(nodeId, netconfNode, nodeOptional);
184         final NetconfDeviceCommunicator deviceCommunicator = deviceCommunicatorDTO.getCommunicator();
185         final NetconfClientSessionListener netconfClientSessionListener = deviceCommunicatorDTO.getSessionListener();
186         final NetconfReconnectingClientConfiguration clientConfig =
187                 getClientConfig(netconfClientSessionListener, netconfNode, nodeId);
188         final ListenableFuture<Empty> future =
189                 deviceCommunicator.initializeRemoteConnection(clientDispatcher, clientConfig);
190
191         activeConnectors.put(nodeId, deviceCommunicatorDTO);
192
193         Futures.addCallback(future, new FutureCallback<>() {
194             @Override
195             public void onSuccess(final Empty result) {
196                 LOG.debug("Connector for {} started succesfully", nodeId.getValue());
197             }
198
199             @Override
200             public void onFailure(final Throwable throwable) {
201                 LOG.error("Connector for {} failed", nodeId.getValue(), throwable);
202                 // remove this node from active connectors?
203             }
204         }, MoreExecutors.directExecutor());
205
206         return future;
207     }
208
209     protected NetconfConnectorDTO createDeviceCommunicator(final NodeId nodeId, final NetconfNode node,
210             final NetconfNodeAugmentedOptional nodeOptional) {
211         final var deviceId = NetconfNodeUtils.toRemoteDeviceId(nodeId, node);
212         final long keepaliveDelay = node.requireKeepaliveDelay().toJava();
213
214         final var deviceSalFacade = new NetconfTopologyDeviceSalFacade(deviceId, mountPointService,
215             node.requireLockDatastore(), dataBroker);
216         // The facade we are going it present to NetconfDevice
217         RemoteDeviceHandler salFacade;
218         final KeepaliveSalFacade keepAliveFacade;
219         if (keepaliveDelay > 0) {
220             LOG.info("Adding keepalive facade, for device {}", nodeId);
221             salFacade = keepAliveFacade = new KeepaliveSalFacade(deviceId, deviceSalFacade,
222                 keepaliveExecutor.getExecutor(), keepaliveDelay, node.requireDefaultRequestTimeoutMillis().toJava());
223         } else {
224             salFacade = deviceSalFacade;
225             keepAliveFacade = null;
226         }
227
228         // Setup reconnection on empty context, if so configured
229         if (nodeOptional != null && nodeOptional.getIgnoreMissingSchemaSources().getAllowed()) {
230             LOG.warn("Ignoring missing schema sources is not currently implemented for {}", deviceId);
231         }
232
233         final RemoteDevice<NetconfDeviceCommunicator> device;
234         final List<SchemaSourceRegistration<?>> yanglibRegistrations;
235         if (node.requireSchemaless()) {
236             device = new SchemalessNetconfDevice(baseSchemas, deviceId, salFacade);
237             yanglibRegistrations = List.of();
238         } else {
239             final SchemaResourcesDTO resources = schemaManager.getSchemaResources(node.getSchemaCacheDirectory(),
240                 nodeId.getValue());
241             device = new NetconfDeviceBuilder()
242                 .setReconnectOnSchemasChange(node.requireReconnectOnChangedSchema())
243                 .setSchemaResourcesDTO(resources)
244                 .setGlobalProcessingExecutor(processingExecutor)
245                 .setId(deviceId)
246                 .setSalFacade(salFacade)
247                 .setDeviceActionFactory(deviceActionFactory)
248                 .setBaseSchemas(baseSchemas)
249                 .build();
250             yanglibRegistrations = registerDeviceSchemaSources(deviceId, node, resources);
251         }
252
253         final int rpcMessageLimit = node.requireConcurrentRpcLimit().toJava();
254         if (rpcMessageLimit < 1) {
255             LOG.info("Concurrent rpc limit is smaller than 1, no limit will be enforced for device {}", deviceId);
256         }
257
258         final var netconfDeviceCommunicator = new NetconfDeviceCommunicator(deviceId, device, rpcMessageLimit,
259             NetconfNodeUtils.extractUserCapabilities(node));
260
261         if (keepAliveFacade != null) {
262             keepAliveFacade.setListener(netconfDeviceCommunicator);
263         }
264
265         return new NetconfConnectorDTO(netconfDeviceCommunicator, salFacade, yanglibRegistrations);
266     }
267
268     private static List<SchemaSourceRegistration<?>> registerDeviceSchemaSources(final RemoteDeviceId remoteDeviceId,
269             final NetconfNode node, final SchemaResourcesDTO resources) {
270         final var yangLibrary = node.getYangLibrary();
271         if (yangLibrary != null) {
272             final Uri uri = yangLibrary.getYangLibraryUrl();
273             if (uri != null) {
274                 final List<SchemaSourceRegistration<?>> registrations = new ArrayList<>();
275                 final String yangLibURL = uri.getValue();
276                 final SchemaSourceRegistry schemaRegistry = resources.getSchemaRegistry();
277
278                 // pre register yang library sources as fallback schemas to schema registry
279                 final LibraryModulesSchemas schemas;
280                 final String yangLibUsername = yangLibrary.getUsername();
281                 final String yangLigPassword = yangLibrary.getPassword();
282                 if (yangLibUsername != null && yangLigPassword != null) {
283                     schemas = LibraryModulesSchemas.create(yangLibURL, yangLibUsername, yangLigPassword);
284                 } else {
285                     schemas = LibraryModulesSchemas.create(yangLibURL);
286                 }
287
288                 for (final Map.Entry<SourceIdentifier, URL> entry : schemas.getAvailableModels().entrySet()) {
289                     registrations.add(schemaRegistry.registerSchemaSource(new YangLibrarySchemaYangSourceProvider(
290                         remoteDeviceId, schemas.getAvailableModels()),
291                         PotentialSchemaSource.create(entry.getKey(), YangTextSchemaSource.class,
292                             PotentialSchemaSource.Costs.REMOTE_IO.getValue())));
293                 }
294                 return List.copyOf(registrations);
295             }
296         }
297
298         return List.of();
299     }
300
301     /**
302      * Sets the private key path from location specified in configuration file using blueprint.
303      */
304     public void setPrivateKeyPath(final String privateKeyPath) {
305         this.privateKeyPath = privateKeyPath;
306     }
307
308     /**
309      * Sets the private key passphrase from location specified in configuration file using blueprint.
310      */
311     public void setPrivateKeyPassphrase(final String privateKeyPassphrase) {
312         this.privateKeyPassphrase = privateKeyPassphrase;
313     }
314
315     public NetconfReconnectingClientConfiguration getClientConfig(final NetconfClientSessionListener listener,
316                                                                   final NetconfNode node, final NodeId nodeId) {
317         final ReconnectStrategyFactory sf = new TimedReconnectStrategyFactory(eventExecutor,
318                 node.requireMaxConnectionAttempts().toJava(), node.requireBetweenAttemptsTimeoutMillis().toJava(),
319                 node.requireSleepFactor().decimalValue());
320         final NetconfReconnectingClientConfigurationBuilder reconnectingClientConfigurationBuilder;
321         final var protocol = node.getProtocol();
322         if (node.requireTcpOnly()) {
323             reconnectingClientConfigurationBuilder = NetconfReconnectingClientConfigurationBuilder.create()
324                     .withProtocol(NetconfClientConfiguration.NetconfClientProtocol.TCP)
325                     .withAuthHandler(getHandlerFromCredentials(node.getCredentials()));
326         } else if (protocol == null || protocol.getName() == Name.SSH) {
327             reconnectingClientConfigurationBuilder = NetconfReconnectingClientConfigurationBuilder.create()
328                     .withProtocol(NetconfClientConfiguration.NetconfClientProtocol.SSH)
329                     .withAuthHandler(getHandlerFromCredentials(node.getCredentials()));
330         } else if (protocol.getName() == Name.TLS) {
331             reconnectingClientConfigurationBuilder = NetconfReconnectingClientConfigurationBuilder.create()
332                 .withSslHandlerFactory(new SslHandlerFactoryImpl(keystoreAdapter, protocol.getSpecification()))
333                 .withProtocol(NetconfClientConfiguration.NetconfClientProtocol.TLS);
334         } else {
335             throw new IllegalStateException("Unsupported protocol type: " + protocol.getName());
336         }
337
338         if (node.getOdlHelloMessageCapabilities() != null) {
339             reconnectingClientConfigurationBuilder.withOdlHelloCapabilities(
340                     Lists.newArrayList(node.getOdlHelloMessageCapabilities().getCapability()));
341         }
342
343         return reconnectingClientConfigurationBuilder
344                 .withName(nodeId.getValue())
345                 .withAddress(NetconfNodeUtils.toInetSocketAddress(node))
346                 .withConnectionTimeoutMillis(node.requireConnectionTimeoutMillis().toJava())
347                 .withReconnectStrategy(sf.createReconnectStrategy())
348                 .withConnectStrategyFactory(sf)
349                 .withSessionListener(listener)
350                 .build();
351     }
352
353     private AuthenticationHandler getHandlerFromCredentials(final Credentials credentials) {
354         if (credentials
355                 instanceof org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.device.rev221225
356                     .credentials.credentials.LoginPassword loginPassword) {
357             return new LoginPasswordHandler(loginPassword.getUsername(), loginPassword.getPassword());
358         }
359         if (credentials instanceof LoginPwUnencrypted unencrypted) {
360             final var loginPassword = unencrypted.getLoginPasswordUnencrypted();
361             return new LoginPasswordHandler(loginPassword.getUsername(), loginPassword.getPassword());
362         }
363         if (credentials instanceof LoginPw loginPw) {
364             final var loginPassword = loginPw.getLoginPassword();
365             return new LoginPasswordHandler(loginPassword.getUsername(),
366                     encryptionService.decrypt(loginPassword.getPassword()));
367         }
368         if (credentials instanceof KeyAuth keyAuth) {
369             final var keyPair = keyAuth.getKeyBased();
370             return new DatastoreBackedPublicKeyAuth(keyPair.getUsername(), keyPair.getKeyId(),
371                     keystoreAdapter, encryptionService);
372         }
373         throw new IllegalStateException("Unsupported credential type: " + credentials.getClass());
374     }
375 }