|  | @@ -22,24 +22,24 @@ import org.apache.logging.log4j.LogManager;
 | 
	
		
			
				|  |  |  import org.apache.logging.log4j.Logger;
 | 
	
		
			
				|  |  |  import org.apache.logging.log4j.message.ParameterizedMessage;
 | 
	
		
			
				|  |  |  import org.elasticsearch.action.ActionListener;
 | 
	
		
			
				|  |  | -import org.elasticsearch.action.support.PlainActionFuture;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.node.DiscoveryNode;
 | 
	
		
			
				|  |  | -import org.elasticsearch.common.CheckedBiConsumer;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lease.Releasable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Settings;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.util.concurrent.KeyedLock;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.util.concurrent.RunOnce;
 | 
	
		
			
				|  |  |  import org.elasticsearch.core.internal.io.IOUtils;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import java.io.Closeable;
 | 
	
		
			
				|  |  | -import java.io.IOException;
 | 
	
		
			
				|  |  | +import java.util.ArrayList;
 | 
	
		
			
				|  |  |  import java.util.Iterator;
 | 
	
		
			
				|  |  | +import java.util.List;
 | 
	
		
			
				|  |  |  import java.util.Map;
 | 
	
		
			
				|  |  |  import java.util.concurrent.ConcurrentMap;
 | 
	
		
			
				|  |  |  import java.util.concurrent.CopyOnWriteArrayList;
 | 
	
		
			
				|  |  | +import java.util.concurrent.CountDownLatch;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicBoolean;
 | 
	
		
			
				|  |  | -import java.util.concurrent.locks.ReadWriteLock;
 | 
	
		
			
				|  |  | -import java.util.concurrent.locks.ReentrantReadWriteLock;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  /**
 | 
	
		
			
				|  |  |   * This class manages node connections. The connection is opened by the underlying transport. Once the
 | 
	
	
		
			
				|  | @@ -51,11 +51,18 @@ public class ConnectionManager implements Closeable {
 | 
	
		
			
				|  |  |      private static final Logger logger = LogManager.getLogger(ConnectionManager.class);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private final ConcurrentMap<DiscoveryNode, Transport.Connection> connectedNodes = ConcurrentCollections.newConcurrentMap();
 | 
	
		
			
				|  |  | -    private final KeyedLock<String> connectionLock = new KeyedLock<>();
 | 
	
		
			
				|  |  | +    private final KeyedLock<String> connectionLock = new KeyedLock<>(); // protects concurrent access to connectingNodes
 | 
	
		
			
				|  |  | +    private final Map<DiscoveryNode, List<ActionListener<Void>>> connectingNodes = ConcurrentCollections.newConcurrentMap();
 | 
	
		
			
				|  |  | +    private final AbstractRefCounted connectingRefCounter = new AbstractRefCounted("connection manager") {
 | 
	
		
			
				|  |  | +        @Override
 | 
	
		
			
				|  |  | +        protected void closeInternal() {
 | 
	
		
			
				|  |  | +            closeLatch.countDown();
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    };
 | 
	
		
			
				|  |  |      private final Transport transport;
 | 
	
		
			
				|  |  |      private final ConnectionProfile defaultProfile;
 | 
	
		
			
				|  |  | -    private final AtomicBoolean isClosed = new AtomicBoolean(false);
 | 
	
		
			
				|  |  | -    private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
 | 
	
		
			
				|  |  | +    private final AtomicBoolean closing = new AtomicBoolean(false);
 | 
	
		
			
				|  |  | +    private final CountDownLatch closeLatch = new CountDownLatch(1);
 | 
	
		
			
				|  |  |      private final DelegatingNodeConnectionListener connectionListener = new DelegatingNodeConnectionListener();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      public ConnectionManager(Settings settings, Transport transport) {
 | 
	
	
		
			
				|  | @@ -75,66 +82,119 @@ public class ConnectionManager implements Closeable {
 | 
	
		
			
				|  |  |          this.connectionListener.listeners.remove(listener);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public Transport.Connection openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) {
 | 
	
		
			
				|  |  | +    public void openConnection(DiscoveryNode node, ConnectionProfile connectionProfile, ActionListener<Transport.Connection> listener) {
 | 
	
		
			
				|  |  |          ConnectionProfile resolvedProfile = ConnectionProfile.resolveConnectionProfile(connectionProfile, defaultProfile);
 | 
	
		
			
				|  |  | -        return internalOpenConnection(node, resolvedProfile);
 | 
	
		
			
				|  |  | +        internalOpenConnection(node, resolvedProfile, listener);
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    @FunctionalInterface
 | 
	
		
			
				|  |  | +    public interface ConnectionValidator {
 | 
	
		
			
				|  |  | +        void validate(Transport.Connection connection, ConnectionProfile profile, ActionListener<Void> listener);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      /**
 | 
	
		
			
				|  |  |       * Connects to a node with the given connection profile. If the node is already connected this method has no effect.
 | 
	
		
			
				|  |  |       * Once a successful is established, it can be validated before being exposed.
 | 
	
		
			
				|  |  | +     * The ActionListener will be called on the calling thread or the generic thread pool.
 | 
	
		
			
				|  |  |       */
 | 
	
		
			
				|  |  |      public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile,
 | 
	
		
			
				|  |  | -                              CheckedBiConsumer<Transport.Connection, ConnectionProfile, IOException> connectionValidator)
 | 
	
		
			
				|  |  | -        throws ConnectTransportException {
 | 
	
		
			
				|  |  | +                              ConnectionValidator connectionValidator,
 | 
	
		
			
				|  |  | +                              ActionListener<Void> listener) throws ConnectTransportException {
 | 
	
		
			
				|  |  |          ConnectionProfile resolvedProfile = ConnectionProfile.resolveConnectionProfile(connectionProfile, defaultProfile);
 | 
	
		
			
				|  |  |          if (node == null) {
 | 
	
		
			
				|  |  | -            throw new ConnectTransportException(null, "can't connect to a null node");
 | 
	
		
			
				|  |  | +            listener.onFailure(new ConnectTransportException(null, "can't connect to a null node"));
 | 
	
		
			
				|  |  | +            return;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | -        closeLock.readLock().lock(); // ensure we don't open connections while we are closing
 | 
	
		
			
				|  |  | -        try {
 | 
	
		
			
				|  |  | -            ensureOpen();
 | 
	
		
			
				|  |  | -            try (Releasable ignored = connectionLock.acquire(node.getId())) {
 | 
	
		
			
				|  |  | -                Transport.Connection connection = connectedNodes.get(node);
 | 
	
		
			
				|  |  | -                if (connection != null) {
 | 
	
		
			
				|  |  | -                    return;
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -                boolean success = false;
 | 
	
		
			
				|  |  | -                try {
 | 
	
		
			
				|  |  | -                    connection = internalOpenConnection(node, resolvedProfile);
 | 
	
		
			
				|  |  | -                    connectionValidator.accept(connection, resolvedProfile);
 | 
	
		
			
				|  |  | -                    // we acquire a connection lock, so no way there is an existing connection
 | 
	
		
			
				|  |  | -                    connectedNodes.put(node, connection);
 | 
	
		
			
				|  |  | -                    if (logger.isDebugEnabled()) {
 | 
	
		
			
				|  |  | -                        logger.debug("connected to node [{}]", node);
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        if (connectingRefCounter.tryIncRef() == false) {
 | 
	
		
			
				|  |  | +            listener.onFailure(new IllegalStateException("connection manager is closed"));
 | 
	
		
			
				|  |  | +            return;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        try (Releasable lock = connectionLock.acquire(node.getId())) {
 | 
	
		
			
				|  |  | +            Transport.Connection connection = connectedNodes.get(node);
 | 
	
		
			
				|  |  | +            if (connection != null) {
 | 
	
		
			
				|  |  | +                assert connectingNodes.containsKey(node) == false;
 | 
	
		
			
				|  |  | +                lock.close();
 | 
	
		
			
				|  |  | +                connectingRefCounter.decRef();
 | 
	
		
			
				|  |  | +                listener.onResponse(null);
 | 
	
		
			
				|  |  | +                return;
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +            final List<ActionListener<Void>> connectionListeners = connectingNodes.computeIfAbsent(node, n -> new ArrayList<>());
 | 
	
		
			
				|  |  | +            connectionListeners.add(listener);
 | 
	
		
			
				|  |  | +            if (connectionListeners.size() > 1) {
 | 
	
		
			
				|  |  | +                // wait on previous entry to complete connection attempt
 | 
	
		
			
				|  |  | +                connectingRefCounter.decRef();
 | 
	
		
			
				|  |  | +                return;
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        final RunOnce releaseOnce = new RunOnce(connectingRefCounter::decRef);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        internalOpenConnection(node, resolvedProfile, ActionListener.wrap(conn -> {
 | 
	
		
			
				|  |  | +            connectionValidator.validate(conn, resolvedProfile, ActionListener.wrap(
 | 
	
		
			
				|  |  | +                ignored -> {
 | 
	
		
			
				|  |  | +                    assert Transports.assertNotTransportThread("connection validator success");
 | 
	
		
			
				|  |  | +                    boolean success = false;
 | 
	
		
			
				|  |  | +                    List<ActionListener<Void>> listeners = null;
 | 
	
		
			
				|  |  |                      try {
 | 
	
		
			
				|  |  | -                        connectionListener.onNodeConnected(node);
 | 
	
		
			
				|  |  | +                        // we acquire a connection lock, so no way there is an existing connection
 | 
	
		
			
				|  |  | +                        try (Releasable ignored2 = connectionLock.acquire(node.getId())) {
 | 
	
		
			
				|  |  | +                            connectedNodes.put(node, conn);
 | 
	
		
			
				|  |  | +                            if (logger.isDebugEnabled()) {
 | 
	
		
			
				|  |  | +                                logger.debug("connected to node [{}]", node);
 | 
	
		
			
				|  |  | +                            }
 | 
	
		
			
				|  |  | +                            try {
 | 
	
		
			
				|  |  | +                                connectionListener.onNodeConnected(node);
 | 
	
		
			
				|  |  | +                            } finally {
 | 
	
		
			
				|  |  | +                                final Transport.Connection finalConnection = conn;
 | 
	
		
			
				|  |  | +                                conn.addCloseListener(ActionListener.wrap(() -> {
 | 
	
		
			
				|  |  | +                                    logger.trace("unregistering {} after connection close and marking as disconnected", node);
 | 
	
		
			
				|  |  | +                                    connectedNodes.remove(node, finalConnection);
 | 
	
		
			
				|  |  | +                                    connectionListener.onNodeDisconnected(node);
 | 
	
		
			
				|  |  | +                                }));
 | 
	
		
			
				|  |  | +                            }
 | 
	
		
			
				|  |  | +                            if (conn.isClosed()) {
 | 
	
		
			
				|  |  | +                                throw new NodeNotConnectedException(node, "connection concurrently closed");
 | 
	
		
			
				|  |  | +                            }
 | 
	
		
			
				|  |  | +                            success = true;
 | 
	
		
			
				|  |  | +                            listeners = connectingNodes.remove(node);
 | 
	
		
			
				|  |  | +                        }
 | 
	
		
			
				|  |  | +                    } catch (ConnectTransportException e) {
 | 
	
		
			
				|  |  | +                        throw e;
 | 
	
		
			
				|  |  | +                    } catch (Exception e) {
 | 
	
		
			
				|  |  | +                        throw new ConnectTransportException(node, "general node connection failure", e);
 | 
	
		
			
				|  |  |                      } finally {
 | 
	
		
			
				|  |  | -                        final Transport.Connection finalConnection = connection;
 | 
	
		
			
				|  |  | -                        connection.addCloseListener(ActionListener.wrap(() -> {
 | 
	
		
			
				|  |  | -                            connectedNodes.remove(node, finalConnection);
 | 
	
		
			
				|  |  | -                            connectionListener.onNodeDisconnected(node);
 | 
	
		
			
				|  |  | -                        }));
 | 
	
		
			
				|  |  | +                        if (success == false) { // close the connection if there is a failure
 | 
	
		
			
				|  |  | +                            logger.trace(() -> new ParameterizedMessage("failed to connect to [{}], cleaning dangling connections", node));
 | 
	
		
			
				|  |  | +                            IOUtils.closeWhileHandlingException(conn);
 | 
	
		
			
				|  |  | +                        } else {
 | 
	
		
			
				|  |  | +                            releaseOnce.run();
 | 
	
		
			
				|  |  | +                            ActionListener.onResponse(listeners, null);
 | 
	
		
			
				|  |  | +                        }
 | 
	
		
			
				|  |  |                      }
 | 
	
		
			
				|  |  | -                    if (connection.isClosed()) {
 | 
	
		
			
				|  |  | -                        throw new NodeNotConnectedException(node, "connection concurrently closed");
 | 
	
		
			
				|  |  | +                }, e -> {
 | 
	
		
			
				|  |  | +                    assert Transports.assertNotTransportThread("connection validator failure");
 | 
	
		
			
				|  |  | +                    IOUtils.closeWhileHandlingException(conn);
 | 
	
		
			
				|  |  | +                    final List<ActionListener<Void>> listeners;
 | 
	
		
			
				|  |  | +                    try (Releasable ignored = connectionLock.acquire(node.getId())) {
 | 
	
		
			
				|  |  | +                        listeners = connectingNodes.remove(node);
 | 
	
		
			
				|  |  |                      }
 | 
	
		
			
				|  |  | -                    success = true;
 | 
	
		
			
				|  |  | -                } catch (ConnectTransportException e) {
 | 
	
		
			
				|  |  | -                    throw e;
 | 
	
		
			
				|  |  | -                } catch (Exception e) {
 | 
	
		
			
				|  |  | -                    throw new ConnectTransportException(node, "general node connection failure", e);
 | 
	
		
			
				|  |  | -                } finally {
 | 
	
		
			
				|  |  | -                    if (success == false) { // close the connection if there is a failure
 | 
	
		
			
				|  |  | -                        logger.trace(() -> new ParameterizedMessage("failed to connect to [{}], cleaning dangling connections", node));
 | 
	
		
			
				|  |  | -                        IOUtils.closeWhileHandlingException(connection);
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | +                    releaseOnce.run();
 | 
	
		
			
				|  |  | +                    ActionListener.onFailure(listeners, e);
 | 
	
		
			
				|  |  | +                }));
 | 
	
		
			
				|  |  | +        }, e -> {
 | 
	
		
			
				|  |  | +            assert Transports.assertNotTransportThread("internalOpenConnection failure");
 | 
	
		
			
				|  |  | +            final List<ActionListener<Void>> listeners;
 | 
	
		
			
				|  |  | +            try (Releasable ignored = connectionLock.acquire(node.getId())) {
 | 
	
		
			
				|  |  | +                listeners = connectingNodes.remove(node);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | -        } finally {
 | 
	
		
			
				|  |  | -            closeLock.readLock().unlock();
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | +            releaseOnce.run();
 | 
	
		
			
				|  |  | +            if (listeners != null) {
 | 
	
		
			
				|  |  | +                ActionListener.onFailure(listeners, e);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        }));
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      /**
 | 
	
	
		
			
				|  | @@ -143,7 +203,7 @@ public class ConnectionManager implements Closeable {
 | 
	
		
			
				|  |  |       * maintained by this connection manager
 | 
	
		
			
				|  |  |       *
 | 
	
		
			
				|  |  |       * @throws NodeNotConnectedException if the node is not connected
 | 
	
		
			
				|  |  | -     * @see #connectToNode(DiscoveryNode, ConnectionProfile, CheckedBiConsumer)
 | 
	
		
			
				|  |  | +     * @see #connectToNode(DiscoveryNode, ConnectionProfile, ConnectionValidator, ActionListener)
 | 
	
		
			
				|  |  |       */
 | 
	
		
			
				|  |  |      public Transport.Connection getConnection(DiscoveryNode node) {
 | 
	
		
			
				|  |  |          Transport.Connection connection = connectedNodes.get(node);
 | 
	
	
		
			
				|  | @@ -180,55 +240,41 @@ public class ConnectionManager implements Closeable {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      @Override
 | 
	
		
			
				|  |  |      public void close() {
 | 
	
		
			
				|  |  | -        Transports.assertNotTransportThread("Closing ConnectionManager");
 | 
	
		
			
				|  |  | -        if (isClosed.compareAndSet(false, true)) {
 | 
	
		
			
				|  |  | -            closeLock.writeLock().lock();
 | 
	
		
			
				|  |  | +        assert Transports.assertNotTransportThread("Closing ConnectionManager");
 | 
	
		
			
				|  |  | +        if (closing.compareAndSet(false, true)) {
 | 
	
		
			
				|  |  | +            connectingRefCounter.decRef();
 | 
	
		
			
				|  |  |              try {
 | 
	
		
			
				|  |  | -                // we are holding a write lock so nobody adds to the connectedNodes / openConnections map - it's safe to first close
 | 
	
		
			
				|  |  | -                // all instances and then clear them maps
 | 
	
		
			
				|  |  | -                Iterator<Map.Entry<DiscoveryNode, Transport.Connection>> iterator = connectedNodes.entrySet().iterator();
 | 
	
		
			
				|  |  | -                while (iterator.hasNext()) {
 | 
	
		
			
				|  |  | -                    Map.Entry<DiscoveryNode, Transport.Connection> next = iterator.next();
 | 
	
		
			
				|  |  | -                    try {
 | 
	
		
			
				|  |  | -                        IOUtils.closeWhileHandlingException(next.getValue());
 | 
	
		
			
				|  |  | -                    } finally {
 | 
	
		
			
				|  |  | -                        iterator.remove();
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | +                closeLatch.await();
 | 
	
		
			
				|  |  | +            } catch (InterruptedException e) {
 | 
	
		
			
				|  |  | +                Thread.currentThread().interrupt();
 | 
	
		
			
				|  |  | +                throw new IllegalStateException(e);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +            Iterator<Map.Entry<DiscoveryNode, Transport.Connection>> iterator = connectedNodes.entrySet().iterator();
 | 
	
		
			
				|  |  | +            while (iterator.hasNext()) {
 | 
	
		
			
				|  |  | +                Map.Entry<DiscoveryNode, Transport.Connection> next = iterator.next();
 | 
	
		
			
				|  |  | +                try {
 | 
	
		
			
				|  |  | +                    IOUtils.closeWhileHandlingException(next.getValue());
 | 
	
		
			
				|  |  | +                } finally {
 | 
	
		
			
				|  |  | +                    iterator.remove();
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  | -            } finally {
 | 
	
		
			
				|  |  | -                closeLock.writeLock().unlock();
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private Transport.Connection internalOpenConnection(DiscoveryNode node, ConnectionProfile connectionProfile) {
 | 
	
		
			
				|  |  | -        PlainActionFuture<Transport.Connection> future = PlainActionFuture.newFuture();
 | 
	
		
			
				|  |  | -        Releasable pendingConnection = transport.openConnection(node, connectionProfile, future);
 | 
	
		
			
				|  |  | -        Transport.Connection connection;
 | 
	
		
			
				|  |  | -        try {
 | 
	
		
			
				|  |  | -            connection = future.actionGet();
 | 
	
		
			
				|  |  | -        } catch (IllegalStateException e) {
 | 
	
		
			
				|  |  | -            // If the future was interrupted we must cancel the pending connection to avoid channels leaking
 | 
	
		
			
				|  |  | -            if (e.getCause() instanceof InterruptedException) {
 | 
	
		
			
				|  |  | -                pendingConnection.close();
 | 
	
		
			
				|  |  | +    private void internalOpenConnection(DiscoveryNode node, ConnectionProfile connectionProfile,
 | 
	
		
			
				|  |  | +                                        ActionListener<Transport.Connection> listener) {
 | 
	
		
			
				|  |  | +        transport.openConnection(node, connectionProfile, ActionListener.map(listener, connection -> {
 | 
	
		
			
				|  |  | +            assert Transports.assertNotTransportThread("internalOpenConnection success");
 | 
	
		
			
				|  |  | +            try {
 | 
	
		
			
				|  |  | +                connectionListener.onConnectionOpened(connection);
 | 
	
		
			
				|  |  | +            } finally {
 | 
	
		
			
				|  |  | +                connection.addCloseListener(ActionListener.wrap(() -> connectionListener.onConnectionClosed(connection)));
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | -            throw e;
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -        try {
 | 
	
		
			
				|  |  | -            connectionListener.onConnectionOpened(connection);
 | 
	
		
			
				|  |  | -        } finally {
 | 
	
		
			
				|  |  | -            connection.addCloseListener(ActionListener.wrap(() -> connectionListener.onConnectionClosed(connection)));
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -        if (connection.isClosed()) {
 | 
	
		
			
				|  |  | -            throw new ConnectTransportException(node, "a channel closed while connecting");
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -        return connection;
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    private void ensureOpen() {
 | 
	
		
			
				|  |  | -        if (isClosed.get()) {
 | 
	
		
			
				|  |  | -            throw new IllegalStateException("connection manager is closed");
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | +            if (connection.isClosed()) {
 | 
	
		
			
				|  |  | +                throw new ConnectTransportException(node, "a channel closed while connecting");
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +            return connection;
 | 
	
		
			
				|  |  | +        }));
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      ConnectionProfile getConnectionProfile() {
 |