Merge "Bug-6346: Allow over-ride of non-module capabilities"
[netconf.git] / netconf / netconf-topology / src / main / java / org / opendaylight / netconf / topology / 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
9 package org.opendaylight.netconf.topology;
10
11 import com.google.common.base.Optional;
12 import com.google.common.base.Preconditions;
13 import com.google.common.base.Strings;
14 import com.google.common.collect.Lists;
15 import com.google.common.util.concurrent.FutureCallback;
16 import com.google.common.util.concurrent.Futures;
17 import com.google.common.util.concurrent.ListenableFuture;
18 import io.netty.util.concurrent.EventExecutor;
19 import java.io.File;
20 import java.math.BigDecimal;
21 import java.net.InetSocketAddress;
22 import java.net.URL;
23 import java.util.ArrayList;
24 import java.util.HashMap;
25 import java.util.List;
26 import java.util.Map;
27 import org.opendaylight.controller.config.threadpool.ScheduledThreadPool;
28 import org.opendaylight.controller.config.threadpool.ThreadPool;
29 import org.opendaylight.controller.md.sal.binding.api.DataBroker;
30 import org.opendaylight.controller.sal.binding.api.BindingAwareBroker;
31 import org.opendaylight.controller.sal.core.api.Broker;
32 import org.opendaylight.netconf.api.NetconfMessage;
33 import org.opendaylight.netconf.client.NetconfClientDispatcher;
34 import org.opendaylight.netconf.client.NetconfClientSessionListener;
35 import org.opendaylight.netconf.client.conf.NetconfClientConfiguration;
36 import org.opendaylight.netconf.client.conf.NetconfReconnectingClientConfiguration;
37 import org.opendaylight.netconf.client.conf.NetconfReconnectingClientConfigurationBuilder;
38 import org.opendaylight.netconf.nettyutil.handler.ssh.authentication.AuthenticationHandler;
39 import org.opendaylight.netconf.nettyutil.handler.ssh.authentication.LoginPassword;
40 import org.opendaylight.netconf.sal.connect.api.RemoteDevice;
41 import org.opendaylight.netconf.sal.connect.api.RemoteDeviceHandler;
42 import org.opendaylight.netconf.sal.connect.netconf.LibraryModulesSchemas;
43 import org.opendaylight.netconf.sal.connect.netconf.NetconfDevice;
44 import org.opendaylight.netconf.sal.connect.netconf.NetconfDeviceBuilder;
45 import org.opendaylight.netconf.sal.connect.netconf.NetconfStateSchemasResolverImpl;
46 import org.opendaylight.netconf.sal.connect.netconf.SchemalessNetconfDevice;
47 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCapabilities;
48 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
49 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
50 import org.opendaylight.netconf.sal.connect.netconf.listener.UserPreferences;
51 import org.opendaylight.netconf.sal.connect.netconf.sal.KeepaliveSalFacade;
52 import org.opendaylight.netconf.sal.connect.netconf.schema.YangLibrarySchemaYangSourceProvider;
53 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
54 import org.opendaylight.netconf.topology.api.NetconfTopology;
55 import org.opendaylight.netconf.topology.api.SchemaRepositoryProvider;
56 import org.opendaylight.protocol.framework.ReconnectStrategy;
57 import org.opendaylight.protocol.framework.ReconnectStrategyFactory;
58 import org.opendaylight.protocol.framework.TimedReconnectStrategy;
59 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.Host;
60 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.IpAddress;
61 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.NetconfNode;
62 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.status.available.capabilities.AvailableCapability.CapabilityOrigin;
63 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.Credentials;
64 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
65 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
66 import org.opendaylight.yangtools.yang.model.repo.api.SchemaContextFactory;
67 import org.opendaylight.yangtools.yang.model.repo.api.SchemaRepository;
68 import org.opendaylight.yangtools.yang.model.repo.api.SchemaSourceFilter;
69 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
70 import org.opendaylight.yangtools.yang.model.repo.api.YangTextSchemaSource;
71 import org.opendaylight.yangtools.yang.model.repo.spi.PotentialSchemaSource;
72 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistration;
73 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistry;
74 import org.opendaylight.yangtools.yang.model.repo.util.FilesystemSchemaSourceCache;
75 import org.opendaylight.yangtools.yang.parser.repo.SharedSchemaRepository;
76 import org.opendaylight.yangtools.yang.parser.util.TextToASTTransformer;
77 import org.slf4j.Logger;
78 import org.slf4j.LoggerFactory;
79
80 public abstract class AbstractNetconfTopology implements NetconfTopology {
81
82     private static final Logger LOG = LoggerFactory.getLogger(AbstractNetconfTopology.class);
83
84     protected static final long DEFAULT_REQUEST_TIMEOUT_MILLIS = 60000L;
85     protected static final int DEFAULT_KEEPALIVE_DELAY = 0;
86     protected static final boolean DEFAULT_RECONNECT_ON_CHANGED_SCHEMA = false;
87     protected static final int DEFAULT_CONCURRENT_RPC_LIMIT = 0;
88     private static final int DEFAULT_MAX_CONNECTION_ATTEMPTS = 0;
89     private static final int DEFAULT_BETWEEN_ATTEMPTS_TIMEOUT_MILLIS = 2000;
90     private static final long DEFAULT_CONNECTION_TIMEOUT_MILLIS = 20000L;
91     private static final BigDecimal DEFAULT_SLEEP_FACTOR = new BigDecimal(1.5);
92
93     // constants related to Schema Cache(s)
94     /**
95      * Filesystem based caches are stored relative to the cache directory.
96      */
97     private static final String CACHE_DIRECTORY = "cache";
98
99     /**
100      * The default cache directory relative to <code>CACHE_DIRECTORY</code>
101      */
102     private static final String DEFAULT_CACHE_DIRECTORY = "schema";
103
104     /**
105      * The qualified schema cache directory <code>cache/schema</code>
106      */
107     private static final String QUALIFIED_DEFAULT_CACHE_DIRECTORY = CACHE_DIRECTORY + File.separator+ DEFAULT_CACHE_DIRECTORY;
108
109     /**
110      * The name for the default schema repository
111      */
112     private static final String DEFAULT_SCHEMA_REPOSITORY_NAME = "sal-netconf-connector";
113
114     /**
115      * The default schema repository in the case that one is not specified.
116      */
117     private static final SharedSchemaRepository DEFAULT_SCHEMA_REPOSITORY =
118             new SharedSchemaRepository(DEFAULT_SCHEMA_REPOSITORY_NAME);
119
120     /**
121      * The default <code>FilesystemSchemaSourceCache</code>, which stores cached files in <code>cache/schema</code>.
122      */
123     private static final FilesystemSchemaSourceCache<YangTextSchemaSource> DEFAULT_CACHE =
124             new FilesystemSchemaSourceCache<>(DEFAULT_SCHEMA_REPOSITORY, YangTextSchemaSource.class,
125                     new File(QUALIFIED_DEFAULT_CACHE_DIRECTORY));
126
127     /**
128      * The default factory for creating <code>SchemaContext</code> instances.
129      */
130     private static final SchemaContextFactory DEFAULT_SCHEMA_CONTEXT_FACTORY =
131             DEFAULT_SCHEMA_REPOSITORY.createSchemaContextFactory(SchemaSourceFilter.ALWAYS_ACCEPT);
132
133     /**
134      * Keeps track of initialized Schema resources.  A Map is maintained in which the key represents the name
135      * of the schema cache directory, and the value is a corresponding <code>SchemaResourcesDTO</code>.  The
136      * <code>SchemaResourcesDTO</code> is essentially a container that allows for the extraction of the
137      * <code>SchemaRegistry</code> and <code>SchemaContextFactory</code> which should be used for a particular
138      * Netconf mount.  Access to <code>schemaResourcesDTOs</code> should be surrounded by appropriate
139      * synchronization locks.
140      */
141     private static final Map<String, NetconfDevice.SchemaResourcesDTO> schemaResourcesDTOs = new HashMap<>();
142
143     // Initializes default constant instances for the case when the default schema repository
144     // directory cache/schema is used.
145     static {
146         schemaResourcesDTOs.put(DEFAULT_CACHE_DIRECTORY,
147                 new NetconfDevice.SchemaResourcesDTO(DEFAULT_SCHEMA_REPOSITORY, DEFAULT_SCHEMA_REPOSITORY,
148                         DEFAULT_SCHEMA_CONTEXT_FACTORY,
149                         new NetconfStateSchemasResolverImpl()));
150         DEFAULT_SCHEMA_REPOSITORY.registerSchemaSourceListener(DEFAULT_CACHE);
151         DEFAULT_SCHEMA_REPOSITORY.registerSchemaSourceListener(
152                 TextToASTTransformer.create(DEFAULT_SCHEMA_REPOSITORY, DEFAULT_SCHEMA_REPOSITORY));
153     }
154
155     protected final String topologyId;
156     private final NetconfClientDispatcher clientDispatcher;
157     protected final BindingAwareBroker bindingAwareBroker;
158     protected final Broker domBroker;
159     private final EventExecutor eventExecutor;
160     protected final ScheduledThreadPool keepaliveExecutor;
161     protected final ThreadPool processingExecutor;
162     protected final SharedSchemaRepository sharedSchemaRepository;
163     protected final DataBroker dataBroker;
164
165     protected SchemaSourceRegistry schemaRegistry = DEFAULT_SCHEMA_REPOSITORY;
166     protected SchemaRepository schemaRepository = DEFAULT_SCHEMA_REPOSITORY;
167     protected SchemaContextFactory schemaContextFactory = DEFAULT_SCHEMA_CONTEXT_FACTORY;
168
169     protected final HashMap<NodeId, NetconfConnectorDTO> activeConnectors = new HashMap<>();
170
171     protected AbstractNetconfTopology(final String topologyId, final NetconfClientDispatcher clientDispatcher,
172                                       final BindingAwareBroker bindingAwareBroker, final Broker domBroker,
173                                       final EventExecutor eventExecutor, final ScheduledThreadPool keepaliveExecutor,
174                                       final ThreadPool processingExecutor, final SchemaRepositoryProvider schemaRepositoryProvider,
175                                       final DataBroker dataBroker) {
176         this.topologyId = topologyId;
177         this.clientDispatcher = clientDispatcher;
178         this.bindingAwareBroker = bindingAwareBroker;
179         this.domBroker = domBroker;
180         this.eventExecutor = eventExecutor;
181         this.keepaliveExecutor = keepaliveExecutor;
182         this.processingExecutor = processingExecutor;
183         this.sharedSchemaRepository = schemaRepositoryProvider.getSharedSchemaRepository();
184         this.dataBroker = dataBroker;
185     }
186
187     public void setSchemaRegistry(final SchemaSourceRegistry schemaRegistry) {
188         this.schemaRegistry = schemaRegistry;
189     }
190
191     public void setSchemaContextFactory(final SchemaContextFactory schemaContextFactory) {
192         this.schemaContextFactory = schemaContextFactory;
193     }
194
195     @Override
196     public ListenableFuture<NetconfDeviceCapabilities> connectNode(final NodeId nodeId, final Node configNode) {
197         LOG.info("Connecting RemoteDevice{{}} , with config {}", nodeId, configNode);
198         return setupConnection(nodeId, configNode);
199     }
200
201     @Override
202     public ListenableFuture<Void> disconnectNode(final NodeId nodeId) {
203         LOG.debug("Disconnecting RemoteDevice{{}}", nodeId.getValue());
204         if (!activeConnectors.containsKey(nodeId)) {
205             return Futures.immediateFailedFuture(new IllegalStateException("Unable to disconnect device that is not connected"));
206         }
207
208         // retrieve connection, and disconnect it
209         final NetconfConnectorDTO connectorDTO = activeConnectors.remove(nodeId);
210         connectorDTO.getCommunicator().close();
211         connectorDTO.getFacade().close();
212         return Futures.immediateFuture(null);
213     }
214
215     protected ListenableFuture<NetconfDeviceCapabilities> setupConnection(final NodeId nodeId,
216                                                                         final Node configNode) {
217         final NetconfNode netconfNode = configNode.getAugmentation(NetconfNode.class);
218
219         Preconditions.checkNotNull(netconfNode.getHost());
220         Preconditions.checkNotNull(netconfNode.getPort());
221         Preconditions.checkNotNull(netconfNode.isTcpOnly());
222
223         final NetconfConnectorDTO deviceCommunicatorDTO = createDeviceCommunicator(nodeId, netconfNode);
224         final NetconfDeviceCommunicator deviceCommunicator = deviceCommunicatorDTO.getCommunicator();
225         final NetconfClientSessionListener netconfClientSessionListener = deviceCommunicatorDTO.getSessionListener();
226         final NetconfReconnectingClientConfiguration clientConfig = getClientConfig(netconfClientSessionListener, netconfNode);
227         final ListenableFuture<NetconfDeviceCapabilities> future = deviceCommunicator.initializeRemoteConnection(clientDispatcher, clientConfig);
228
229         activeConnectors.put(nodeId, deviceCommunicatorDTO);
230
231         Futures.addCallback(future, new FutureCallback<NetconfDeviceCapabilities>() {
232             @Override
233             public void onSuccess(final NetconfDeviceCapabilities result) {
234                 LOG.debug("Connector for : " + nodeId.getValue() + " started succesfully");
235             }
236
237             @Override
238             public void onFailure(final Throwable t) {
239                 LOG.error("Connector for : " + nodeId.getValue() + " failed");
240                 // remove this node from active connectors?
241             }
242         });
243
244         return future;
245     }
246
247     protected NetconfConnectorDTO createDeviceCommunicator(final NodeId nodeId,
248                                                          final NetconfNode node) {
249         //setup default values since default value is not supported in mdsal
250         final Long defaultRequestTimeoutMillis = node.getDefaultRequestTimeoutMillis() == null ? DEFAULT_REQUEST_TIMEOUT_MILLIS : node.getDefaultRequestTimeoutMillis();
251         final Long keepaliveDelay = node.getKeepaliveDelay() == null ? DEFAULT_KEEPALIVE_DELAY : node.getKeepaliveDelay();
252         final Boolean reconnectOnChangedSchema = node.isReconnectOnChangedSchema() == null ? DEFAULT_RECONNECT_ON_CHANGED_SCHEMA : node.isReconnectOnChangedSchema();
253
254         final IpAddress ipAddress = node.getHost().getIpAddress();
255         final InetSocketAddress address = new InetSocketAddress(ipAddress.getIpv4Address() != null ?
256                 ipAddress.getIpv4Address().getValue() : ipAddress.getIpv6Address().getValue(),
257                 node.getPort().getValue());
258         final RemoteDeviceId remoteDeviceId = new RemoteDeviceId(nodeId.getValue(), address);
259
260         RemoteDeviceHandler<NetconfSessionPreferences> salFacade =
261                 createSalFacade(remoteDeviceId, domBroker, bindingAwareBroker);
262
263         if (keepaliveDelay > 0) {
264             LOG.warn("Adding keepalive facade, for device {}", nodeId);
265             salFacade = new KeepaliveSalFacade(remoteDeviceId, salFacade, keepaliveExecutor.getExecutor(), keepaliveDelay, defaultRequestTimeoutMillis);
266         }
267
268         // pre register yang library sources as fallback schemas to schema registry
269         final List<SchemaSourceRegistration<YangTextSchemaSource>> registeredYangLibSources = Lists.newArrayList();
270         if (node.getYangLibrary() != null) {
271             final String yangLibURL = node.getYangLibrary().getYangLibraryUrl().getValue();
272             final String yangLibUsername = node.getYangLibrary().getUsername();
273             final String yangLigPassword = node.getYangLibrary().getPassword();
274
275             final LibraryModulesSchemas libraryModulesSchemas;
276             if(yangLibURL != null) {
277                 if(yangLibUsername != null && yangLigPassword != null) {
278                     libraryModulesSchemas = LibraryModulesSchemas.create(yangLibURL, yangLibUsername, yangLigPassword);
279                 } else {
280                     libraryModulesSchemas = LibraryModulesSchemas.create(yangLibURL);
281                 }
282
283                 for (final Map.Entry<SourceIdentifier, URL> sourceIdentifierURLEntry : libraryModulesSchemas.getAvailableModels().entrySet()) {
284                     registeredYangLibSources.
285                             add(schemaRegistry.registerSchemaSource(
286                                     new YangLibrarySchemaYangSourceProvider(remoteDeviceId, libraryModulesSchemas.getAvailableModels()),
287                                     PotentialSchemaSource
288                                             .create(sourceIdentifierURLEntry.getKey(), YangTextSchemaSource.class,
289                                             PotentialSchemaSource.Costs.REMOTE_IO.getValue())));
290                 }
291             }
292         }
293
294         final NetconfDevice.SchemaResourcesDTO schemaResourcesDTO = setupSchemaCacheDTO(nodeId, node);
295         final RemoteDevice<NetconfSessionPreferences, NetconfMessage, NetconfDeviceCommunicator> device;
296         if (node.isSchemaless()) {
297             device = new SchemalessNetconfDevice(remoteDeviceId, salFacade);
298         } else {
299             device = new NetconfDeviceBuilder()
300                     .setReconnectOnSchemasChange(reconnectOnChangedSchema)
301                     .setSchemaResourcesDTO(schemaResourcesDTO)
302                     .setGlobalProcessingExecutor(processingExecutor.getExecutor())
303                     .setId(remoteDeviceId)
304                     .setSalFacade(salFacade)
305                     .build();
306         }
307
308         final Optional<UserPreferences> userCapabilities = getUserCapabilities(node);
309         final int rpcMessageLimit =
310                 node.getConcurrentRpcLimit() == null ? DEFAULT_CONCURRENT_RPC_LIMIT : node.getConcurrentRpcLimit();
311
312         if (rpcMessageLimit < 1) {
313             LOG.info("Concurrent rpc limit is smaller than 1, no limit will be enforced for device {}", remoteDeviceId);
314         }
315
316         return new NetconfConnectorDTO(userCapabilities.isPresent()
317                 ? new NetconfDeviceCommunicator(remoteDeviceId, device, userCapabilities.get(), rpcMessageLimit)
318                 : new NetconfDeviceCommunicator(remoteDeviceId, device, rpcMessageLimit), salFacade);
319     }
320
321     protected NetconfDevice.SchemaResourcesDTO setupSchemaCacheDTO(final NodeId nodeId, final NetconfNode node) {
322         // Setup information related to the SchemaRegistry, SchemaResourceFactory, etc.
323         NetconfDevice.SchemaResourcesDTO schemaResourcesDTO = null;
324         final String moduleSchemaCacheDirectory = node.getSchemaCacheDirectory();
325         // Only checks to ensure the String is not empty or null;  further checks related to directory accessibility and file permissions
326         // are handled during the FilesystemSchemaSourceCache initialization.
327         if (!Strings.isNullOrEmpty(moduleSchemaCacheDirectory)) {
328             // If a custom schema cache directory is specified, create the backing DTO; otherwise, the SchemaRegistry and
329             // SchemaContextFactory remain the default values.
330             if (!moduleSchemaCacheDirectory.equals(DEFAULT_CACHE_DIRECTORY)) {
331                 // Multiple modules may be created at once;  synchronize to avoid issues with data consistency among threads.
332                 synchronized(schemaResourcesDTOs) {
333                     // Look for the cached DTO to reuse SchemaRegistry and SchemaContextFactory variables if they already exist
334                     schemaResourcesDTO = schemaResourcesDTOs.get(moduleSchemaCacheDirectory);
335                     if (schemaResourcesDTO == null) {
336                         schemaResourcesDTO = createSchemaResourcesDTO(moduleSchemaCacheDirectory);
337                         schemaResourcesDTO.getSchemaRegistry().registerSchemaSourceListener(
338                                 TextToASTTransformer.create((SchemaRepository) schemaResourcesDTO.getSchemaRegistry(), schemaResourcesDTO.getSchemaRegistry())
339                         );
340                         schemaResourcesDTOs.put(moduleSchemaCacheDirectory, schemaResourcesDTO);
341                     }
342                 }
343                 LOG.info("Netconf connector for device {} will use schema cache directory {} instead of {}",
344                         nodeId.getValue(), moduleSchemaCacheDirectory, DEFAULT_CACHE_DIRECTORY);
345             }
346         } else {
347             LOG.warn("schema-cache-directory for {} is null or empty;  using the default {}",
348                     nodeId.getValue(), QUALIFIED_DEFAULT_CACHE_DIRECTORY);
349         }
350
351         if (schemaResourcesDTO == null) {
352             schemaResourcesDTO = new NetconfDevice.SchemaResourcesDTO(schemaRegistry, schemaRepository, schemaContextFactory,
353                     new NetconfStateSchemasResolverImpl());
354         }
355
356         return schemaResourcesDTO;
357     }
358
359     /**
360      * Creates the backing Schema classes for a particular directory.
361      *
362      * @param moduleSchemaCacheDirectory The string directory relative to "cache"
363      * @return A DTO containing the Schema classes for the Netconf mount.
364      */
365     private NetconfDevice.SchemaResourcesDTO createSchemaResourcesDTO(final String moduleSchemaCacheDirectory) {
366         final SharedSchemaRepository repository = new SharedSchemaRepository(moduleSchemaCacheDirectory);
367         final SchemaContextFactory schemaContextFactory
368                 = repository.createSchemaContextFactory(SchemaSourceFilter.ALWAYS_ACCEPT);
369         setSchemaRegistry(repository);
370         setSchemaContextFactory(schemaContextFactory);
371         final FilesystemSchemaSourceCache<YangTextSchemaSource> deviceCache =
372                 createDeviceFilesystemCache(moduleSchemaCacheDirectory);
373         repository.registerSchemaSourceListener(deviceCache);
374         return new NetconfDevice.SchemaResourcesDTO(repository, repository, schemaContextFactory,
375                 new NetconfStateSchemasResolverImpl());
376     }
377
378     /**
379      * Creates a <code>FilesystemSchemaSourceCache</code> for the custom schema cache directory.
380      *
381      * @param schemaCacheDirectory The custom cache directory relative to "cache"
382      * @return A <code>FilesystemSchemaSourceCache</code> for the custom schema cache directory
383      */
384     private FilesystemSchemaSourceCache<YangTextSchemaSource> createDeviceFilesystemCache(final String schemaCacheDirectory) {
385         final String relativeSchemaCacheDirectory = CACHE_DIRECTORY + File.separator + schemaCacheDirectory;
386         return new FilesystemSchemaSourceCache<>(schemaRegistry, YangTextSchemaSource.class, new File(relativeSchemaCacheDirectory));
387     }
388
389     public NetconfReconnectingClientConfiguration getClientConfig(final NetconfClientSessionListener listener, final NetconfNode node) {
390
391         //setup default values since default value is not supported in mdsal
392         final long clientConnectionTimeoutMillis = node.getConnectionTimeoutMillis() == null ? DEFAULT_CONNECTION_TIMEOUT_MILLIS : node.getConnectionTimeoutMillis();
393         final long maxConnectionAttempts = node.getMaxConnectionAttempts() == null ? DEFAULT_MAX_CONNECTION_ATTEMPTS : node.getMaxConnectionAttempts();
394         final int betweenAttemptsTimeoutMillis = node.getBetweenAttemptsTimeoutMillis() == null ? DEFAULT_BETWEEN_ATTEMPTS_TIMEOUT_MILLIS : node.getBetweenAttemptsTimeoutMillis();
395         final BigDecimal sleepFactor = node.getSleepFactor() == null ? DEFAULT_SLEEP_FACTOR : node.getSleepFactor();
396
397         final InetSocketAddress socketAddress = getSocketAddress(node.getHost(), node.getPort().getValue());
398
399         final ReconnectStrategyFactory sf = new TimedReconnectStrategyFactory(eventExecutor,
400                 maxConnectionAttempts, betweenAttemptsTimeoutMillis, sleepFactor);
401         final ReconnectStrategy strategy = sf.createReconnectStrategy();
402
403         final AuthenticationHandler authHandler;
404         final Credentials credentials = node.getCredentials();
405         if (credentials instanceof org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.LoginPassword) {
406             authHandler = new LoginPassword(
407                     ((org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.LoginPassword) credentials).getUsername(),
408                     ((org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.LoginPassword) credentials).getPassword());
409         } else {
410             throw new IllegalStateException("Only login/password authentification is supported");
411         }
412
413         return NetconfReconnectingClientConfigurationBuilder.create()
414                 .withAddress(socketAddress)
415                 .withConnectionTimeoutMillis(clientConnectionTimeoutMillis)
416                 .withReconnectStrategy(strategy)
417                 .withAuthHandler(authHandler)
418                 .withProtocol(node.isTcpOnly() ?
419                         NetconfClientConfiguration.NetconfClientProtocol.TCP :
420                         NetconfClientConfiguration.NetconfClientProtocol.SSH)
421                 .withConnectStrategyFactory(sf)
422                 .withSessionListener(listener)
423                 .build();
424     }
425
426     protected abstract RemoteDeviceHandler<NetconfSessionPreferences> createSalFacade(final RemoteDeviceId id, final Broker domBroker, final BindingAwareBroker bindingBroker);
427
428     private InetSocketAddress getSocketAddress(final Host host, final int port) {
429         if(host.getDomainName() != null) {
430             return new InetSocketAddress(host.getDomainName().getValue(), port);
431         } else {
432             final IpAddress ipAddress = host.getIpAddress();
433             final String ip = ipAddress.getIpv4Address() != null ? ipAddress.getIpv4Address().getValue() : ipAddress.getIpv6Address().getValue();
434             return new InetSocketAddress(ip, port);
435         }
436     }
437
438     private Optional<UserPreferences> getUserCapabilities(final NetconfNode node) {
439         // if none of yang-module-capabilities or non-module-capabilities is specified
440         // just return absent
441         if (node.getYangModuleCapabilities() == null && node.getNonModuleCapabilities() == null) {
442             return Optional.absent();
443         }
444
445         final List<String> capabilities = new ArrayList<>();
446
447         boolean overrideYangModuleCaps = false;
448         if (node.getYangModuleCapabilities() != null) {
449             capabilities.addAll(node.getYangModuleCapabilities().getCapability());
450             overrideYangModuleCaps = node.getYangModuleCapabilities().isOverride();
451         }
452
453         //non-module capabilities should not exist in yang module capabilities
454         final NetconfSessionPreferences netconfSessionPreferences = NetconfSessionPreferences.fromStrings(capabilities);
455         Preconditions.checkState(netconfSessionPreferences.getNonModuleCaps().isEmpty(), "List yang-module-capabilities/capability " +
456                 "should contain only module based capabilities. Non-module capabilities used: " +
457                 netconfSessionPreferences.getNonModuleCaps());
458
459         boolean overrideNonModuleCaps = false;
460         if (node.getNonModuleCapabilities() != null) {
461             capabilities.addAll(node.getNonModuleCapabilities().getCapability());
462             overrideNonModuleCaps = node.getNonModuleCapabilities().isOverride();
463         }
464
465         return Optional.of(new UserPreferences(NetconfSessionPreferences.fromStrings(capabilities, CapabilityOrigin.UserDefined),
466                 overrideYangModuleCaps, overrideNonModuleCaps));
467     }
468
469     private static final class TimedReconnectStrategyFactory implements ReconnectStrategyFactory {
470         private final Long connectionAttempts;
471         private final EventExecutor executor;
472         private final double sleepFactor;
473         private final int minSleep;
474
475         TimedReconnectStrategyFactory(final EventExecutor executor, final Long maxConnectionAttempts, final int minSleep, final BigDecimal sleepFactor) {
476             if (maxConnectionAttempts != null && maxConnectionAttempts > 0) {
477                 connectionAttempts = maxConnectionAttempts;
478             } else {
479                 connectionAttempts = null;
480             }
481
482             this.sleepFactor = sleepFactor.doubleValue();
483             this.executor = executor;
484             this.minSleep = minSleep;
485         }
486
487         @Override
488         public ReconnectStrategy createReconnectStrategy() {
489             final Long maxSleep = null;
490             final Long deadline = null;
491
492             return new TimedReconnectStrategy(executor, minSleep,
493                     minSleep, sleepFactor, maxSleep, connectionAttempts, deadline);
494         }
495     }
496
497     protected static class NetconfConnectorDTO implements AutoCloseable {
498
499         private final NetconfDeviceCommunicator communicator;
500         private final RemoteDeviceHandler<NetconfSessionPreferences> facade;
501
502         public NetconfConnectorDTO(final NetconfDeviceCommunicator communicator, final RemoteDeviceHandler<NetconfSessionPreferences> facade) {
503             this.communicator = communicator;
504             this.facade = facade;
505         }
506
507         public NetconfDeviceCommunicator getCommunicator() {
508             return communicator;
509         }
510
511         public RemoteDeviceHandler<NetconfSessionPreferences> getFacade() {
512             return facade;
513         }
514
515         public NetconfClientSessionListener getSessionListener() {
516             return communicator;
517         }
518
519         @Override
520         public void close() {
521             communicator.close();
522             facade.close();
523         }
524     }
525 }