Attempt netconf remount regardless of error-type
[netconf.git] / netconf / netconf-topology / src / main / java / org / opendaylight / netconf / topology / AbstractNetconfTopology.java
index c7f5cf450988efd12356880aff73404aa1edc550..cb96ffb405f1fffd302d5d955b8a5f6fd359f850 100644 (file)
@@ -5,7 +5,6 @@
  * 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;
 
 import com.google.common.base.Optional;
@@ -15,6 +14,9 @@ import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.Uninterruptibles;
 import io.netty.util.concurrent.EventExecutor;
 import java.io.File;
 import java.math.BigDecimal;
@@ -24,19 +26,23 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 import org.opendaylight.aaa.encrypt.AAAEncryptionService;
 import org.opendaylight.controller.config.threadpool.ScheduledThreadPool;
 import org.opendaylight.controller.config.threadpool.ThreadPool;
-import org.opendaylight.controller.md.sal.binding.api.DataBroker;
-import org.opendaylight.controller.md.sal.dom.api.DOMMountPointService;
+import org.opendaylight.mdsal.binding.api.DataBroker;
+import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.netconf.api.NetconfMessage;
 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.client.conf.NetconfReconnectingClientConfigurationBuilder;
+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.LoginPassword;
+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;
@@ -44,24 +50,32 @@ import org.opendaylight.netconf.sal.connect.netconf.NetconfDevice;
 import org.opendaylight.netconf.sal.connect.netconf.NetconfDeviceBuilder;
 import org.opendaylight.netconf.sal.connect.netconf.NetconfStateSchemasResolverImpl;
 import org.opendaylight.netconf.sal.connect.netconf.SchemalessNetconfDevice;
+import org.opendaylight.netconf.sal.connect.netconf.auth.DatastoreBackedPublicKeyAuth;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCapabilities;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfDeviceCommunicator;
 import org.opendaylight.netconf.sal.connect.netconf.listener.NetconfSessionPreferences;
 import org.opendaylight.netconf.sal.connect.netconf.listener.UserPreferences;
 import org.opendaylight.netconf.sal.connect.netconf.sal.KeepaliveSalFacade;
+import org.opendaylight.netconf.sal.connect.netconf.sal.NetconfKeystoreAdapter;
 import org.opendaylight.netconf.sal.connect.netconf.schema.YangLibrarySchemaYangSourceProvider;
-import org.opendaylight.netconf.sal.connect.util.AuthEncryptor;
 import org.opendaylight.netconf.sal.connect.util.RemoteDeviceId;
+import org.opendaylight.netconf.sal.connect.util.SslHandlerFactoryImpl;
 import org.opendaylight.netconf.topology.api.NetconfTopology;
 import org.opendaylight.netconf.topology.api.SchemaRepositoryProvider;
-import org.opendaylight.protocol.framework.ReconnectStrategy;
-import org.opendaylight.protocol.framework.ReconnectStrategyFactory;
-import org.opendaylight.protocol.framework.TimedReconnectStrategy;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.Host;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.IpAddress;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.optional.rev190614.NetconfNodeAugmentedOptional;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.NetconfNode;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.parameters.Protocol;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.parameters.Protocol.Name;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.status.available.capabilities.AvailableCapability.CapabilityOrigin;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.Credentials;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.KeyAuth;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.LoginPw;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.LoginPwUnencrypted;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.key.auth.KeyBased;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.login.pw.LoginPassword;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.credentials.credentials.login.pw.unencrypted.LoginPasswordUnencrypted;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.NodeId;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
 import org.opendaylight.yangtools.yang.model.repo.api.SchemaContextFactory;
@@ -73,8 +87,10 @@ import org.opendaylight.yangtools.yang.model.repo.spi.PotentialSchemaSource;
 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistration;
 import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistry;
 import org.opendaylight.yangtools.yang.model.repo.util.FilesystemSchemaSourceCache;
+import org.opendaylight.yangtools.yang.model.repo.util.InMemorySchemaSourceCache;
 import org.opendaylight.yangtools.yang.parser.repo.SharedSchemaRepository;
-import org.opendaylight.yangtools.yang.parser.util.TextToASTTransformer;
+import org.opendaylight.yangtools.yang.parser.rfc7950.repo.ASTSchemaSource;
+import org.opendaylight.yangtools.yang.parser.rfc7950.repo.TextToASTTransformer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -119,12 +135,8 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
     private static final SharedSchemaRepository DEFAULT_SCHEMA_REPOSITORY =
             new SharedSchemaRepository(DEFAULT_SCHEMA_REPOSITORY_NAME);
 
-    /**
-     * The default <code>FilesystemSchemaSourceCache</code>, which stores cached files in <code>cache/schema</code>.
-     */
-    private static final FilesystemSchemaSourceCache<YangTextSchemaSource> DEFAULT_CACHE =
-            new FilesystemSchemaSourceCache<>(DEFAULT_SCHEMA_REPOSITORY, YangTextSchemaSource.class,
-                    new File(QUALIFIED_DEFAULT_CACHE_DIRECTORY));
+    public static final InMemorySchemaSourceCache<ASTSchemaSource> DEFAULT_AST_CACHE =
+            InMemorySchemaSourceCache.createSoftCache(DEFAULT_SCHEMA_REPOSITORY, ASTSchemaSource.class);
 
     /**
      * The default factory for creating <code>SchemaContext</code> instances.
@@ -149,43 +161,72 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
                 new NetconfDevice.SchemaResourcesDTO(DEFAULT_SCHEMA_REPOSITORY, DEFAULT_SCHEMA_REPOSITORY,
                         DEFAULT_SCHEMA_CONTEXT_FACTORY,
                         new NetconfStateSchemasResolverImpl()));
-        DEFAULT_SCHEMA_REPOSITORY.registerSchemaSourceListener(DEFAULT_CACHE);
+        DEFAULT_SCHEMA_REPOSITORY.registerSchemaSourceListener(DEFAULT_AST_CACHE);
         DEFAULT_SCHEMA_REPOSITORY.registerSchemaSourceListener(
                 TextToASTTransformer.create(DEFAULT_SCHEMA_REPOSITORY, DEFAULT_SCHEMA_REPOSITORY));
+
+        /*
+         * Create the default <code>FilesystemSchemaSourceCache</code>, which stores cached files
+         * in <code>cache/schema</code>. Try up to 3 times - we've seen intermittent failures on jenkins where
+         * FilesystemSchemaSourceCache throws an IAE due to mkdirs failure. The theory is that there's a race
+         * creating the dir and it already exists when mkdirs is called (mkdirs returns false in this case). In this
+         * scenario, a retry should succeed.
+         */
+        int tries = 1;
+        while (true) {
+            try {
+                FilesystemSchemaSourceCache<YangTextSchemaSource> defaultCache =
+                        new FilesystemSchemaSourceCache<>(DEFAULT_SCHEMA_REPOSITORY, YangTextSchemaSource.class,
+                                new File(QUALIFIED_DEFAULT_CACHE_DIRECTORY));
+                DEFAULT_SCHEMA_REPOSITORY.registerSchemaSourceListener(defaultCache);
+                break;
+            } catch (IllegalArgumentException e) {
+                if (tries++ >= 3) {
+                    LOG.error("Error creating default schema cache", e);
+                    break;
+                }
+                Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+            }
+        }
     }
 
-    protected final String topologyId;
     private final NetconfClientDispatcher clientDispatcher;
     private final EventExecutor eventExecutor;
+    private final DeviceActionFactory deviceActionFactory;
+    private final NetconfKeystoreAdapter keystoreAdapter;
     protected final ScheduledThreadPool keepaliveExecutor;
-    protected final ThreadPool processingExecutor;
+    protected final ListeningExecutorService processingExecutor;
     protected final SharedSchemaRepository sharedSchemaRepository;
     protected final DataBroker dataBroker;
     protected final DOMMountPointService mountPointService;
-
+    protected final String topologyId;
     protected SchemaSourceRegistry schemaRegistry = DEFAULT_SCHEMA_REPOSITORY;
     protected SchemaRepository schemaRepository = DEFAULT_SCHEMA_REPOSITORY;
     protected SchemaContextFactory schemaContextFactory = DEFAULT_SCHEMA_CONTEXT_FACTORY;
-
-    protected final HashMap<NodeId, NetconfConnectorDTO> activeConnectors = new HashMap<>();
-
+    protected String privateKeyPath;
+    protected String privateKeyPassphrase;
     protected final AAAEncryptionService encryptionService;
+    protected final HashMap<NodeId, NetconfConnectorDTO> activeConnectors = new HashMap<>();
 
     protected AbstractNetconfTopology(final String topologyId, final NetconfClientDispatcher clientDispatcher,
                                       final EventExecutor eventExecutor, final ScheduledThreadPool keepaliveExecutor,
                                       final ThreadPool processingExecutor,
                                       final SchemaRepositoryProvider schemaRepositoryProvider,
                                       final DataBroker dataBroker, final DOMMountPointService mountPointService,
-                                      final AAAEncryptionService encryptionService) {
+                                      final AAAEncryptionService encryptionService,
+                                      final DeviceActionFactory deviceActionFactory) {
         this.topologyId = topologyId;
         this.clientDispatcher = clientDispatcher;
         this.eventExecutor = eventExecutor;
         this.keepaliveExecutor = keepaliveExecutor;
-        this.processingExecutor = processingExecutor;
+        this.processingExecutor = MoreExecutors.listeningDecorator(processingExecutor.getExecutor());
+        this.deviceActionFactory = deviceActionFactory;
         this.sharedSchemaRepository = schemaRepositoryProvider.getSharedSchemaRepository();
         this.dataBroker = dataBroker;
         this.mountPointService = mountPointService;
         this.encryptionService = encryptionService;
+
+        this.keystoreAdapter = new NetconfKeystoreAdapter(dataBroker);
     }
 
     public void setSchemaRegistry(final SchemaSourceRegistry schemaRegistry) {
@@ -218,16 +259,15 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
     }
 
     protected ListenableFuture<NetconfDeviceCapabilities> setupConnection(final NodeId nodeId,
-                                                                        final Node configNode) {
-        final NetconfNode netconfNode = configNode.getAugmentation(NetconfNode.class);
-
-        AuthEncryptor.encryptIfNeeded(nodeId, netconfNode, encryptionService, topologyId, dataBroker);
+                                                                          final Node configNode) {
+        final NetconfNode netconfNode = configNode.augmentation(NetconfNode.class);
+        final NetconfNodeAugmentedOptional nodeOptional = configNode.augmentation(NetconfNodeAugmentedOptional.class);
 
         Preconditions.checkNotNull(netconfNode.getHost());
         Preconditions.checkNotNull(netconfNode.getPort());
         Preconditions.checkNotNull(netconfNode.isTcpOnly());
 
-        final NetconfConnectorDTO deviceCommunicatorDTO = createDeviceCommunicator(nodeId, netconfNode);
+        final NetconfConnectorDTO deviceCommunicatorDTO = createDeviceCommunicator(nodeId, netconfNode, nodeOptional);
         final NetconfDeviceCommunicator deviceCommunicator = deviceCommunicatorDTO.getCommunicator();
         final NetconfClientSessionListener netconfClientSessionListener = deviceCommunicatorDTO.getSessionListener();
         final NetconfReconnectingClientConfiguration clientConfig =
@@ -240,27 +280,31 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
         Futures.addCallback(future, new FutureCallback<NetconfDeviceCapabilities>() {
             @Override
             public void onSuccess(final NetconfDeviceCapabilities result) {
-                LOG.debug("Connector for : " + nodeId.getValue() + " started succesfully");
+                LOG.debug("Connector for {} started succesfully", nodeId.getValue());
             }
 
             @Override
             public void onFailure(final Throwable throwable) {
-                LOG.error("Connector for : " + nodeId.getValue() + " failed");
+                LOG.error("Connector for {} failed", nodeId.getValue(), throwable);
                 // remove this node from active connectors?
             }
-        });
+        }, MoreExecutors.directExecutor());
 
         return future;
     }
 
-    protected NetconfConnectorDTO createDeviceCommunicator(final NodeId nodeId,
-                                                         final NetconfNode node) {
+    protected NetconfConnectorDTO createDeviceCommunicator(final NodeId nodeId, final NetconfNode node) {
+        return createDeviceCommunicator(nodeId, node, null);
+    }
+
+    protected NetconfConnectorDTO createDeviceCommunicator(final NodeId nodeId, final NetconfNode node,
+            final NetconfNodeAugmentedOptional nodeOptional) {
         //setup default values since default value is not supported in mdsal
-        final Long defaultRequestTimeoutMillis = node.getDefaultRequestTimeoutMillis() == null
+        final long defaultRequestTimeoutMillis = node.getDefaultRequestTimeoutMillis() == null
                 ? DEFAULT_REQUEST_TIMEOUT_MILLIS : node.getDefaultRequestTimeoutMillis();
-        final Long keepaliveDelay = node.getKeepaliveDelay() == null
+        final long keepaliveDelay = node.getKeepaliveDelay() == null
                 ? DEFAULT_KEEPALIVE_DELAY : node.getKeepaliveDelay();
-        final Boolean reconnectOnChangedSchema = node.isReconnectOnChangedSchema() == null
+        final boolean reconnectOnChangedSchema = node.isReconnectOnChangedSchema() == null
                 ? DEFAULT_RECONNECT_ON_CHANGED_SCHEMA : node.isReconnectOnChangedSchema();
 
         final IpAddress ipAddress = node.getHost().getIpAddress();
@@ -274,7 +318,7 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
 
         if (keepaliveDelay > 0) {
             LOG.warn("Adding keepalive facade, for device {}", nodeId);
-            salFacade = new KeepaliveSalFacade(remoteDeviceId, salFacade, keepaliveExecutor.getExecutor(),
+            salFacade = new KeepaliveSalFacade(remoteDeviceId, salFacade, this.keepaliveExecutor.getExecutor(),
                     keepaliveDelay, defaultRequestTimeoutMillis);
         }
 
@@ -310,13 +354,19 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
         if (node.isSchemaless()) {
             device = new SchemalessNetconfDevice(remoteDeviceId, salFacade);
         } else {
-            device = new NetconfDeviceBuilder()
+            NetconfDeviceBuilder netconfDeviceBuilder = new NetconfDeviceBuilder()
                     .setReconnectOnSchemasChange(reconnectOnChangedSchema)
                     .setSchemaResourcesDTO(schemaResourcesDTO)
-                    .setGlobalProcessingExecutor(processingExecutor.getExecutor())
+                    .setGlobalProcessingExecutor(this.processingExecutor)
                     .setId(remoteDeviceId)
                     .setSalFacade(salFacade)
-                    .build();
+                    .setNode(node)
+                    .setEventExecutor(eventExecutor)
+                    .setNodeOptional(nodeOptional);
+            if (this.deviceActionFactory != null) {
+                netconfDeviceBuilder.setDeviceActionFactory(this.deviceActionFactory);
+            }
+            device = netconfDeviceBuilder.build();
         }
 
         final Optional<UserPreferences> userCapabilities = getUserCapabilities(node);
@@ -327,9 +377,15 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
             LOG.info("Concurrent rpc limit is smaller than 1, no limit will be enforced for device {}", remoteDeviceId);
         }
 
-        return new NetconfConnectorDTO(userCapabilities.isPresent()
-                ? new NetconfDeviceCommunicator(remoteDeviceId, device, userCapabilities.get(), rpcMessageLimit)
-                : new NetconfDeviceCommunicator(remoteDeviceId, device, rpcMessageLimit), salFacade);
+        NetconfDeviceCommunicator netconfDeviceCommunicator =
+             userCapabilities.isPresent() ? new NetconfDeviceCommunicator(remoteDeviceId, device,
+                     userCapabilities.get(), rpcMessageLimit)
+            : new NetconfDeviceCommunicator(remoteDeviceId, device, rpcMessageLimit);
+
+        if (salFacade instanceof KeepaliveSalFacade) {
+            ((KeepaliveSalFacade)salFacade).setListener(netconfDeviceCommunicator);
+        }
+        return new NetconfConnectorDTO(netconfDeviceCommunicator, salFacade);
     }
 
     protected NetconfDevice.SchemaResourcesDTO setupSchemaCacheDTO(final NodeId nodeId, final NetconfNode node) {
@@ -381,14 +437,16 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
      */
     private NetconfDevice.SchemaResourcesDTO createSchemaResourcesDTO(final String moduleSchemaCacheDirectory) {
         final SharedSchemaRepository repository = new SharedSchemaRepository(moduleSchemaCacheDirectory);
-        final SchemaContextFactory schemaContextFactory
+        final SchemaContextFactory contextFactory
                 = repository.createSchemaContextFactory(SchemaSourceFilter.ALWAYS_ACCEPT);
         setSchemaRegistry(repository);
-        setSchemaContextFactory(schemaContextFactory);
+        setSchemaContextFactory(contextFactory);
         final FilesystemSchemaSourceCache<YangTextSchemaSource> deviceCache =
                 createDeviceFilesystemCache(moduleSchemaCacheDirectory);
         repository.registerSchemaSourceListener(deviceCache);
-        return new NetconfDevice.SchemaResourcesDTO(repository, repository, schemaContextFactory,
+        repository.registerSchemaSourceListener(
+            InMemorySchemaSourceCache.createSoftCache(repository, ASTSchemaSource.class));
+        return new NetconfDevice.SchemaResourcesDTO(repository, repository, contextFactory,
                 new NetconfStateSchemasResolverImpl());
     }
 
@@ -405,6 +463,20 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
                 new File(relativeSchemaCacheDirectory));
     }
 
+    /**
+     * Sets the private key path from location specified in configuration file using blueprint.
+     */
+    public void setPrivateKeyPath(final String privateKeyPath) {
+        this.privateKeyPath = privateKeyPath;
+    }
+
+    /**
+     * Sets the private key passphrase from location specified in configuration file using blueprint.
+     */
+    public void setPrivateKeyPassphrase(final String privateKeyPassphrase) {
+        this.privateKeyPassphrase = privateKeyPassphrase;
+    }
+
     public NetconfReconnectingClientConfiguration getClientConfig(final NetconfClientSessionListener listener,
                                                                   final NetconfNode node) {
 
@@ -421,48 +493,80 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
 
         final ReconnectStrategyFactory sf = new TimedReconnectStrategyFactory(eventExecutor,
                 maxConnectionAttempts, betweenAttemptsTimeoutMillis, sleepFactor);
-        final ReconnectStrategy strategy = sf.createReconnectStrategy();
-
-        final AuthenticationHandler authHandler;
-        final Credentials credentials = node.getCredentials();
-        if (credentials instanceof org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114
-                .netconf.node.credentials.credentials.LoginPassword) {
-            authHandler = new LoginPassword(
-                    ((org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114
-                            .netconf.node.credentials.credentials.LoginPassword) credentials).getUsername(),
-                    ((org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114
-                            .netconf.node.credentials.credentials.LoginPassword) credentials).getPassword(),
-                    encryptionService);
+
+        final NetconfReconnectingClientConfigurationBuilder reconnectingClientConfigurationBuilder;
+        final Protocol protocol = node.getProtocol();
+        if (node.isTcpOnly()) {
+            reconnectingClientConfigurationBuilder = NetconfReconnectingClientConfigurationBuilder.create()
+                    .withProtocol(NetconfClientConfiguration.NetconfClientProtocol.TCP)
+                    .withAuthHandler(getHandlerFromCredentials(node.getCredentials()));
+        } else if (protocol == null || protocol.getName() == Name.SSH) {
+            reconnectingClientConfigurationBuilder = NetconfReconnectingClientConfigurationBuilder.create()
+                    .withProtocol(NetconfClientConfiguration.NetconfClientProtocol.SSH)
+                    .withAuthHandler(getHandlerFromCredentials(node.getCredentials()));
+        } else if (protocol.getName() == Name.TLS) {
+            reconnectingClientConfigurationBuilder = NetconfReconnectingClientConfigurationBuilder.create()
+                .withSslHandlerFactory(new SslHandlerFactoryImpl(keystoreAdapter, protocol.getSpecification()))
+                .withProtocol(NetconfClientConfiguration.NetconfClientProtocol.TLS);
         } else {
-            throw new IllegalStateException("Only login/password authentification is supported");
+            throw new IllegalStateException("Unsupported protocol type: " + protocol.getName());
+        }
+
+        if (node.getOdlHelloMessageCapabilities() != null) {
+            reconnectingClientConfigurationBuilder
+                    .withOdlHelloCapabilities(node.getOdlHelloMessageCapabilities().getCapability());
         }
 
-        return NetconfReconnectingClientConfigurationBuilder.create()
+        return reconnectingClientConfigurationBuilder
                 .withAddress(socketAddress)
                 .withConnectionTimeoutMillis(clientConnectionTimeoutMillis)
-                .withReconnectStrategy(strategy)
-                .withAuthHandler(authHandler)
-                .withProtocol(node.isTcpOnly() ? NetconfClientConfiguration.NetconfClientProtocol.TCP :
-                        NetconfClientConfiguration.NetconfClientProtocol.SSH)
+                .withReconnectStrategy(sf.createReconnectStrategy())
                 .withConnectStrategyFactory(sf)
                 .withSessionListener(listener)
                 .build();
     }
 
+    private AuthenticationHandler getHandlerFromCredentials(final Credentials credentials) {
+        if (credentials instanceof org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology
+                .rev150114.netconf.node.credentials.credentials.LoginPassword) {
+            final org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology
+                    .rev150114.netconf.node.credentials.credentials.LoginPassword loginPassword
+                    = (org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology
+                    .rev150114.netconf.node.credentials.credentials.LoginPassword) credentials;
+            return new LoginPasswordHandler(loginPassword.getUsername(), loginPassword.getPassword());
+        }
+        if (credentials instanceof LoginPwUnencrypted) {
+            final LoginPasswordUnencrypted loginPassword =
+                    ((LoginPwUnencrypted) credentials).getLoginPasswordUnencrypted();
+            return new LoginPasswordHandler(loginPassword.getUsername(), loginPassword.getPassword());
+        }
+        if (credentials instanceof LoginPw) {
+            final LoginPassword loginPassword = ((LoginPw) credentials).getLoginPassword();
+            return new LoginPasswordHandler(loginPassword.getUsername(),
+                    encryptionService.decrypt(loginPassword.getPassword()));
+        }
+        if (credentials instanceof KeyAuth) {
+            final KeyBased keyPair = ((KeyAuth) credentials).getKeyBased();
+            return new DatastoreBackedPublicKeyAuth(keyPair.getUsername(), keyPair.getKeyId(),
+                    keystoreAdapter, encryptionService);
+        }
+        throw new IllegalStateException("Unsupported credential type: " + credentials.getClass());
+    }
+
     protected abstract RemoteDeviceHandler<NetconfSessionPreferences> createSalFacade(RemoteDeviceId id);
 
-    private InetSocketAddress getSocketAddress(final Host host, final int port) {
+    private static InetSocketAddress getSocketAddress(final Host host, final int port) {
         if (host.getDomainName() != null) {
             return new InetSocketAddress(host.getDomainName().getValue(), port);
-        } else {
-            final IpAddress ipAddress = host.getIpAddress();
-            final String ip = ipAddress.getIpv4Address() != null
-                    ? ipAddress.getIpv4Address().getValue() : ipAddress.getIpv6Address().getValue();
-            return new InetSocketAddress(ip, port);
         }
+
+        final IpAddress ipAddress = host.getIpAddress();
+        final String ip = ipAddress.getIpv4Address() != null ? ipAddress.getIpv4Address().getValue()
+                : ipAddress.getIpv6Address().getValue();
+        return new InetSocketAddress(ip, port);
     }
 
-    private Optional<UserPreferences> getUserCapabilities(final NetconfNode node) {
+    private static Optional<UserPreferences> getUserCapabilities(final NetconfNode node) {
         // if none of yang-module-capabilities or non-module-capabilities is specified
         // just return absent
         if (node.getYangModuleCapabilities() == null && node.getNonModuleCapabilities() == null) {
@@ -493,35 +597,6 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
             .fromStrings(capabilities, CapabilityOrigin.UserDefined), overrideYangModuleCaps, overrideNonModuleCaps));
     }
 
-    private static final class TimedReconnectStrategyFactory implements ReconnectStrategyFactory {
-        private final Long connectionAttempts;
-        private final EventExecutor executor;
-        private final double sleepFactor;
-        private final int minSleep;
-
-        TimedReconnectStrategyFactory(final EventExecutor executor, final Long maxConnectionAttempts,
-                                      final int minSleep, final BigDecimal sleepFactor) {
-            if (maxConnectionAttempts != null && maxConnectionAttempts > 0) {
-                connectionAttempts = maxConnectionAttempts;
-            } else {
-                connectionAttempts = null;
-            }
-
-            this.sleepFactor = sleepFactor.doubleValue();
-            this.executor = executor;
-            this.minSleep = minSleep;
-        }
-
-        @Override
-        public ReconnectStrategy createReconnectStrategy() {
-            final Long maxSleep = null;
-            final Long deadline = null;
-
-            return new TimedReconnectStrategy(executor, minSleep,
-                    minSleep, sleepFactor, maxSleep, connectionAttempts, deadline);
-        }
-    }
-
     protected static class NetconfConnectorDTO implements AutoCloseable {
 
         private final NetconfDeviceCommunicator communicator;