Attempt netconf remount regardless of error-type
[netconf.git] / netconf / netconf-topology / src / main / java / org / opendaylight / netconf / topology / AbstractNetconfTopology.java
index 5f7f3594dedc5df2ff7cf6bfdf246779b9da7676..cb96ffb405f1fffd302d5d955b8a5f6fd359f850 100644 (file)
@@ -5,50 +5,44 @@
  * 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;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 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 io.netty.handler.ssl.SslHandler;
+import com.google.common.util.concurrent.Uninterruptibles;
 import io.netty.util.concurrent.EventExecutor;
 import java.io.File;
-import java.io.IOException;
 import java.math.BigDecimal;
 import java.net.InetSocketAddress;
 import java.net.URL;
-import java.security.GeneralSecurityException;
-import java.security.KeyStore;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLEngine;
-import javax.net.ssl.TrustManagerFactory;
+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.SslHandlerFactory;
 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.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;
@@ -65,17 +59,15 @@ 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.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.parameters.protocol.Specification;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.netconf.node.topology.rev150114.netconf.node.connection.parameters.protocol.specification.TlsCase;
 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;
@@ -143,13 +135,6 @@ 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);
 
@@ -176,21 +161,45 @@ 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;
-    private final NetconfKeystoreAdapter keystoreAdapter;
+    protected final String topologyId;
     protected SchemaSourceRegistry schemaRegistry = DEFAULT_SCHEMA_REPOSITORY;
     protected SchemaRepository schemaRepository = DEFAULT_SCHEMA_REPOSITORY;
     protected SchemaContextFactory schemaContextFactory = DEFAULT_SCHEMA_CONTEXT_FACTORY;
@@ -204,12 +213,14 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
                                       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;
@@ -249,13 +260,14 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
 
     protected ListenableFuture<NetconfDeviceCapabilities> setupConnection(final NodeId nodeId,
                                                                           final Node configNode) {
-        final NetconfNode netconfNode = configNode.getAugmentation(NetconfNode.class);
+        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 =
@@ -268,12 +280,12 @@ 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());
@@ -281,8 +293,12 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
         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
                 ? DEFAULT_REQUEST_TIMEOUT_MILLIS : node.getDefaultRequestTimeoutMillis();
@@ -302,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);
         }
 
@@ -338,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);
@@ -355,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) {
@@ -465,31 +493,34 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
 
         final ReconnectStrategyFactory sf = new TimedReconnectStrategyFactory(eventExecutor,
                 maxConnectionAttempts, betweenAttemptsTimeoutMillis, sleepFactor);
-        final ReconnectStrategy strategy = sf.createReconnectStrategy();
-
-        final NetconfReconnectingClientConfigurationBuilder reconnectingClientConfigurationBuilder =
-                NetconfReconnectingClientConfigurationBuilder.create();
 
-        if (node.isTcpOnly() || node.getProtocol() == null || node.getProtocol().getName() == Name.SSH) {
-            final AuthenticationHandler authHandler = getHandlerFromCredentials(node.getCredentials());
-            reconnectingClientConfigurationBuilder
-                .withAuthHandler(authHandler)
-                .withProtocol(node.isTcpOnly() ? NetconfClientConfiguration.NetconfClientProtocol.TCP :
-                    NetconfClientConfiguration.NetconfClientProtocol.SSH);
-        } else if (node.getProtocol().getName() == Name.TLS) {
-            final SslHandlerFactory sslHandlerFactory = new SslHandlerFactoryImpl(keystoreAdapter,
-                    node.getProtocol().getSpecification());
-            reconnectingClientConfigurationBuilder
-                .withSslHandlerFactory(sslHandlerFactory)
+        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("Unsupported protocol type: " + node.getProtocol().getName().getClass());
+            throw new IllegalStateException("Unsupported protocol type: " + protocol.getName());
+        }
+
+        if (node.getOdlHelloMessageCapabilities() != null) {
+            reconnectingClientConfigurationBuilder
+                    .withOdlHelloCapabilities(node.getOdlHelloMessageCapabilities().getCapability());
         }
 
         return reconnectingClientConfigurationBuilder
                 .withAddress(socketAddress)
                 .withConnectionTimeoutMillis(clientConnectionTimeoutMillis)
-                .withReconnectStrategy(strategy)
+                .withReconnectStrategy(sf.createReconnectStrategy())
                 .withConnectStrategyFactory(sf)
                 .withSessionListener(listener)
                 .build();
@@ -524,18 +555,18 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
 
     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) {
@@ -566,32 +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() {
-            return new TimedReconnectStrategy(executor, minSleep,
-                    minSleep, sleepFactor, null /*maxSleep*/, connectionAttempts, null /*deadline*/);
-        }
-    }
-
     protected static class NetconfConnectorDTO implements AutoCloseable {
 
         private final NetconfDeviceCommunicator communicator;
@@ -621,50 +626,4 @@ public abstract class AbstractNetconfTopology implements NetconfTopology {
             facade.close();
         }
     }
-
-    private static final class SslHandlerFactoryImpl implements SslHandlerFactory {
-        private final NetconfKeystoreAdapter keystoreAdapter;
-        private final Optional<Specification> specOptional;
-
-        SslHandlerFactoryImpl(final NetconfKeystoreAdapter keystoreAdapter, final Specification specification) {
-            this.keystoreAdapter = keystoreAdapter;
-            this.specOptional = Optional.fromNullable(specification);
-        }
-
-        @Override
-        public SslHandler createSslHandler() {
-            try {
-                final KeyStore keyStore = keystoreAdapter.getJavaKeyStore();
-
-                final KeyManagerFactory kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-                kmf.init(keyStore, "".toCharArray());
-
-                final TrustManagerFactory tmf =
-                        TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-                tmf.init(keyStore);
-
-                final SSLContext sslCtx = SSLContext.getInstance("TLS");
-                sslCtx.init(kmf.getKeyManagers(), tmf.getTrustManagers(), null);
-                final SSLEngine engine = sslCtx.createSSLEngine();
-                engine.setUseClientMode(true);
-
-                final Set<String> protocols = Sets.newHashSet(engine.getSupportedProtocols());
-                if (specOptional.isPresent()) {
-                    final Specification specification = specOptional.get();
-                    if (!(specification instanceof TlsCase)) {
-                        throw new IllegalArgumentException("Cannot get TLS specification from: " + specification);
-                    }
-                    protocols.removeAll(((TlsCase)specification).getTls().getExcludedVersions());
-                }
-
-                engine.setEnabledProtocols(protocols.toArray(new String[0]));
-                engine.setEnabledCipherSuites(engine.getSupportedCipherSuites());
-                engine.setEnableSessionCreation(true);
-
-                return new SslHandler(engine);
-            } catch (GeneralSecurityException | IOException exc) {
-                throw new IllegalStateException(exc);
-            }
-        }
-    }
 }