From b26a55fcaa21b93f411734aee785bdde538bd6f3 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 29 Oct 2018 12:12:54 -0600
Subject: [PATCH 01/24] WIP

---
 .../elasticsearch/transport/TcpTransport.java | 227 ++++++++++--------
 1 file changed, 133 insertions(+), 94 deletions(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index d7d57beda26f8..32e835e8ecd82 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -97,6 +97,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -432,77 +433,112 @@ public NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connect
         if (node == null) {
             throw new ConnectTransportException(null, "can't open connection to a null node");
         }
-        boolean success = false;
-        NodeChannels nodeChannels = null;
         connectionProfile = maybeOverrideConnectionProfile(connectionProfile);
         closeLock.readLock().lock(); // ensure we don't open connections while we are closing
         try {
             ensureOpen();
-            try {
-                int numConnections = connectionProfile.getNumConnections();
-                assert numConnections > 0 : "A connection profile must be configured with at least one connection";
-                List<TcpChannel> channels = new ArrayList<>(numConnections);
-                List<ActionFuture<Void>> connectionFutures = new ArrayList<>(numConnections);
-                for (int i = 0; i < numConnections; ++i) {
-                    try {
-                        PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
-                        connectionFutures.add(connectFuture);
-                        TcpChannel channel = initiateChannel(node, connectFuture);
-                        logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
-                        channels.add(channel);
-                    } catch (Exception e) {
-                        // If there was an exception when attempting to instantiate the raw channels, we close all of the channels
-                        CloseableChannel.closeChannels(channels, false);
-                        throw e;
+            PlainActionFuture<NodeChannels> connectionFuture = PlainActionFuture.newFuture();
+            initiateConnection(node, connectionProfile, connectionFuture);
+            // TODO: This will not return the correct exceptions currently
+            return connectionFuture.actionGet();
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    private void initiateConnection(DiscoveryNode node, ConnectionProfile connectionProfile, ActionListener<NodeChannels> listener) {
+        boolean success = false;
+        NodeChannels nodeChannels = null;
+        int numConnections = connectionProfile.getNumConnections();
+        assert numConnections > 0 : "A connection profile must be configured with at least one connection";
+
+
+        try {
+            List<TcpChannel> channels = new ArrayList<>(numConnections);
+            ActionListener<Void> channelsConnectedListener = new ActionListener<Void>() {
+
+                private AtomicInteger pendingConnections = new AtomicInteger(numConnections);
+
+                @Override
+                public void onResponse(Void aVoid) {
+                    if (pendingConnections.decrementAndGet() == 0) {
+                        final TcpChannel handshakeChannel = channels.get(0);
+                        handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
+                        try {
+                            // TODO: This is currently synchronous
+                            Version version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
+                        } catch (Exception ex) {
+                            CloseableChannel.closeChannels(channels, false);
+                            listener.onFailure(ex);
+                        }
                     }
                 }
 
-                // If we make it past the block above, we successfully instantiated all of the channels
-                try {
-                    TcpChannel.awaitConnected(node, connectionFutures, connectionProfile.getConnectTimeout());
-                } catch (Exception ex) {
-                    CloseableChannel.closeChannels(channels, false);
-                    throw ex;
-                }
+                @Override
+                public void onFailure(Exception e) {
 
-                // If we make it past the block above, we have successfully established connections for all of the channels
-                final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile
-                handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
-                Version version;
+                }
+            };
+            List<ActionFuture<Void>> connectionFutures = new ArrayList<>(numConnections);
+            for (int i = 0; i < numConnections; ++i) {
                 try {
-                    version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
+                    PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
+                    connectionFutures.add(connectFuture);
+                    TcpChannel channel = initiateChannel(node, connectFuture);
+                    logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
+                    channels.add(channel);
                 } catch (Exception ex) {
+                    // If there was an exception when attempting to instantiate the raw channels, we close all of the channels
                     CloseableChannel.closeChannels(channels, false);
-                    throw ex;
+                    listener.onFailure(ex);
                 }
+            }
 
-                // If we make it past the block above, we have successfully completed the handshake and the connection is now open.
-                // At this point we should construct the connection, notify the transport service, and attach close listeners to the
-                // underlying channels.
-                nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
-                final NodeChannels finalNodeChannels = nodeChannels;
-
-                Consumer<TcpChannel> onClose = c -> {
-                    assert c.isOpen() == false : "channel is still open when onClose is called";
-                    finalNodeChannels.close();
-                };
+            // If we make it past the block above, we successfully instantiated all of the channels
+            try {
+                TcpChannel.awaitConnected(node, connectionFutures, connectionProfile.getConnectTimeout());
+            } catch (Exception ex) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(ex);
+                return;
+            }
 
-                nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(() -> onClose.accept(ch))));
-                success = true;
-                return nodeChannels;
-            } catch (ConnectTransportException e) {
-                throw e;
-            } catch (Exception e) {
-                // ConnectTransportExceptions are handled specifically on the caller end - we wrap the actual exception to ensure
-                // only relevant exceptions are logged on the caller end.. this is the same as in connectToNode
-                throw new ConnectTransportException(node, "general node connection failure", e);
-            } finally {
-                if (success == false) {
-                    IOUtils.closeWhileHandlingException(nodeChannels);
-                }
+            // If we make it past the block above, we have successfully established connections for all of the channels
+            final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile
+            handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
+            Version version;
+            try {
+                version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
+            } catch (Exception ex) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(ex);
+                return;
             }
+
+            // If we make it past the block above, we have successfully completed the handshake and the connection is now open.
+            // At this point we should construct the connection, notify the transport service, and attach close listeners to the
+            // underlying channels.
+            nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
+            final NodeChannels finalNodeChannels = nodeChannels;
+
+            Consumer<TcpChannel> onClose = c -> {
+                assert c.isOpen() == false : "channel is still open when onClose is called";
+                finalNodeChannels.close();
+            };
+
+            nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(() -> onClose.accept(ch))));
+            success = true;
+            listener.onResponse(nodeChannels);
+        } catch (ConnectTransportException ex) {
+            listener.onFailure(ex);
+        } catch (Exception ex) {
+            // ConnectTransportExceptions are handled specifically on the caller end - we wrap the actual exception to ensure
+            // only relevant exceptions are logged on the caller end.. this is the same as in connectToNode
+            listener.onFailure(new ConnectTransportException(node, "general node connection failure", ex));
         } finally {
-            closeLock.readLock().unlock();
+            if (success == false) {
+                IOUtils.closeWhileHandlingException(nodeChannels);
+            }
         }
     }
 
@@ -676,7 +712,9 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi
     // not perfect, but PortsRange should take care of any port range validation, not a regex
     private static final Pattern BRACKET_PATTERN = Pattern.compile("^\\[(.*:.*)\\](?::([\\d\\-]*))?$");
 
-    /** parse a hostname+port range spec into its equivalent addresses */
+    /**
+     * parse a hostname+port range spec into its equivalent addresses
+     */
     static TransportAddress[] parse(String hostPortString, String defaultPortRange, int perAddressLimit) throws UnknownHostException {
         Objects.requireNonNull(hostPortString);
         String host;
@@ -739,7 +777,8 @@ protected final void doStop() {
                 for (Map.Entry<String, List<TcpServerChannel>> entry : serverChannels.entrySet()) {
                     String profile = entry.getKey();
                     List<TcpServerChannel> channels = entry.getValue();
-                    ActionListener<Void> closeFailLogger = ActionListener.wrap(c -> {},
+                    ActionListener<Void> closeFailLogger = ActionListener.wrap(c -> {
+                        },
                         e -> logger.warn(() -> new ParameterizedMessage("Error closing serverChannel for profile [{}]", profile), e));
                     channels.forEach(c -> c.addCloseListener(closeFailLogger));
                     CloseableChannel.closeChannels(channels, true);
@@ -774,7 +813,7 @@ public void onException(TcpChannel channel, Exception e) {
 
         if (isCloseConnectionException(e)) {
             logger.trace(() -> new ParameterizedMessage(
-                    "close connection exception caught on transport layer [{}], disconnecting from relevant node", channel), e);
+                "close connection exception caught on transport layer [{}], disconnecting from relevant node", channel), e);
             // close the channel, which will cause a node to be disconnected if relevant
             CloseableChannel.closeChannel(channel);
         } else if (isConnectException(e)) {
@@ -787,7 +826,7 @@ public void onException(TcpChannel channel, Exception e) {
             CloseableChannel.closeChannel(channel);
         } else if (e instanceof CancelledKeyException) {
             logger.trace(() -> new ParameterizedMessage(
-                    "cancelled key exception caught on transport layer [{}], disconnecting from relevant node", channel), e);
+                "cancelled key exception caught on transport layer [{}], disconnecting from relevant node", channel), e);
             // close the channel as safe measure, which will cause a node to be disconnected if relevant
             CloseableChannel.closeChannel(channel);
         } else if (e instanceof TcpTransport.HttpOnTransportException) {
@@ -854,7 +893,7 @@ protected void serverAcceptedChannel(TcpChannel channel) {
     /**
      * Initiate a single tcp socket channel.
      *
-     * @param node for the initiated connection
+     * @param node            for the initiated connection
      * @param connectListener listener to be called when connection complete
      * @return the pending connection
      * @throws IOException if an I/O exception occurs while opening the channel
@@ -940,12 +979,12 @@ private void internalSendMessage(TcpChannel channel, BytesReference message, Sen
      * @param action      the action this response replies to
      */
     public void sendErrorResponse(
-            final Version nodeVersion,
-            final Set<String> features,
-            final TcpChannel channel,
-            final Exception error,
-            final long requestId,
-            final String action) throws IOException {
+        final Version nodeVersion,
+        final Set<String> features,
+        final TcpChannel channel,
+        final Exception error,
+        final long requestId,
+        final String action) throws IOException {
         try (BytesStreamOutput stream = new BytesStreamOutput()) {
             stream.setVersion(nodeVersion);
             stream.setFeatures(features);
@@ -971,25 +1010,25 @@ public void sendErrorResponse(
      * @see #sendErrorResponse(Version, Set, TcpChannel, Exception, long, String) for sending back errors to the caller
      */
     public void sendResponse(
-            final Version nodeVersion,
-            final Set<String> features,
-            final TcpChannel channel,
-            final TransportResponse response,
-            final long requestId,
-            final String action,
-            final TransportResponseOptions options) throws IOException {
+        final Version nodeVersion,
+        final Set<String> features,
+        final TcpChannel channel,
+        final TransportResponse response,
+        final long requestId,
+        final String action,
+        final TransportResponseOptions options) throws IOException {
         sendResponse(nodeVersion, features, channel, response, requestId, action, options, (byte) 0);
     }
 
     private void sendResponse(
-            final Version nodeVersion,
-            final Set<String> features,
-            final TcpChannel channel,
-            final TransportResponse response,
-            final long requestId,
-            final String action,
-            TransportResponseOptions options,
-            byte status) throws IOException {
+        final Version nodeVersion,
+        final Set<String> features,
+        final TcpChannel channel,
+        final TransportResponse response,
+        final long requestId,
+        final String action,
+        TransportResponseOptions options,
+        byte status) throws IOException {
         if (compress) {
             options = TransportResponseOptions.builder(options).withCompress(true).build();
         }
@@ -1086,13 +1125,13 @@ public void inboundMessage(TcpChannel channel, BytesReference message) {
      * Consumes bytes that are available from network reads. This method returns the number of bytes consumed
      * in this call.
      *
-     * @param channel the channel read from
+     * @param channel        the channel read from
      * @param bytesReference the bytes available to consume
      * @return the number of bytes consumed
-     * @throws StreamCorruptedException if the message header format is not recognized
+     * @throws StreamCorruptedException              if the message header format is not recognized
      * @throws TcpTransport.HttpOnTransportException if the message header appears to be an HTTP message
-     * @throws IllegalArgumentException if the message length is greater that the maximum allowed frame size.
-     *                                  This is dependent on the available memory.
+     * @throws IllegalArgumentException              if the message length is greater that the maximum allowed frame size.
+     *                                               This is dependent on the available memory.
      */
     public int consumeNetworkReads(TcpChannel channel, BytesReference bytesReference) throws IOException {
         BytesReference message = decodeFrame(bytesReference);
@@ -1111,10 +1150,10 @@ public int consumeNetworkReads(TcpChannel channel, BytesReference bytesReference
      *
      * @param networkBytes the will be read
      * @return the message decoded
-     * @throws StreamCorruptedException if the message header format is not recognized
+     * @throws StreamCorruptedException              if the message header format is not recognized
      * @throws TcpTransport.HttpOnTransportException if the message header appears to be an HTTP message
-     * @throws IllegalArgumentException if the message length is greater that the maximum allowed frame size.
-     *                                  This is dependent on the available memory.
+     * @throws IllegalArgumentException              if the message length is greater that the maximum allowed frame size.
+     *                                               This is dependent on the available memory.
      */
     static BytesReference decodeFrame(BytesReference networkBytes) throws IOException {
         int messageLength = readMessageLength(networkBytes);
@@ -1138,10 +1177,10 @@ static BytesReference decodeFrame(BytesReference networkBytes) throws IOExceptio
      *
      * @param networkBytes the will be read
      * @return the length of the message
-     * @throws StreamCorruptedException if the message header format is not recognized
+     * @throws StreamCorruptedException              if the message header format is not recognized
      * @throws TcpTransport.HttpOnTransportException if the message header appears to be an HTTP message
-     * @throws IllegalArgumentException if the message length is greater that the maximum allowed frame size.
-     *                                  This is dependent on the available memory.
+     * @throws IllegalArgumentException              if the message length is greater that the maximum allowed frame size.
+     *                                               This is dependent on the available memory.
      */
     public static int readMessageLength(BytesReference networkBytes) throws IOException {
         if (networkBytes.length() < BYTES_NEEDED_FOR_MESSAGE_SIZE) {
@@ -1324,7 +1363,7 @@ static void ensureVersionCompatibility(Version version, Version currentVersion,
     }
 
     private <T extends TransportResponse> void handleResponse(InetSocketAddress remoteAddress, final StreamInput stream,
-                                final TransportResponseHandler<T> handler) {
+                                                              final TransportResponseHandler<T> handler) {
         final T response;
         try {
             response = handler.read(stream);
@@ -1414,7 +1453,7 @@ protected String handleRequest(TcpChannel channel, String profileName, final Str
             // the circuit breaker tripped
             if (transportChannel == null) {
                 transportChannel =
-                        new TcpTransportChannel(this, channel, transportName, action, requestId, version, features, profileName, 0);
+                    new TcpTransportChannel(this, channel, transportName, action, requestId, version, features, profileName, 0);
             }
             try {
                 transportChannel.sendResponse(e);
@@ -1467,7 +1506,7 @@ public void onFailure(Exception e) {
                 } catch (Exception inner) {
                     inner.addSuppressed(e);
                     logger.warn(() -> new ParameterizedMessage(
-                            "Failed to send error message back to client for action [{}]", reg.getAction()), inner);
+                        "Failed to send error message back to client for action [{}]", reg.getAction()), inner);
                 }
             }
         }

From 725d10e5ef611a54eeb9bafb14687c3d2e489b87 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 29 Oct 2018 17:00:43 -0600
Subject: [PATCH 02/24] WIP

---
 .../netty4/SimpleNetty4TransportTests.java    |   8 +-
 .../nio/SimpleNioTransportTests.java          |   8 +-
 .../elasticsearch/transport/TcpTransport.java | 125 ++++++++++++------
 .../AbstractSimpleTransportTestCase.java      |  11 +-
 .../transport/MockTcpTransportTests.java      |   9 +-
 .../nio/SimpleMockNioTransportTests.java      |   8 +-
 ...stractSimpleSecurityTransportTestCase.java |   8 +-
 ...pleSecurityNetty4ServerTransportTests.java |   8 +-
 .../nio/SimpleSecurityNioTransportTests.java  |   8 +-
 9 files changed, 122 insertions(+), 71 deletions(-)

diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
index e7faac8ae01db..7b8dce9c39111 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
@@ -20,6 +20,7 @@
 package org.elasticsearch.transport.netty4;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -58,12 +59,11 @@ public static MockTransportService nettyFromThreadPool(Settings settings, Thread
             BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
-            public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException,
-                InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
                 if (doHandshake) {
-                    return super.executeHandshake(node, channel, timeout);
+                    super.asyncHandshake(node, channel, timeout, listener);
                 } else {
-                    return version.minimumCompatibilityVersion();
+                    listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
 
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
index 33d40b9f735fa..d4515d511821b 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
@@ -20,6 +20,7 @@
 package org.elasticsearch.transport.nio;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -62,12 +63,11 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException,
-                InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
                 if (doHandshake) {
-                    return super.executeHandshake(node, channel, timeout);
+                    super.asyncHandshake(node, channel, timeout, listener);
                 } else {
-                    return version.minimumCompatibilityVersion();
+                    listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
 
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 32e835e8ecd82..ef945532789d9 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -201,7 +201,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
     private volatile BoundTransportAddress boundAddress;
     private final String transportName;
 
-    private final ConcurrentMap<Long, HandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Long, AsyncHandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
     private final CounterMetric numHandshakes = new CounterMetric();
     private static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
 
@@ -277,14 +277,15 @@ public synchronized <Request extends TransportRequest> void registerRequestHandl
         requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
     }
 
-    private static class HandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
-        final AtomicReference<Version> versionRef = new AtomicReference<>();
-        final CountDownLatch latch = new CountDownLatch(1);
-        final AtomicReference<Exception> exceptionRef = new AtomicReference<>();
+    private static class AsyncHandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
+
         final TcpChannel channel;
+        private final ActionListener<Version> listener;
+        private final AtomicBoolean isDone = new AtomicBoolean(false);
 
-        HandshakeResponseHandler(TcpChannel channel) {
+        AsyncHandshakeResponseHandler(TcpChannel channel, ActionListener<Version> listener) {
             this.channel = channel;
+            this.listener = listener;
         }
 
         @Override
@@ -294,16 +295,16 @@ public VersionHandshakeResponse read(StreamInput in) throws IOException {
 
         @Override
         public void handleResponse(VersionHandshakeResponse response) {
-            final boolean success = versionRef.compareAndSet(null, response.version);
-            latch.countDown();
-            assert success;
+            if (isDone.compareAndSet(false, true)) {
+                listener.onResponse(response.version);
+            }
         }
 
         @Override
         public void handleException(TransportException exp) {
-            final boolean success = exceptionRef.compareAndSet(null, exp);
-            latch.countDown();
-            assert success;
+            if (isDone.compareAndSet(false, true)) {
+                listener.onFailure(exp);
+            }
         }
 
         @Override
@@ -466,7 +467,7 @@ public void onResponse(Void aVoid) {
                         handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
                         try {
                             // TODO: This is currently synchronous
-                            Version version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
+                            asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), null);
                         } catch (Exception ex) {
                             CloseableChannel.closeChannels(channels, false);
                             listener.onFailure(ex);
@@ -506,14 +507,14 @@ public void onFailure(Exception e) {
             // If we make it past the block above, we have successfully established connections for all of the channels
             final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile
             handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
-            Version version;
-            try {
-                version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
-            } catch (Exception ex) {
-                CloseableChannel.closeChannels(channels, false);
-                listener.onFailure(ex);
-                return;
-            }
+            Version version = null;
+//            try {
+//                version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
+//            } catch (Exception ex) {
+//                CloseableChannel.closeChannels(channels, false);
+//                listener.onFailure(ex);
+//                return;
+//            }
 
             // If we make it past the block above, we have successfully completed the handshake and the connection is now open.
             // At this point we should construct the connection, notify the transport service, and attach close listeners to the
@@ -1532,13 +1533,25 @@ public void writeTo(StreamOutput out) throws IOException {
         }
     }
 
-    public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout)
-        throws IOException, InterruptedException {
+    public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
         numHandshakes.inc();
         final long requestId = responseHandlers.newRequestId();
-        final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel);
-        AtomicReference<Version> versionRef = handler.versionRef;
-        AtomicReference<Exception> exceptionRef = handler.exceptionRef;
+        final AsyncHandshakeResponseHandler handler = new AsyncHandshakeResponseHandler(channel, new ActionListener<Version>() {
+            @Override
+            public void onResponse(Version version) {
+                if (getCurrentVersion().isCompatible(version) == false) {
+                    listener.onFailure(new IllegalStateException("Received message from unsupported version: [" + version
+                        + "] minimal compatible version is: [" + getCurrentVersion().minimumCompatibilityVersion() + "]"));
+                } else {
+                    listener.onResponse(version);
+                }
+            }
+
+            @Override
+            public void onFailure(Exception e) {
+                listener.onFailure(new IllegalStateException("handshake failed", e));
+            }
+        });
         pendingHandshakes.put(requestId, handler);
         boolean success = false;
         try {
@@ -1555,20 +1568,10 @@ public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValu
             final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
             sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
                 TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
-            if (handler.latch.await(timeout.millis(), TimeUnit.MILLISECONDS) == false) {
+            if (false) {
                 throw new ConnectTransportException(node, "handshake_timeout[" + timeout + "]");
             }
             success = true;
-            if (exceptionRef.get() != null) {
-                throw new IllegalStateException("handshake failed", exceptionRef.get());
-            } else {
-                Version version = versionRef.get();
-                if (getCurrentVersion().isCompatible(version) == false) {
-                    throw new IllegalStateException("Received message from unsupported version: [" + version
-                        + "] minimal compatible version is: [" + getCurrentVersion().minimumCompatibilityVersion() + "]");
-                }
-                return version;
-            }
         } finally {
             final TransportResponseHandler<?> removedHandler = pendingHandshakes.remove(requestId);
             // in the case of a timeout or an exception on the send part the handshake has not been removed yet.
@@ -1577,6 +1580,52 @@ public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValu
         }
     }
 
+    public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout)
+        throws IOException, InterruptedException {
+        return null;
+//        numHandshakes.inc();
+//        final long requestId = responseHandlers.newRequestId();
+//        final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel);
+//        AtomicReference<Version> versionRef = handler.versionRef;
+//        AtomicReference<Exception> exceptionRef = handler.exceptionRef;
+//        pendingHandshakes.put(requestId, handler);
+//        boolean success = false;
+//        try {
+//            if (channel.isOpen() == false) {
+//                // we have to protect us here since sendRequestToChannel won't barf if the channel is closed.
+//                // it's weird but to change it will cause a lot of impact on the exception handling code all over the codebase.
+//                // yet, if we don't check the state here we might have registered a pending handshake handler but the close
+//                // listener calling #onChannelClosed might have already run and we are waiting on the latch below unitl we time out.
+//                throw new IllegalStateException("handshake failed, channel already closed");
+//            }
+//            // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
+//            // we also have no payload on the request but the response will contain the actual version of the node we talk
+//            // to as the payload.
+//            final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
+//            sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
+//                TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
+//            if (handler.latch.await(timeout.millis(), TimeUnit.MILLISECONDS) == false) {
+//                throw new ConnectTransportException(node, "handshake_timeout[" + timeout + "]");
+//            }
+//            success = true;
+//            if (exceptionRef.get() != null) {
+//                throw new IllegalStateException("handshake failed", exceptionRef.get());
+//            } else {
+//                Version version = versionRef.get();
+//                if (getCurrentVersion().isCompatible(version) == false) {
+//                    throw new IllegalStateException("Received message from unsupported version: [" + version
+//                        + "] minimal compatible version is: [" + getCurrentVersion().minimumCompatibilityVersion() + "]");
+//                }
+//                return version;
+//            }
+//        } finally {
+//            final TransportResponseHandler<?> removedHandler = pendingHandshakes.remove(requestId);
+//            // in the case of a timeout or an exception on the send part the handshake has not been removed yet.
+//            // but the timeout is tricky since it's basically a race condition so we only assert on the success case.
+//            assert success && removedHandler == null || success == false : "handler for requestId [" + requestId + "] is not been removed";
+//        }
+    }
+
     final int getNumPendingHandshakes() { // for testing
         return pendingHandshakes.size();
     }
@@ -1593,7 +1642,7 @@ private void cancelHandshakeForChannel(TcpChannel channel) {
             .filter((entry) -> entry.getValue().channel == channel).map(Map.Entry::getKey).findFirst();
         if (first.isPresent()) {
             final Long requestId = first.get();
-            final HandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
+            final TransportResponseHandler<?> handler = pendingHandshakes.remove(requestId);
             if (handler != null) {
                 // there might be a race removing this or this method might be called twice concurrently depending on how
                 // the channel is closed ie. due to connection reset or broken pipes
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
index f4cf6e09642de..31acd2446ec80 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
@@ -199,9 +199,7 @@ public void tearDown() throws Exception {
             assertNoPendingHandshakes(serviceA.getOriginalTransport());
             assertNoPendingHandshakes(serviceB.getOriginalTransport());
         } finally {
-            IOUtils.close(serviceA, serviceB, () -> {
-                terminate(threadPool);
-            });
+            IOUtils.close(serviceA, serviceB, () -> terminate(threadPool));
         }
     }
 
@@ -2030,9 +2028,10 @@ protected String handleRequest(TcpChannel mockChannel, String profileName, Strea
              TcpTransport.NodeChannels connection = originalTransport.openConnection(
                  new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0),
                  connectionProfile)) {
-            Version version = originalTransport.executeHandshake(connection.getNode(),
-                connection.channel(TransportRequestOptions.Type.PING), TimeValue.timeValueSeconds(10));
-            assertEquals(version, Version.CURRENT);
+            PlainActionFuture<Version> listener = PlainActionFuture.newFuture();
+            originalTransport.asyncHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
+                TimeValue.timeValueSeconds(10), listener);
+            assertEquals(listener.actionGet(), Version.CURRENT);
         }
     }
 
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
index e8b5f38b88df1..8aa543a097464 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
@@ -19,6 +19,7 @@
 package org.elasticsearch.transport;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -39,13 +40,13 @@ protected MockTransportService build(Settings settings, Version version, Cluster
         NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
         Transport transport = new MockTcpTransport(settings, threadPool, BigArrays.NON_RECYCLING_INSTANCE,
             new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()), version) {
+
             @Override
-            public Version executeHandshake(DiscoveryNode node, TcpChannel mockChannel, TimeValue timeout) throws IOException,
-                InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
                 if (doHandshake) {
-                    return super.executeHandshake(node, mockChannel, timeout);
+                    super.asyncHandshake(node, channel, timeout, listener);
                 } else {
-                    return version.minimumCompatibilityVersion();
+                    listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
         };
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
index 10f089e855a5d..98c6faf65c6a7 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
@@ -20,6 +20,7 @@
 package org.elasticsearch.transport.nio;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -62,12 +63,11 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException,
-                InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
                 if (doHandshake) {
-                    return super.executeHandshake(node, channel, timeout);
+                    super.asyncHandshake(node, channel, timeout, listener);
                 } else {
-                    return version.minimumCompatibilityVersion();
+                    listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
 
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
index 077edf22c91ca..2d372bc085f5a 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
@@ -7,6 +7,7 @@
 
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.elasticsearch.Version;
+import org.elasticsearch.action.support.PlainActionFuture;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.SuppressForbidden;
 import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
@@ -115,9 +116,10 @@ public void testTcpHandshake() throws IOException, InterruptedException {
              TcpTransport.NodeChannels connection = originalTransport.openConnection(
                  new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0),
                  connectionProfile)) {
-            Version version = originalTransport.executeHandshake(connection.getNode(),
-                connection.channel(TransportRequestOptions.Type.PING), TimeValue.timeValueSeconds(10));
-            assertEquals(version, Version.CURRENT);
+            PlainActionFuture<Version> listener = PlainActionFuture.newFuture();
+            originalTransport.asyncHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
+                TimeValue.timeValueSeconds(10), listener);
+            assertEquals(listener.actionGet(), Version.CURRENT);
         }
     }
 
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
index 291b39f4b05ba..e5190a5f0eb27 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
@@ -13,6 +13,7 @@
 import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.handler.ssl.SslHandler;
 import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -77,12 +78,11 @@ public MockTransportService nettyFromThreadPool(Settings settings, ThreadPool th
             new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException,
-                InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
                 if (doHandshake) {
-                    return super.executeHandshake(node, channel, timeout);
+                    super.asyncHandshake(node, channel, timeout, listener);
                 } else {
-                    return version.minimumCompatibilityVersion();
+                    listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
 
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
index 7fd4d8b5e0319..8db487c17d82a 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
@@ -6,6 +6,7 @@
 package org.elasticsearch.xpack.security.transport.nio;
 
 import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -39,12 +40,11 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
                 new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException,
-                    InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
                 if (doHandshake) {
-                    return super.executeHandshake(node, channel, timeout);
+                    super.asyncHandshake(node, channel, timeout, listener);
                 } else {
-                    return version.minimumCompatibilityVersion();
+                    listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
 

From bd536b4b518c58bf63850c091901e17f6ba983d9 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 29 Oct 2018 21:09:26 -0600
Subject: [PATCH 03/24] WIP

---
 .../netty4/SimpleNetty4TransportTests.java    |  2 +-
 .../nio/SimpleNioTransportTests.java          |  2 +-
 .../elasticsearch/transport/TcpTransport.java | 37 ++++++++-----------
 .../transport/MockTcpTransportTests.java      |  2 +-
 .../nio/SimpleMockNioTransportTests.java      |  2 +-
 ...pleSecurityNetty4ServerTransportTests.java |  2 +-
 .../nio/SimpleSecurityNioTransportTests.java  |  2 +-
 7 files changed, 21 insertions(+), 28 deletions(-)

diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
index 7b8dce9c39111..c78f2af8e78c3 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
@@ -59,7 +59,7 @@ public static MockTransportService nettyFromThreadPool(Settings settings, Thread
             BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
index d4515d511821b..ac9363339ffe1 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
@@ -63,7 +63,7 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index ef945532789d9..11d59359c488d 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -277,14 +277,16 @@ public synchronized <Request extends TransportRequest> void registerRequestHandl
         requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
     }
 
-    private static class AsyncHandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
+    private class AsyncHandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
 
-        final TcpChannel channel;
+        private final TcpChannel channel;
         private final ActionListener<Version> listener;
+        private final Version currentVersion;
         private final AtomicBoolean isDone = new AtomicBoolean(false);
 
-        AsyncHandshakeResponseHandler(TcpChannel channel, ActionListener<Version> listener) {
+        private AsyncHandshakeResponseHandler(TcpChannel channel, Version currentVersion, ActionListener<Version> listener) {
             this.channel = channel;
+            this.currentVersion = currentVersion;
             this.listener = listener;
         }
 
@@ -296,14 +298,20 @@ public VersionHandshakeResponse read(StreamInput in) throws IOException {
         @Override
         public void handleResponse(VersionHandshakeResponse response) {
             if (isDone.compareAndSet(false, true)) {
-                listener.onResponse(response.version);
+                Version version = response.version;
+                if (currentVersion.isCompatible(version) == false) {
+                    listener.onFailure(new IllegalStateException("Received message from unsupported version: [" + version
+                        + "] minimal compatible version is: [" + currentVersion.minimumCompatibilityVersion() + "]"));
+                } else {
+                    listener.onResponse(version);
+                }
             }
         }
 
         @Override
-        public void handleException(TransportException exp) {
+        public void handleException(TransportException e) {
             if (isDone.compareAndSet(false, true)) {
-                listener.onFailure(exp);
+                listener.onFailure(new IllegalStateException("handshake failed", e));
             }
         }
 
@@ -1536,22 +1544,7 @@ public void writeTo(StreamOutput out) throws IOException {
     public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
         numHandshakes.inc();
         final long requestId = responseHandlers.newRequestId();
-        final AsyncHandshakeResponseHandler handler = new AsyncHandshakeResponseHandler(channel, new ActionListener<Version>() {
-            @Override
-            public void onResponse(Version version) {
-                if (getCurrentVersion().isCompatible(version) == false) {
-                    listener.onFailure(new IllegalStateException("Received message from unsupported version: [" + version
-                        + "] minimal compatible version is: [" + getCurrentVersion().minimumCompatibilityVersion() + "]"));
-                } else {
-                    listener.onResponse(version);
-                }
-            }
-
-            @Override
-            public void onFailure(Exception e) {
-                listener.onFailure(new IllegalStateException("handshake failed", e));
-            }
-        });
+        final AsyncHandshakeResponseHandler handler = new AsyncHandshakeResponseHandler(channel, getCurrentVersion(), listener);
         pendingHandshakes.put(requestId, handler);
         boolean success = false;
         try {
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
index 8aa543a097464..bbea613e1135f 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
@@ -42,7 +42,7 @@ protected MockTransportService build(Settings settings, Version version, Cluster
             new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()), version) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
index 98c6faf65c6a7..74e96040d9f22 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
@@ -63,7 +63,7 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
index e5190a5f0eb27..a2ea9c56b1466 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
@@ -78,7 +78,7 @@ public MockTransportService nettyFromThreadPool(Settings settings, ThreadPool th
             new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
index 8db487c17d82a..e6f2a80ab9f36 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
@@ -40,7 +40,7 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
                 new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException, InterruptedException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {

From ccf1b1a6a6a378d650139627cb6cbd316deeb51f Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Tue, 30 Oct 2018 13:43:52 -0600
Subject: [PATCH 04/24] WIP

---
 .../elasticsearch/transport/TcpChannel.java   | 14 ++-
 .../elasticsearch/transport/TcpTransport.java | 93 +++++++++----------
 2 files changed, 57 insertions(+), 50 deletions(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
index bc5cc2c92f2cb..32dd58dca7228 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
@@ -46,7 +46,6 @@ public interface TcpChannel extends CloseableChannel {
      */
     String getProfile();
 
-
     /**
      * This sets the low level socket option {@link java.net.StandardSocketOptions} SO_LINGER on a channel.
      *
@@ -55,7 +54,6 @@ public interface TcpChannel extends CloseableChannel {
      */
     void setSoLinger(int value) throws IOException;
 
-
     /**
      * Returns the local address for this channel.
      *
@@ -79,6 +77,18 @@ public interface TcpChannel extends CloseableChannel {
      */
     void sendMessage(BytesReference reference, ActionListener<Void> listener);
 
+    /**
+     * Adds a listener that will be executed when the channel is connected. If the channel is still
+     * unconnected when this listener is added, the listener will be executed by the thread that eventually
+     * finishes the channel connection. If the channel is already connected when the listener is added the
+     * listener will immediately be executed by the thread that is attempting to add the listener.
+     *
+     * @param listener to be executed
+     */
+    default void addConnectionListener(ActionListener<Void> listener) {
+        // TODO: Implement
+    }
+
     /**
      * Awaits for all of the pending connections to complete. Will throw an exception if at least one of the
      * connections fails.
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 11d59359c488d..0f6db5841fbef 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -279,13 +279,14 @@ public synchronized <Request extends TransportRequest> void registerRequestHandl
 
     private class AsyncHandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
 
+        private final long requestId;
         private final TcpChannel channel;
         private final ActionListener<Version> listener;
         private final Version currentVersion;
-        private final AtomicBoolean isDone = new AtomicBoolean(false);
 
-        private AsyncHandshakeResponseHandler(TcpChannel channel, Version currentVersion, ActionListener<Version> listener) {
+        private AsyncHandshakeResponseHandler(TcpChannel channel, long requestId, Version currentVersion, ActionListener<Version> listener) {
             this.channel = channel;
+            this.requestId = requestId;
             this.currentVersion = currentVersion;
             this.listener = listener;
         }
@@ -297,7 +298,8 @@ public VersionHandshakeResponse read(StreamInput in) throws IOException {
 
         @Override
         public void handleResponse(VersionHandshakeResponse response) {
-            if (isDone.compareAndSet(false, true)) {
+            AsyncHandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
+            if (handler != null) {
                 Version version = response.version;
                 if (currentVersion.isCompatible(version) == false) {
                     listener.onFailure(new IllegalStateException("Received message from unsupported version: [" + version
@@ -310,7 +312,9 @@ public void handleResponse(VersionHandshakeResponse response) {
 
         @Override
         public void handleException(TransportException e) {
-            if (isDone.compareAndSet(false, true)) {
+            AsyncHandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
+            if (handler != null) {
+                // TODO: Do we really need to wrap in ise?
                 listener.onFailure(new IllegalStateException("handshake failed", e));
             }
         }
@@ -461,7 +465,6 @@ private void initiateConnection(DiscoveryNode node, ConnectionProfile connection
         int numConnections = connectionProfile.getNumConnections();
         assert numConnections > 0 : "A connection profile must be configured with at least one connection";
 
-
         try {
             List<TcpChannel> channels = new ArrayList<>(numConnections);
             ActionListener<Void> channelsConnectedListener = new ActionListener<Void>() {
@@ -469,12 +472,11 @@ private void initiateConnection(DiscoveryNode node, ConnectionProfile connection
                 private AtomicInteger pendingConnections = new AtomicInteger(numConnections);
 
                 @Override
-                public void onResponse(Void aVoid) {
+                public void onResponse(Void v) {
                     if (pendingConnections.decrementAndGet() == 0) {
                         final TcpChannel handshakeChannel = channels.get(0);
                         handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
                         try {
-                            // TODO: This is currently synchronous
                             asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), null);
                         } catch (Exception ex) {
                             CloseableChannel.closeChannels(channels, false);
@@ -484,16 +486,15 @@ public void onResponse(Void aVoid) {
                 }
 
                 @Override
-                public void onFailure(Exception e) {
-
+                public void onFailure(Exception ex) {
+                    CloseableChannel.closeChannels(channels, false);
+                    listener.onFailure(ex);
                 }
             };
-            List<ActionFuture<Void>> connectionFutures = new ArrayList<>(numConnections);
             for (int i = 0; i < numConnections; ++i) {
                 try {
-                    PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
-                    connectionFutures.add(connectFuture);
-                    TcpChannel channel = initiateChannel(node, connectFuture);
+                    TcpChannel channel = initiateChannel(node, PlainActionFuture.newFuture());
+                    channel.addConnectionListener(channelsConnectedListener);
                     logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
                     channels.add(channel);
                 } catch (Exception ex) {
@@ -503,19 +504,19 @@ public void onFailure(Exception e) {
                 }
             }
 
-            // If we make it past the block above, we successfully instantiated all of the channels
-            try {
-                TcpChannel.awaitConnected(node, connectionFutures, connectionProfile.getConnectTimeout());
-            } catch (Exception ex) {
-                CloseableChannel.closeChannels(channels, false);
-                listener.onFailure(ex);
-                return;
-            }
+//            // If we make it past the block above, we successfully instantiated all of the channels
+//            try {
+//                TcpChannel.awaitConnected(node, connectionFutures, connectionProfile.getConnectTimeout());
+//            } catch (Exception ex) {
+//                CloseableChannel.closeChannels(channels, false);
+//                listener.onFailure(ex);
+//                return;
+//            }
 
             // If we make it past the block above, we have successfully established connections for all of the channels
-            final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile
-            handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
-            Version version = null;
+//            final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile
+//            handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
+//            Version version = null;
 //            try {
 //                version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
 //            } catch (Exception ex) {
@@ -527,7 +528,7 @@ public void onFailure(Exception e) {
             // If we make it past the block above, we have successfully completed the handshake and the connection is now open.
             // At this point we should construct the connection, notify the transport service, and attach close listeners to the
             // underlying channels.
-            nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
+//            nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
             final NodeChannels finalNodeChannels = nodeChannels;
 
             Consumer<TcpChannel> onClose = c -> {
@@ -1541,35 +1542,31 @@ public void writeTo(StreamOutput out) throws IOException {
         }
     }
 
-    public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+    public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
         numHandshakes.inc();
         final long requestId = responseHandlers.newRequestId();
-        final AsyncHandshakeResponseHandler handler = new AsyncHandshakeResponseHandler(channel, getCurrentVersion(), listener);
+        final AsyncHandshakeResponseHandler handler = new AsyncHandshakeResponseHandler(channel, requestId, getCurrentVersion(), listener);
         pendingHandshakes.put(requestId, handler);
+        channel.addCloseListener(ActionListener.wrap(() -> handler.handleException(new TransportException("connection reset"))));
         boolean success = false;
         try {
-            if (channel.isOpen() == false) {
-                // we have to protect us here since sendRequestToChannel won't barf if the channel is closed.
-                // it's weird but to change it will cause a lot of impact on the exception handling code all over the codebase.
-                // yet, if we don't check the state here we might have registered a pending handshake handler but the close
-                // listener calling #onChannelClosed might have already run and we are waiting on the latch below unitl we time out.
-                throw new IllegalStateException("handshake failed, channel already closed");
-            }
-            // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
-            // we also have no payload on the request but the response will contain the actual version of the node we talk
-            // to as the payload.
-            final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
-            sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
-                TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
-            if (false) {
-                throw new ConnectTransportException(node, "handshake_timeout[" + timeout + "]");
-            }
-            success = true;
+        // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
+        // we also have no payload on the request but the response will contain the actual version of the node we talk
+        // to as the payload.
+        final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
+        sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
+            TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
+
+        threadPool.schedule(timeout, ThreadPool.Names.GENERIC,
+            () -> handler.handleException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
+        success = true;
+        } catch (Exception e) {
+            handler.handleException(new SendRequestTransportException(node, HANDSHAKE_ACTION_NAME, e));
         } finally {
-            final TransportResponseHandler<?> removedHandler = pendingHandshakes.remove(requestId);
-            // in the case of a timeout or an exception on the send part the handshake has not been removed yet.
-            // but the timeout is tricky since it's basically a race condition so we only assert on the success case.
-            assert success && removedHandler == null || success == false : "handler for requestId [" + requestId + "] is not been removed";
+            if (success == false) {
+                AsyncHandshakeResponseHandler removed = pendingHandshakes.remove(requestId);
+                assert removed == null : "Handshake should not be pending if exception was thrown";
+            }
         }
     }
 

From ad5d50320aef9a229c08e36080cf336d5942f53e Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Tue, 30 Oct 2018 15:14:47 -0600
Subject: [PATCH 05/24] WIP

---
 .../transport/netty4/Netty4TcpChannel.java    |  17 ++
 .../transport/netty4/Netty4Transport.java     |  11 +-
 .../transport/nio/NioTcpChannel.java          |   5 +
 .../nio/SimpleNioTransportTests.java          |   2 +-
 .../elasticsearch/transport/TcpChannel.java   |   2 +-
 .../elasticsearch/transport/TcpTransport.java | 185 +++++++-----------
 .../transport/TcpTransportTests.java          |   4 +
 .../transport/MockTcpTransport.java           |  12 +-
 .../transport/nio/MockNioTransport.java       |   5 +
 9 files changed, 116 insertions(+), 127 deletions(-)

diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
index bee98362e0c1e..d370c396e980d 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
@@ -37,11 +37,17 @@ public class Netty4TcpChannel implements TcpChannel {
 
     private final Channel channel;
     private final String profile;
+    private final CompletableContext<Void> connectContext;
     private final CompletableContext<Void> closeContext = new CompletableContext<>();
 
     Netty4TcpChannel(Channel channel, String profile) {
+        this(channel, profile, completedConnectContext());
+    }
+
+    Netty4TcpChannel(Channel channel, String profile, CompletableContext<Void> connectContext) {
         this.channel = channel;
         this.profile = profile;
+        this.connectContext = connectContext;
         this.channel.closeFuture().addListener(f -> {
             if (f.isSuccess()) {
                 closeContext.complete(null);
@@ -72,6 +78,11 @@ public void addCloseListener(ActionListener<Void> listener) {
         closeContext.addListener(ActionListener.toBiConsumer(listener));
     }
 
+    @Override
+    public void addConnectListener(ActionListener<Void> listener) {
+        connectContext.addListener(ActionListener.toBiConsumer(listener));
+    }
+
     @Override
     public void setSoLinger(int value) throws IOException {
         if (channel.isOpen()) {
@@ -132,4 +143,10 @@ public String toString() {
             ", remoteAddress=" + channel.remoteAddress() +
             '}';
     }
+
+    private static CompletableContext<Void> completedConnectContext() {
+        CompletableContext<Void> connectContext = new CompletableContext<>();
+        connectContext.complete(null);
+        return connectContext;
+    }
 }
diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
index 009a75b3e3301..1ace3953ebc78 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
@@ -43,6 +43,7 @@
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.SuppressForbidden;
 import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.concurrent.CompletableContext;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.network.NetworkService;
@@ -234,19 +235,21 @@ protected Netty4TcpChannel initiateChannel(DiscoveryNode node, ActionListener<Vo
         }
         addClosedExceptionLogger(channel);
 
-        Netty4TcpChannel nettyChannel = new Netty4TcpChannel(channel, "default");
+        CompletableContext<Void> connectContext = new CompletableContext<>();
+
+        Netty4TcpChannel nettyChannel = new Netty4TcpChannel(channel, "default", connectContext);
         channel.attr(CHANNEL_KEY).set(nettyChannel);
 
         channelFuture.addListener(f -> {
             if (f.isSuccess()) {
-                listener.onResponse(null);
+                connectContext.complete(null);
             } else {
                 Throwable cause = f.cause();
                 if (cause instanceof Error) {
                     ExceptionsHelper.maybeDieOnAnotherThread(cause);
-                    listener.onFailure(new Exception(cause));
+                    connectContext.completeExceptionally(new Exception(cause));
                 } else {
-                    listener.onFailure((Exception) cause);
+                    connectContext.completeExceptionally((Exception) cause);
                 }
             }
         });
diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTcpChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTcpChannel.java
index 947a255b178c8..480043acbd899 100644
--- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTcpChannel.java
+++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTcpChannel.java
@@ -58,6 +58,11 @@ public void addCloseListener(ActionListener<Void> listener) {
         addCloseListener(ActionListener.toBiConsumer(listener));
     }
 
+    @Override
+    public void addConnectListener(ActionListener<Void> listener) {
+        addConnectListener(ActionListener.toBiConsumer(listener));
+    }
+
     @Override
     public void close() {
         getContext().closeChannel();
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
index ac9363339ffe1..52b8812576983 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
@@ -63,7 +63,7 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
                     super.asyncHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
index 32dd58dca7228..8297aa731cafa 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
@@ -85,7 +85,7 @@ public interface TcpChannel extends CloseableChannel {
      *
      * @param listener to be executed
      */
-    default void addConnectionListener(ActionListener<Void> listener) {
+    default void addConnectListener(ActionListener<Void> listener) {
         // TODO: Implement
     }
 
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 0f6db5841fbef..fdcb8407cea84 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -23,7 +23,6 @@
 import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.Version;
-import org.elasticsearch.action.ActionFuture;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.NotifyOnceListener;
 import org.elasticsearch.action.support.PlainActionFuture;
@@ -88,7 +87,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
@@ -101,7 +99,6 @@
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -201,7 +198,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
     private volatile BoundTransportAddress boundAddress;
     private final String transportName;
 
-    private final ConcurrentMap<Long, AsyncHandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Long, HandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
     private final CounterMetric numHandshakes = new CounterMetric();
     private static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
 
@@ -277,14 +274,14 @@ public synchronized <Request extends TransportRequest> void registerRequestHandl
         requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
     }
 
-    private class AsyncHandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
+    private class HandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
 
         private final long requestId;
         private final TcpChannel channel;
         private final ActionListener<Version> listener;
         private final Version currentVersion;
 
-        private AsyncHandshakeResponseHandler(TcpChannel channel, long requestId, Version currentVersion, ActionListener<Version> listener) {
+        private HandshakeResponseHandler(TcpChannel channel, long requestId, Version currentVersion, ActionListener<Version> listener) {
             this.channel = channel;
             this.requestId = requestId;
             this.currentVersion = currentVersion;
@@ -298,7 +295,7 @@ public VersionHandshakeResponse read(StreamInput in) throws IOException {
 
         @Override
         public void handleResponse(VersionHandshakeResponse response) {
-            AsyncHandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
+            HandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
             if (handler != null) {
                 Version version = response.version;
                 if (currentVersion.isCompatible(version) == false) {
@@ -312,7 +309,7 @@ public void handleResponse(VersionHandshakeResponse response) {
 
         @Override
         public void handleException(TransportException e) {
-            AsyncHandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
+            HandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
             if (handler != null) {
                 // TODO: Do we really need to wrap in ise?
                 listener.onFailure(new IllegalStateException("handshake failed", e));
@@ -460,95 +457,67 @@ public NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connect
     }
 
     private void initiateConnection(DiscoveryNode node, ConnectionProfile connectionProfile, ActionListener<NodeChannels> listener) {
-        boolean success = false;
-        NodeChannels nodeChannels = null;
         int numConnections = connectionProfile.getNumConnections();
         assert numConnections > 0 : "A connection profile must be configured with at least one connection";
 
-        try {
-            List<TcpChannel> channels = new ArrayList<>(numConnections);
-            ActionListener<Void> channelsConnectedListener = new ActionListener<Void>() {
-
-                private AtomicInteger pendingConnections = new AtomicInteger(numConnections);
-
-                @Override
-                public void onResponse(Void v) {
-                    if (pendingConnections.decrementAndGet() == 0) {
-                        final TcpChannel handshakeChannel = channels.get(0);
-                        handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
-                        try {
-                            asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), null);
-                        } catch (Exception ex) {
-                            CloseableChannel.closeChannels(channels, false);
-                            listener.onFailure(ex);
-                        }
-                    }
-                }
+        final List<TcpChannel> channels = new ArrayList<>(numConnections);
 
-                @Override
-                public void onFailure(Exception ex) {
-                    CloseableChannel.closeChannels(channels, false);
-                    listener.onFailure(ex);
-                }
-            };
-            for (int i = 0; i < numConnections; ++i) {
-                try {
-                    TcpChannel channel = initiateChannel(node, PlainActionFuture.newFuture());
-                    channel.addConnectionListener(channelsConnectedListener);
-                    logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
-                    channels.add(channel);
-                } catch (Exception ex) {
-                    // If there was an exception when attempting to instantiate the raw channels, we close all of the channels
-                    CloseableChannel.closeChannels(channels, false);
-                    listener.onFailure(ex);
-                }
+        for (int i = 0; i < numConnections; ++i) {
+            try {
+                TcpChannel channel = initiateChannel(node, PlainActionFuture.newFuture());
+                logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
+                channels.add(channel);
+            } catch (Exception e) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(new ConnectTransportException(node, "general node connection failure", e));
+                return;
             }
+        }
 
-//            // If we make it past the block above, we successfully instantiated all of the channels
-//            try {
-//                TcpChannel.awaitConnected(node, connectionFutures, connectionProfile.getConnectTimeout());
-//            } catch (Exception ex) {
-//                CloseableChannel.closeChannels(channels, false);
-//                listener.onFailure(ex);
-//                return;
-//            }
+        ActionListener<Void> channelsConnectedListener = new ActionListener<Void>() {
 
-            // If we make it past the block above, we have successfully established connections for all of the channels
-//            final TcpChannel handshakeChannel = channels.get(0); // one channel is guaranteed by the connection profile
-//            handshakeChannel.addCloseListener(ActionListener.wrap(() -> cancelHandshakeForChannel(handshakeChannel)));
-//            Version version = null;
-//            try {
-//                version = executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout());
-//            } catch (Exception ex) {
-//                CloseableChannel.closeChannels(channels, false);
-//                listener.onFailure(ex);
-//                return;
-//            }
+            private AtomicInteger pendingConnections = new AtomicInteger(numConnections);
 
-            // If we make it past the block above, we have successfully completed the handshake and the connection is now open.
-            // At this point we should construct the connection, notify the transport service, and attach close listeners to the
-            // underlying channels.
-//            nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
-            final NodeChannels finalNodeChannels = nodeChannels;
+            @Override
+            public void onResponse(Void v) {
+                if (pendingConnections.decrementAndGet() == 0) {
+                    final TcpChannel handshakeChannel = channels.get(0);
+                    try {
+                        asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
+                            @Override
+                            public void onResponse(Version version) {
+                                NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
+                                nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(nodeChannels::close)));
+                                listener.onResponse(nodeChannels);
+                            }
 
-            Consumer<TcpChannel> onClose = c -> {
-                assert c.isOpen() == false : "channel is still open when onClose is called";
-                finalNodeChannels.close();
-            };
+                            @Override
+                            public void onFailure(Exception e) {
+                                CloseableChannel.closeChannels(channels, false);
 
-            nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(() -> onClose.accept(ch))));
-            success = true;
-            listener.onResponse(nodeChannels);
-        } catch (ConnectTransportException ex) {
-            listener.onFailure(ex);
-        } catch (Exception ex) {
-            // ConnectTransportExceptions are handled specifically on the caller end - we wrap the actual exception to ensure
-            // only relevant exceptions are logged on the caller end.. this is the same as in connectToNode
-            listener.onFailure(new ConnectTransportException(node, "general node connection failure", ex));
-        } finally {
-            if (success == false) {
-                IOUtils.closeWhileHandlingException(nodeChannels);
+                                if (e instanceof ConnectTransportException) {
+                                    listener.onFailure(e);
+                                } else {
+                                    listener.onFailure(new ConnectTransportException(node, "general node connection failure", e));
+                                }
+                            }
+                        });
+                    } catch (Exception ex) {
+                        CloseableChannel.closeChannels(channels, false);
+                        listener.onFailure(ex);
+                    }
+                }
+            }
+
+            @Override
+            public void onFailure(Exception ex) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(ex);
             }
+        };
+
+        for (TcpChannel channel : channels) {
+            channel.addConnectListener(channelsConnectedListener);
         }
     }
 
@@ -787,8 +756,7 @@ protected final void doStop() {
                 for (Map.Entry<String, List<TcpServerChannel>> entry : serverChannels.entrySet()) {
                     String profile = entry.getKey();
                     List<TcpServerChannel> channels = entry.getValue();
-                    ActionListener<Void> closeFailLogger = ActionListener.wrap(c -> {
-                        },
+                    ActionListener<Void> closeFailLogger = ActionListener.wrap(c -> {},
                         e -> logger.warn(() -> new ParameterizedMessage("Error closing serverChannel for profile [{}]", profile), e));
                     channels.forEach(c -> c.addCloseListener(closeFailLogger));
                     CloseableChannel.closeChannels(channels, true);
@@ -1545,26 +1513,26 @@ public void writeTo(StreamOutput out) throws IOException {
     public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
         numHandshakes.inc();
         final long requestId = responseHandlers.newRequestId();
-        final AsyncHandshakeResponseHandler handler = new AsyncHandshakeResponseHandler(channel, requestId, getCurrentVersion(), listener);
+        final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel, requestId, getCurrentVersion(), listener);
         pendingHandshakes.put(requestId, handler);
         channel.addCloseListener(ActionListener.wrap(() -> handler.handleException(new TransportException("connection reset"))));
         boolean success = false;
         try {
-        // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
-        // we also have no payload on the request but the response will contain the actual version of the node we talk
-        // to as the payload.
-        final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
-        sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
-            TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
-
-        threadPool.schedule(timeout, ThreadPool.Names.GENERIC,
-            () -> handler.handleException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
-        success = true;
+            // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
+            // we also have no payload on the request but the response will contain the actual version of the node we talk
+            // to as the payload.
+            final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
+            sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
+                TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
+
+            threadPool.schedule(timeout, ThreadPool.Names.GENERIC,
+                () -> handler.handleException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
+            success = true;
         } catch (Exception e) {
             handler.handleException(new SendRequestTransportException(node, HANDSHAKE_ACTION_NAME, e));
         } finally {
             if (success == false) {
-                AsyncHandshakeResponseHandler removed = pendingHandshakes.remove(requestId);
+                TransportResponseHandler<?> removed = pendingHandshakes.remove(requestId);
                 assert removed == null : "Handshake should not be pending if exception was thrown";
             }
         }
@@ -1624,23 +1592,6 @@ final long getNumHandshakes() {
         return numHandshakes.count(); // for testing
     }
 
-    /**
-     * Called once the channel is closed for instance due to a disconnect or a closed socket etc.
-     */
-    private void cancelHandshakeForChannel(TcpChannel channel) {
-        final Optional<Long> first = pendingHandshakes.entrySet().stream()
-            .filter((entry) -> entry.getValue().channel == channel).map(Map.Entry::getKey).findFirst();
-        if (first.isPresent()) {
-            final Long requestId = first.get();
-            final TransportResponseHandler<?> handler = pendingHandshakes.remove(requestId);
-            if (handler != null) {
-                // there might be a race removing this or this method might be called twice concurrently depending on how
-                // the channel is closed ie. due to connection reset or broken pipes
-                handler.handleException(new TransportException("connection reset"));
-            }
-        }
-    }
-
     /**
      * Ensures this transport is still started / open
      *
diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
index a17103789f251..5f0327e05527d 100644
--- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
@@ -271,6 +271,10 @@ public String getProfile() {
         public void addCloseListener(ActionListener<Void> listener) {
         }
 
+        @Override
+        public void addConnectListener(ActionListener<Void> listener) {
+        }
+
         @Override
         public void setSoLinger(int value) throws IOException {
         }
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
index 99aa540b68411..5417ec6ce2a01 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
@@ -176,7 +176,6 @@ protected MockChannel initiateChannel(DiscoveryNode node, ActionListener<Void> c
             if (success == false) {
                 IOUtils.close(socket);
             }
-
         }
 
         executor.submit(() -> {
@@ -184,9 +183,9 @@ protected MockChannel initiateChannel(DiscoveryNode node, ActionListener<Void> c
                 socket.connect(address);
                 socket.setSoLinger(false, 0);
                 channel.loopRead(executor);
-                connectListener.onResponse(null);
+                channel.connectFuture.complete(null);
             } catch (Exception ex) {
-                connectListener.onFailure(ex);
+                channel.connectFuture.completeExceptionally(ex);
             }
         });
 
@@ -238,6 +237,7 @@ public final class MockChannel implements Closeable, TcpChannel, TcpServerChanne
         private final String profile;
         private final CancellableThreads cancellableThreads = new CancellableThreads();
         private final CompletableContext<Void> closeFuture = new CompletableContext<>();
+        private final CompletableContext<Void> connectFuture = new CompletableContext<>();
 
         /**
          * Constructs a new MockChannel instance intended for handling the actual incoming / outgoing traffic.
@@ -386,12 +386,16 @@ public void addCloseListener(ActionListener<Void> listener) {
             closeFuture.addListener(ActionListener.toBiConsumer(listener));
         }
 
+        @Override
+        public void addConnectListener(ActionListener<Void> listener) {
+            connectFuture.addListener(ActionListener.toBiConsumer(listener));
+        }
+
         @Override
         public void setSoLinger(int value) throws IOException {
             if (activeChannel != null && activeChannel.isClosed() == false) {
                 activeChannel.setSoLinger(true, value);
             }
-
         }
 
         @Override
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
index b36685d564593..b846fb6d3a1ef 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
@@ -275,6 +275,11 @@ public void addCloseListener(ActionListener<Void> listener) {
             addCloseListener(ActionListener.toBiConsumer(listener));
         }
 
+        @Override
+        public void addConnectListener(ActionListener<Void> listener) {
+            addConnectListener(ActionListener.toBiConsumer(listener));
+        }
+
         @Override
         public void setSoLinger(int value) throws IOException {
             SocketChannel rawChannel = getRawChannel();

From 1417eff16358152e4c317fa273bfae660ad439e1 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Tue, 30 Oct 2018 16:01:12 -0600
Subject: [PATCH 06/24] WIP

---
 .../elasticsearch/transport/TcpTransport.java | 171 +++++++++---------
 1 file changed, 84 insertions(+), 87 deletions(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index fdcb8407cea84..deac05a5d3fa6 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -93,6 +93,7 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -474,51 +475,14 @@ private void initiateConnection(DiscoveryNode node, ConnectionProfile connection
             }
         }
 
-        ActionListener<Void> channelsConnectedListener = new ActionListener<Void>() {
-
-            private AtomicInteger pendingConnections = new AtomicInteger(numConnections);
-
-            @Override
-            public void onResponse(Void v) {
-                if (pendingConnections.decrementAndGet() == 0) {
-                    final TcpChannel handshakeChannel = channels.get(0);
-                    try {
-                        asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
-                            @Override
-                            public void onResponse(Version version) {
-                                NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
-                                nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(nodeChannels::close)));
-                                listener.onResponse(nodeChannels);
-                            }
-
-                            @Override
-                            public void onFailure(Exception e) {
-                                CloseableChannel.closeChannels(channels, false);
-
-                                if (e instanceof ConnectTransportException) {
-                                    listener.onFailure(e);
-                                } else {
-                                    listener.onFailure(new ConnectTransportException(node, "general node connection failure", e));
-                                }
-                            }
-                        });
-                    } catch (Exception ex) {
-                        CloseableChannel.closeChannels(channels, false);
-                        listener.onFailure(ex);
-                    }
-                }
-            }
-
-            @Override
-            public void onFailure(Exception ex) {
-                CloseableChannel.closeChannels(channels, false);
-                listener.onFailure(ex);
-            }
-        };
+        ChannelsConnectedListener channelsConnectedListener = new ChannelsConnectedListener(node, connectionProfile, channels, listener);
 
         for (TcpChannel channel : channels) {
             channel.addConnectListener(channelsConnectedListener);
         }
+
+        TimeValue connectTimeout = connectionProfile.getConnectTimeout();
+        threadPool.schedule(connectTimeout, ThreadPool.Names.GENERIC, channelsConnectedListener::onTimeout);
     }
 
     protected Version getCurrentVersion() {
@@ -1538,52 +1502,6 @@ public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue tim
         }
     }
 
-    public Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout)
-        throws IOException, InterruptedException {
-        return null;
-//        numHandshakes.inc();
-//        final long requestId = responseHandlers.newRequestId();
-//        final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel);
-//        AtomicReference<Version> versionRef = handler.versionRef;
-//        AtomicReference<Exception> exceptionRef = handler.exceptionRef;
-//        pendingHandshakes.put(requestId, handler);
-//        boolean success = false;
-//        try {
-//            if (channel.isOpen() == false) {
-//                // we have to protect us here since sendRequestToChannel won't barf if the channel is closed.
-//                // it's weird but to change it will cause a lot of impact on the exception handling code all over the codebase.
-//                // yet, if we don't check the state here we might have registered a pending handshake handler but the close
-//                // listener calling #onChannelClosed might have already run and we are waiting on the latch below unitl we time out.
-//                throw new IllegalStateException("handshake failed, channel already closed");
-//            }
-//            // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
-//            // we also have no payload on the request but the response will contain the actual version of the node we talk
-//            // to as the payload.
-//            final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
-//            sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
-//                TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
-//            if (handler.latch.await(timeout.millis(), TimeUnit.MILLISECONDS) == false) {
-//                throw new ConnectTransportException(node, "handshake_timeout[" + timeout + "]");
-//            }
-//            success = true;
-//            if (exceptionRef.get() != null) {
-//                throw new IllegalStateException("handshake failed", exceptionRef.get());
-//            } else {
-//                Version version = versionRef.get();
-//                if (getCurrentVersion().isCompatible(version) == false) {
-//                    throw new IllegalStateException("Received message from unsupported version: [" + version
-//                        + "] minimal compatible version is: [" + getCurrentVersion().minimumCompatibilityVersion() + "]");
-//                }
-//                return version;
-//            }
-//        } finally {
-//            final TransportResponseHandler<?> removedHandler = pendingHandshakes.remove(requestId);
-//            // in the case of a timeout or an exception on the send part the handshake has not been removed yet.
-//            // but the timeout is tricky since it's basically a race condition so we only assert on the success case.
-//            assert success && removedHandler == null || success == false : "handler for requestId [" + requestId + "] is not been removed";
-//        }
-    }
-
     final int getNumPendingHandshakes() { // for testing
         return pendingHandshakes.size();
     }
@@ -1769,4 +1687,83 @@ public final ResponseHandlers getResponseHandlers() {
     public final RequestHandlerRegistry<? extends TransportRequest> getRequestHandler(String action) {
         return requestHandlers.get(action);
     }
+
+    private final class ChannelsConnectedListener implements ActionListener<Void> {
+
+        private static final int FAILED = -1;
+
+        private final DiscoveryNode node;
+        private final ConnectionProfile connectionProfile;
+        private final List<TcpChannel> channels;
+        private final ActionListener<NodeChannels> listener;
+        private final AtomicInteger pendingConnections;
+
+        private ChannelsConnectedListener(DiscoveryNode node, ConnectionProfile connectionProfile, List<TcpChannel> channels,
+                                          ActionListener<NodeChannels> listener) {
+            this.node = node;
+            this.connectionProfile = connectionProfile;
+            this.channels = channels;
+            this.listener = listener;
+            this.pendingConnections = new AtomicInteger(channels.size());
+        }
+
+        @Override
+        public void onResponse(Void v) {
+            assert pendingConnections.get() != 0 : "Should not called onResponse when the pending connections is 0.";
+            if (pendingConnections.get() != FAILED && pendingConnections.decrementAndGet() == 0) {
+                final TcpChannel handshakeChannel = channels.get(0);
+                try {
+                    asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
+                        @Override
+                        public void onResponse(Version version) {
+                            NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
+                            nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(nodeChannels::close)));
+                            listener.onResponse(nodeChannels);
+                        }
+
+                        @Override
+                        public void onFailure(Exception e) {
+                            CloseableChannel.closeChannels(channels, false);
+
+                            if (e instanceof ConnectTransportException) {
+                                listener.onFailure(e);
+                            } else {
+                                listener.onFailure(new ConnectTransportException(node, "general node connection failure", e));
+                            }
+                        }
+                    });
+                } catch (Exception ex) {
+                    CloseableChannel.closeChannels(channels, false);
+                    listener.onFailure(ex);
+                }
+            }
+        }
+
+        @Override
+        public void onFailure(Exception ex) {
+            assert pendingConnections.get() != 0 : "Should not receive non-timeout connection exception if no connections pending.";
+            if (setFailed()) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(new ConnectTransportException(node, "connect_exception", ex));
+            }
+        }
+
+        public void onTimeout() {
+            if (setFailed()) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(new ConnectTransportException(node, "connect_timeout[" + connectionProfile.getConnectTimeout()  + "]"));
+            }
+        }
+
+        private boolean setFailed() {
+            while (true) {
+                int pendingConnections = this.pendingConnections.get();
+                if (pendingConnections == 0 || pendingConnections == FAILED) {
+                    return false;
+                } else if (this.pendingConnections.compareAndSet(pendingConnections, FAILED)) {
+                    return true;
+                }
+            }
+        }
+    }
 }

From cc7f9fc17f060a53bec41765244a11c9f5361931 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Tue, 30 Oct 2018 16:12:13 -0600
Subject: [PATCH 07/24] WIP

---
 .../src/main/java/org/elasticsearch/transport/TcpTransport.java  | 1 -
 1 file changed, 1 deletion(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index deac05a5d3fa6..28548af4c896e 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -93,7 +93,6 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;

From 3781f4d2605fbf99016bdab0f4e3537509af5732 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 12:09:50 -0600
Subject: [PATCH 08/24] Move handshaking to specialized class

---
 .../elasticsearch/transport/TcpTransport.java | 120 ++-----------
 .../transport/TcpTransportHandshaker.java     | 170 ++++++++++++++++++
 2 files changed, 184 insertions(+), 106 deletions(-)
 create mode 100644 server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 28548af4c896e..a792c803b7ba4 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -44,7 +44,6 @@
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.metrics.CounterMetric;
 import org.elasticsearch.common.metrics.MeanMetric;
 import org.elasticsearch.common.network.CloseableChannel;
@@ -194,18 +193,15 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
     // this lock is here to make sure we close this transport and disconnect all the client nodes
     // connections while no connect operations is going on
     private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
-    protected final boolean compress;
+    protected final boolean compress;;
     private volatile BoundTransportAddress boundAddress;
     private final String transportName;
 
-    private final ConcurrentMap<Long, HandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
-    private final CounterMetric numHandshakes = new CounterMetric();
-    private static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
-
     private final MeanMetric readBytesMetric = new MeanMetric();
     private final MeanMetric transmittedBytesMetric = new MeanMetric();
     private volatile Map<String, RequestHandlerRegistry<? extends TransportRequest>> requestHandlers = Collections.emptyMap();
     private final ResponseHandlers responseHandlers = new ResponseHandlers();
+    private final TcpTransportHandshaker handshaker;
     private final TransportLogger transportLogger;
     private final BytesReference pingMessage;
     private final String nodeName;
@@ -223,6 +219,9 @@ public TcpTransport(String transportName, Settings settings, ThreadPool threadPo
         this.networkService = networkService;
         this.transportName = transportName;
         this.transportLogger = new TransportLogger();
+        this.handshaker = new TcpTransportHandshaker(getCurrentVersion(), threadPool,
+            (node, channel, requestId, version) -> sendRequestToChannel(node, channel, requestId, TcpTransportHandshaker.HANDSHAKE_ACTION_NAME,
+                TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, version, TransportStatus.setHandshake((byte) 0)));
         this.nodeName = Node.NODE_NAME_SETTING.get(settings);
 
         final Settings defaultFeatures = DEFAULT_FEATURES_SETTING.get(settings);
@@ -274,54 +273,6 @@ public synchronized <Request extends TransportRequest> void registerRequestHandl
         requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
     }
 
-    private class HandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
-
-        private final long requestId;
-        private final TcpChannel channel;
-        private final ActionListener<Version> listener;
-        private final Version currentVersion;
-
-        private HandshakeResponseHandler(TcpChannel channel, long requestId, Version currentVersion, ActionListener<Version> listener) {
-            this.channel = channel;
-            this.requestId = requestId;
-            this.currentVersion = currentVersion;
-            this.listener = listener;
-        }
-
-        @Override
-        public VersionHandshakeResponse read(StreamInput in) throws IOException {
-            return new VersionHandshakeResponse(in);
-        }
-
-        @Override
-        public void handleResponse(VersionHandshakeResponse response) {
-            HandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
-            if (handler != null) {
-                Version version = response.version;
-                if (currentVersion.isCompatible(version) == false) {
-                    listener.onFailure(new IllegalStateException("Received message from unsupported version: [" + version
-                        + "] minimal compatible version is: [" + currentVersion.minimumCompatibilityVersion() + "]"));
-                } else {
-                    listener.onResponse(version);
-                }
-            }
-        }
-
-        @Override
-        public void handleException(TransportException e) {
-            HandshakeResponseHandler handler = pendingHandshakes.remove(requestId);
-            if (handler != null) {
-                // TODO: Do we really need to wrap in ise?
-                listener.onFailure(new IllegalStateException("handshake failed", e));
-            }
-        }
-
-        @Override
-        public String executor() {
-            return ThreadPool.Names.SAME;
-        }
-    }
-
     public final class NodeChannels extends CloseableConnection {
         private final Map<TransportRequestOptions.Type, ConnectionProfile.ConnectionTypeHandle> typeMapping;
         private final List<TcpChannel> channels;
@@ -1254,12 +1205,12 @@ public final void messageReceived(BytesReference reference, TcpChannel channel)
             } else {
                 final TransportResponseHandler<?> handler;
                 if (isHandshake) {
-                    handler = pendingHandshakes.remove(requestId);
+                    handler = handshaker.removeHandlerForHandshake(requestId);
                 } else {
                     TransportResponseHandler<? extends TransportResponse> theHandler =
                         responseHandlers.onResponseReceived(requestId, messageListener);
                     if (theHandler == null && TransportStatus.isError(status)) {
-                        handler = pendingHandshakes.remove(requestId);
+                        handler = handshaker.removeHandlerForHandshake(requestId);
                     } else {
                         handler = theHandler;
                     }
@@ -1369,8 +1320,9 @@ protected String handleRequest(TcpChannel channel, String profileName, final Str
         TransportChannel transportChannel = null;
         try {
             if (TransportStatus.isHandshake(status)) {
-                final VersionHandshakeResponse response = new VersionHandshakeResponse(getCurrentVersion());
-                sendResponse(version, features, channel, response, requestId, HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY,
+                assert TcpTransportHandshaker.HANDSHAKE_ACTION_NAME.equals(action) : "Invalid handshake action name: " + action;
+                final TransportResponse response = handshaker.createHandshakeResponse();
+                sendResponse(version, features, channel, response, requestId, TcpTransportHandshaker.HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY,
                     TransportStatus.setHandshake((byte) 0));
             } else {
                 final RequestHandlerRegistry reg = getRequestHandler(action);
@@ -1453,60 +1405,16 @@ public void onFailure(Exception e) {
         }
     }
 
-    private static final class VersionHandshakeResponse extends TransportResponse {
-        private final Version version;
-
-        private VersionHandshakeResponse(Version version) {
-            this.version = version;
-        }
-
-        private VersionHandshakeResponse(StreamInput in) throws IOException {
-            super.readFrom(in);
-            version = Version.readVersion(in);
-        }
-
-        @Override
-        public void writeTo(StreamOutput out) throws IOException {
-            super.writeTo(out);
-            assert version != null;
-            Version.writeVersion(version, out);
-        }
-    }
-
     public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
-        numHandshakes.inc();
-        final long requestId = responseHandlers.newRequestId();
-        final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel, requestId, getCurrentVersion(), listener);
-        pendingHandshakes.put(requestId, handler);
-        channel.addCloseListener(ActionListener.wrap(() -> handler.handleException(new TransportException("connection reset"))));
-        boolean success = false;
-        try {
-            // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
-            // we also have no payload on the request but the response will contain the actual version of the node we talk
-            // to as the payload.
-            final Version minCompatVersion = getCurrentVersion().minimumCompatibilityVersion();
-            sendRequestToChannel(node, channel, requestId, HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE,
-                TransportRequestOptions.EMPTY, minCompatVersion, TransportStatus.setHandshake((byte) 0));
-
-            threadPool.schedule(timeout, ThreadPool.Names.GENERIC,
-                () -> handler.handleException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
-            success = true;
-        } catch (Exception e) {
-            handler.handleException(new SendRequestTransportException(node, HANDSHAKE_ACTION_NAME, e));
-        } finally {
-            if (success == false) {
-                TransportResponseHandler<?> removed = pendingHandshakes.remove(requestId);
-                assert removed == null : "Handshake should not be pending if exception was thrown";
-            }
-        }
+        handshaker.sendHandshake(responseHandlers.newRequestId(), node, channel, timeout, listener);
     }
 
-    final int getNumPendingHandshakes() { // for testing
-        return pendingHandshakes.size();
+    final int getNumPendingHandshakes() {
+        return handshaker.getNumPendingHandshakes();
     }
 
     final long getNumHandshakes() {
-        return numHandshakes.count(); // for testing
+        return handshaker.getNumHandshakes();
     }
 
     /**
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
new file mode 100644
index 0000000000000..3f764cbf9d6e6
--- /dev/null
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.transport;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.metrics.CounterMetric;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.threadpool.ThreadPool;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+class TcpTransportHandshaker {
+
+    static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
+    private final ConcurrentMap<Long, HandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
+    private final CounterMetric numHandshakes = new CounterMetric();
+
+    private final Version version;
+    private final ThreadPool threadPool;
+    private final HandshakeSender handshakeSender;
+
+    TcpTransportHandshaker(Version version, ThreadPool threadPool, HandshakeSender handshakeSender) {
+        this.version = version;
+        this.threadPool = threadPool;
+        this.handshakeSender = handshakeSender;
+    }
+
+    void sendHandshake(long requestId, DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
+        numHandshakes.inc();
+        final HandshakeResponseHandler handler = new HandshakeResponseHandler(requestId, version, listener);
+        pendingHandshakes.put(requestId, handler);
+        channel.addCloseListener(ActionListener.wrap(
+            () -> handler.handleLocalException(new TransportException("handshake failed because connection reset"))));
+        boolean success = false;
+        try {
+            // for the request we use the minCompatVersion since we don't know what's the version of the node we talk to
+            // we also have no payload on the request but the response will contain the actual version of the node we talk
+            // to as the payload.
+            final Version minCompatVersion = version.minimumCompatibilityVersion();
+            handshakeSender.sendHandshake(node, channel, requestId, minCompatVersion);
+
+            threadPool.schedule(timeout, ThreadPool.Names.GENERIC,
+                () -> handler.handleLocalException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
+            success = true;
+        } catch (Exception e) {
+            handler.handleLocalException(new SendRequestTransportException(node, HANDSHAKE_ACTION_NAME, e));
+        } finally {
+            if (success == false) {
+                TransportResponseHandler<?> removed = pendingHandshakes.remove(requestId);
+                assert removed == null : "Handshake should not be pending if exception was thrown";
+            }
+        }
+    }
+
+    TransportResponse createHandshakeResponse() {
+        return new VersionHandshakeResponse(version);
+    }
+
+    TransportResponseHandler<?> removeHandlerForHandshake(long requestId) {
+        return pendingHandshakes.remove(requestId);
+    }
+
+    int getNumPendingHandshakes() {
+        return pendingHandshakes.size();
+    }
+
+    long getNumHandshakes() {
+        return numHandshakes.count();
+    }
+
+    private class HandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
+
+        private final long requestId;
+        private final Version currentVersion;
+        private final ActionListener<Version> listener;
+        private final AtomicBoolean isDone = new AtomicBoolean(false);
+
+        private HandshakeResponseHandler(long requestId, Version currentVersion, ActionListener<Version> listener) {
+            this.requestId = requestId;
+            this.currentVersion = currentVersion;
+            this.listener = listener;
+        }
+
+        @Override
+        public VersionHandshakeResponse read(StreamInput in) throws IOException {
+            return new VersionHandshakeResponse(in);
+        }
+
+        @Override
+        public void handleResponse(VersionHandshakeResponse response) {
+            if (isDone.compareAndSet(false, true)) {
+                Version version = response.version;
+                if (currentVersion.isCompatible(version) == false) {
+                    listener.onFailure(new IllegalStateException("Received message from unsupported version: [" + version
+                        + "] minimal compatible version is: [" + currentVersion.minimumCompatibilityVersion() + "]"));
+                } else {
+                    listener.onResponse(version);
+                }
+            }
+        }
+
+        @Override
+        public void handleException(TransportException e) {
+            if (isDone.compareAndSet(false, true)) {
+                listener.onFailure(new IllegalStateException("handshake failed", e));
+            }
+        }
+
+        public void handleLocalException(TransportException e) {
+            if (pendingHandshakes.remove(requestId) != null && isDone.compareAndSet(false, true)) {
+                listener.onFailure(e);
+            }
+        }
+
+        @Override
+        public String executor() {
+            return ThreadPool.Names.SAME;
+        }
+    }
+
+    private static final class VersionHandshakeResponse extends TransportResponse {
+
+        private final Version version;
+
+        private VersionHandshakeResponse(Version version) {
+            this.version = version;
+        }
+
+        private VersionHandshakeResponse(StreamInput in) throws IOException {
+            super.readFrom(in);
+            version = Version.readVersion(in);
+        }
+
+        @Override
+        public void writeTo(StreamOutput out) throws IOException {
+            super.writeTo(out);
+            assert version != null;
+            Version.writeVersion(version, out);
+        }
+    }
+
+    @FunctionalInterface
+    interface HandshakeSender {
+
+        void sendHandshake(DiscoveryNode node, TcpChannel channel, long requestId, Version version) throws IOException;
+    }
+}

From 4cf05231084fe13ea441e66a1d3bc427cd79fe7b Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 12:23:47 -0600
Subject: [PATCH 09/24] WIP

---
 .../netty4/SimpleNetty4TransportTests.java    |  4 +--
 .../nio/SimpleNioTransportTests.java          |  4 +--
 .../elasticsearch/transport/TcpTransport.java | 13 +++++----
 .../transport/TcpTransportHandshaker.java     | 27 +++++++++++++------
 .../AbstractSimpleTransportTestCase.java      |  2 +-
 .../transport/MockTcpTransportTests.java      |  4 +--
 .../nio/SimpleMockNioTransportTests.java      |  4 +--
 ...stractSimpleSecurityTransportTestCase.java |  2 +-
 ...pleSecurityNetty4ServerTransportTests.java |  4 +--
 .../nio/SimpleSecurityNioTransportTests.java  |  4 +--
 10 files changed, 39 insertions(+), 29 deletions(-)

diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
index c78f2af8e78c3..7c15e09a0e61d 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
@@ -59,9 +59,9 @@ public static MockTransportService nettyFromThreadPool(Settings settings, Thread
             BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.asyncHandshake(node, channel, timeout, listener);
+                    super.executedHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
index 52b8812576983..c7b65a53cf073 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
@@ -63,9 +63,9 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
+            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
-                    super.asyncHandshake(node, channel, timeout, listener);
+                    super.executedHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index a792c803b7ba4..899b4674b5633 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -221,7 +221,9 @@ public TcpTransport(String transportName, Settings settings, ThreadPool threadPo
         this.transportLogger = new TransportLogger();
         this.handshaker = new TcpTransportHandshaker(getCurrentVersion(), threadPool,
             (node, channel, requestId, version) -> sendRequestToChannel(node, channel, requestId, TcpTransportHandshaker.HANDSHAKE_ACTION_NAME,
-                TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, version, TransportStatus.setHandshake((byte) 0)));
+                TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, version, TransportStatus.setHandshake((byte) 0)),
+            (version, features, channel, response, requestId) -> sendResponse(version, features, channel, response, requestId,
+                TcpTransportHandshaker.HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY, TransportStatus.setHandshake((byte) 0)));
         this.nodeName = Node.NODE_NAME_SETTING.get(settings);
 
         final Settings defaultFeatures = DEFAULT_FEATURES_SETTING.get(settings);
@@ -1320,10 +1322,7 @@ protected String handleRequest(TcpChannel channel, String profileName, final Str
         TransportChannel transportChannel = null;
         try {
             if (TransportStatus.isHandshake(status)) {
-                assert TcpTransportHandshaker.HANDSHAKE_ACTION_NAME.equals(action) : "Invalid handshake action name: " + action;
-                final TransportResponse response = handshaker.createHandshakeResponse();
-                sendResponse(version, features, channel, response, requestId, TcpTransportHandshaker.HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY,
-                    TransportStatus.setHandshake((byte) 0));
+                handshaker.handleHandshake(version, features, channel, requestId);
             } else {
                 final RequestHandlerRegistry reg = getRequestHandler(action);
                 if (reg == null) {
@@ -1405,7 +1404,7 @@ public void onFailure(Exception e) {
         }
     }
 
-    public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
+    public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
         handshaker.sendHandshake(responseHandlers.newRequestId(), node, channel, timeout, listener);
     }
 
@@ -1620,7 +1619,7 @@ public void onResponse(Void v) {
             if (pendingConnections.get() != FAILED && pendingConnections.decrementAndGet() == 0) {
                 final TcpChannel handshakeChannel = channels.get(0);
                 try {
-                    asyncHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
+                    executedHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
                         @Override
                         public void onResponse(Version version) {
                             NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
index 3f764cbf9d6e6..27dd5117724cf 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
@@ -28,6 +28,7 @@
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.IOException;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -40,12 +41,15 @@ class TcpTransportHandshaker {
 
     private final Version version;
     private final ThreadPool threadPool;
-    private final HandshakeSender handshakeSender;
+    private final HandshakeRequestSender handshakeRequestSender;
+    private final HandshakeResponseSender handshakeResponseSender;
 
-    TcpTransportHandshaker(Version version, ThreadPool threadPool, HandshakeSender handshakeSender) {
+    TcpTransportHandshaker(Version version, ThreadPool threadPool, HandshakeRequestSender handshakeRequestSender,
+                           HandshakeResponseSender handshakeResponseSender) {
         this.version = version;
         this.threadPool = threadPool;
-        this.handshakeSender = handshakeSender;
+        this.handshakeRequestSender = handshakeRequestSender;
+        this.handshakeResponseSender = handshakeResponseSender;
     }
 
     void sendHandshake(long requestId, DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
@@ -60,7 +64,7 @@ void sendHandshake(long requestId, DiscoveryNode node, TcpChannel channel, TimeV
             // we also have no payload on the request but the response will contain the actual version of the node we talk
             // to as the payload.
             final Version minCompatVersion = version.minimumCompatibilityVersion();
-            handshakeSender.sendHandshake(node, channel, requestId, minCompatVersion);
+            handshakeRequestSender.sendRequest(node, channel, requestId, minCompatVersion);
 
             threadPool.schedule(timeout, ThreadPool.Names.GENERIC,
                 () -> handler.handleLocalException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
@@ -75,8 +79,8 @@ void sendHandshake(long requestId, DiscoveryNode node, TcpChannel channel, TimeV
         }
     }
 
-    TransportResponse createHandshakeResponse() {
-        return new VersionHandshakeResponse(version);
+    void handleHandshake(Version version, Set<String> features, TcpChannel channel, long requestId) throws IOException {
+        handshakeResponseSender.sendResponse(version, features, channel, new VersionHandshakeResponse(this.version), requestId);
     }
 
     TransportResponseHandler<?> removeHandlerForHandshake(long requestId) {
@@ -163,8 +167,15 @@ public void writeTo(StreamOutput out) throws IOException {
     }
 
     @FunctionalInterface
-    interface HandshakeSender {
+    interface HandshakeRequestSender {
 
-        void sendHandshake(DiscoveryNode node, TcpChannel channel, long requestId, Version version) throws IOException;
+        void sendRequest(DiscoveryNode node, TcpChannel channel, long requestId, Version version) throws IOException;
+    }
+
+    @FunctionalInterface
+    interface HandshakeResponseSender {
+
+        void sendResponse(Version version, Set<String> features, TcpChannel channel, TransportResponse response, long requestId)
+            throws IOException;
     }
 }
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
index 31acd2446ec80..1412693969a28 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
@@ -2029,7 +2029,7 @@ protected String handleRequest(TcpChannel mockChannel, String profileName, Strea
                  new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0),
                  connectionProfile)) {
             PlainActionFuture<Version> listener = PlainActionFuture.newFuture();
-            originalTransport.asyncHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
+            originalTransport.executedHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
                 TimeValue.timeValueSeconds(10), listener);
             assertEquals(listener.actionGet(), Version.CURRENT);
         }
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
index bbea613e1135f..31d9aab8b6afc 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
@@ -42,9 +42,9 @@ protected MockTransportService build(Settings settings, Version version, Cluster
             new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()), version) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.asyncHandshake(node, channel, timeout, listener);
+                    super.executedHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
index 74e96040d9f22..bf480f8b594b3 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
@@ -63,9 +63,9 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.asyncHandshake(node, channel, timeout, listener);
+                    super.executedHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
index 2d372bc085f5a..4568f800ee60d 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
@@ -117,7 +117,7 @@ public void testTcpHandshake() throws IOException, InterruptedException {
                  new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0),
                  connectionProfile)) {
             PlainActionFuture<Version> listener = PlainActionFuture.newFuture();
-            originalTransport.asyncHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
+            originalTransport.executedHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
                 TimeValue.timeValueSeconds(10), listener);
             assertEquals(listener.actionGet(), Version.CURRENT);
         }
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
index a2ea9c56b1466..449a7e56e9e8b 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
@@ -78,9 +78,9 @@ public MockTransportService nettyFromThreadPool(Settings settings, ThreadPool th
             new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.asyncHandshake(node, channel, timeout, listener);
+                    super.executedHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
index e6f2a80ab9f36..8680d7391d370 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
@@ -40,9 +40,9 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
                 new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void asyncHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.asyncHandshake(node, channel, timeout, listener);
+                    super.executedHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }

From 597baf1e0ceb7ada168c9ba91b4c761994fc98e7 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 13:02:20 -0600
Subject: [PATCH 10/24] Cleanup

---
 .../netty4/SimpleNetty4TransportTests.java    |  4 +-
 .../nio/SimpleNioTransportTests.java          |  4 +-
 .../elasticsearch/transport/TcpChannel.java   | 44 +------------------
 .../elasticsearch/transport/TcpTransport.java |  4 +-
 .../AbstractSimpleTransportTestCase.java      |  2 +-
 .../transport/MockTcpTransportTests.java      |  4 +-
 .../nio/SimpleMockNioTransportTests.java      |  4 +-
 ...stractSimpleSecurityTransportTestCase.java |  4 +-
 ...pleSecurityNetty4ServerTransportTests.java |  4 +-
 .../nio/SimpleSecurityNioTransportTests.java  |  4 +-
 10 files changed, 18 insertions(+), 60 deletions(-)

diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
index 7c15e09a0e61d..fb7264313d732 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
@@ -59,9 +59,9 @@ public static MockTransportService nettyFromThreadPool(Settings settings, Thread
             BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
-            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.executedHandshake(node, channel, timeout, listener);
+                    super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
index c7b65a53cf073..552e88f95cd6b 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
@@ -63,9 +63,9 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
-                    super.executedHandshake(node, channel, timeout, listener);
+                    super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
index 8297aa731cafa..bcab670894cf0 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
@@ -85,47 +85,5 @@ public interface TcpChannel extends CloseableChannel {
      *
      * @param listener to be executed
      */
-    default void addConnectListener(ActionListener<Void> listener) {
-        // TODO: Implement
-    }
-
-    /**
-     * Awaits for all of the pending connections to complete. Will throw an exception if at least one of the
-     * connections fails.
-     *
-     * @param discoveryNode the node for the pending connections
-     * @param connectionFutures representing the pending connections
-     * @param connectTimeout to wait for a connection
-     * @throws ConnectTransportException if one of the connections fails
-     */
-    static void awaitConnected(DiscoveryNode discoveryNode, List<ActionFuture<Void>> connectionFutures, TimeValue connectTimeout)
-        throws ConnectTransportException {
-        Exception connectionException = null;
-        boolean allConnected = true;
-
-        for (ActionFuture<Void> connectionFuture : connectionFutures) {
-            try {
-                connectionFuture.get(connectTimeout.getMillis(), TimeUnit.MILLISECONDS);
-            } catch (TimeoutException e) {
-                allConnected = false;
-                break;
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                throw new IllegalStateException(e);
-            } catch (ExecutionException e) {
-                allConnected = false;
-                connectionException = (Exception) e.getCause();
-                break;
-            }
-        }
-
-        if (allConnected == false) {
-            if (connectionException == null) {
-                throw new ConnectTransportException(discoveryNode, "connect_timeout[" + connectTimeout + "]");
-            } else {
-                throw new ConnectTransportException(discoveryNode, "connect_exception", connectionException);
-            }
-        }
-    }
-
+    void addConnectListener(ActionListener<Void> listener);
 }
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 899b4674b5633..bc93e1c34c009 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -1404,7 +1404,7 @@ public void onFailure(Exception e) {
         }
     }
 
-    public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
+    public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
         handshaker.sendHandshake(responseHandlers.newRequestId(), node, channel, timeout, listener);
     }
 
@@ -1619,7 +1619,7 @@ public void onResponse(Void v) {
             if (pendingConnections.get() != FAILED && pendingConnections.decrementAndGet() == 0) {
                 final TcpChannel handshakeChannel = channels.get(0);
                 try {
-                    executedHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
+                    executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
                         @Override
                         public void onResponse(Version version) {
                             NodeChannels nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
index 1412693969a28..2015bbf353de0 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
@@ -2029,7 +2029,7 @@ protected String handleRequest(TcpChannel mockChannel, String profileName, Strea
                  new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0),
                  connectionProfile)) {
             PlainActionFuture<Version> listener = PlainActionFuture.newFuture();
-            originalTransport.executedHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
+            originalTransport.executeHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
                 TimeValue.timeValueSeconds(10), listener);
             assertEquals(listener.actionGet(), Version.CURRENT);
         }
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
index 31d9aab8b6afc..78a1bafb687de 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
@@ -42,9 +42,9 @@ protected MockTransportService build(Settings settings, Version version, Cluster
             new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()), version) {
 
             @Override
-            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.executedHandshake(node, channel, timeout, listener);
+                    super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
index bf480f8b594b3..19fa319e777f6 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
@@ -63,9 +63,9 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
             new NoneCircuitBreakerService()) {
 
             @Override
-            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.executedHandshake(node, channel, timeout, listener);
+                    super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
index 4568f800ee60d..3b98bc8aa5f98 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java
@@ -107,7 +107,7 @@ public void testBindUnavailableAddress() {
     }
 
     @Override
-    public void testTcpHandshake() throws IOException, InterruptedException {
+    public void testTcpHandshake() throws InterruptedException {
         assumeTrue("only tcp transport has a handshake method", serviceA.getOriginalTransport() instanceof TcpTransport);
         TcpTransport originalTransport = (TcpTransport) serviceA.getOriginalTransport();
 
@@ -117,7 +117,7 @@ public void testTcpHandshake() throws IOException, InterruptedException {
                  new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0),
                  connectionProfile)) {
             PlainActionFuture<Version> listener = PlainActionFuture.newFuture();
-            originalTransport.executedHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
+            originalTransport.executeHandshake(connection.getNode(), connection.channel(TransportRequestOptions.Type.PING),
                 TimeValue.timeValueSeconds(10), listener);
             assertEquals(listener.actionGet(), Version.CURRENT);
         }
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
index 449a7e56e9e8b..a94f7f6d3836e 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
@@ -78,9 +78,9 @@ public MockTransportService nettyFromThreadPool(Settings settings, ThreadPool th
             new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.executedHandshake(node, channel, timeout, listener);
+                    super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
index 8680d7391d370..4708b60edd4ec 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
@@ -40,9 +40,9 @@ networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(sett
                 new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void executedHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
                 if (doHandshake) {
-                    super.executedHandshake(node, channel, timeout, listener);
+                    super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }

From fbf4800409c96c1a6ea3a58c5027a1da50d1ab53 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 13:54:49 -0600
Subject: [PATCH 11/24] Fix test

---
 .../elasticsearch/transport/Netty4Plugin.java |  5 ++--
 .../transport/netty4/Netty4Transport.java     |  5 ++--
 .../netty4/Netty4ScheduledPingTests.java      |  9 +++----
 .../Netty4SizeHeaderFrameDecoderTests.java    |  3 ++-
 .../netty4/NettyTransportMultiPortTests.java  |  3 ++-
 .../netty4/SimpleNetty4TransportTests.java    | 10 ++------
 .../discovery/ec2/Ec2DiscoveryTests.java      |  3 +--
 .../transport/nio/NioTransport.java           |  9 +++----
 .../transport/nio/NioTransportPlugin.java     |  5 ++--
 .../transport/nio/NioTransportIT.java         |  3 ++-
 .../nio/SimpleNioTransportTests.java          | 10 ++------
 .../elasticsearch/transport/TcpChannel.java   |  7 ------
 .../elasticsearch/transport/TcpTransport.java | 24 +++++++++----------
 .../TransportReplicationActionTests.java      |  4 +---
 .../discovery/zen/UnicastZenPingTests.java    | 15 ++++--------
 .../transport/TcpTransportTests.java          |  2 +-
 .../transport/MockTcpTransport.java           | 10 ++------
 .../transport/nio/MockNioTransport.java       |  5 ++--
 .../transport/nio/MockNioTransportPlugin.java |  5 ++--
 .../transport/MockTcpTransportTests.java      |  3 +--
 .../nio/SimpleMockNioTransportTests.java      | 13 +++-------
 .../netty4/SecurityNetty4Transport.java       |  4 +++-
 .../xpack/security/Security.java              |  4 ++--
 .../netty4/SecurityNetty4ServerTransport.java |  4 +++-
 .../transport/nio/SecurityNioTransport.java   |  7 +++---
 .../SecurityNetty4ServerTransportTests.java   |  2 ++
 ...pleSecurityNetty4ServerTransportTests.java | 11 ++-------
 .../nio/SimpleSecurityNioTransportTests.java  | 15 ++++--------
 28 files changed, 80 insertions(+), 120 deletions(-)

diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java
index 70afcc86ad8f9..c2c841f889aff 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/Netty4Plugin.java
@@ -19,6 +19,7 @@
 
 package org.elasticsearch.transport;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkModule;
 import org.elasticsearch.common.network.NetworkService;
@@ -81,8 +82,8 @@ public Map<String, Supplier<Transport>> getTransports(Settings settings, ThreadP
                                                           CircuitBreakerService circuitBreakerService,
                                                           NamedWriteableRegistry namedWriteableRegistry,
                                                           NetworkService networkService) {
-        return Collections.singletonMap(NETTY_TRANSPORT_NAME, () -> new Netty4Transport(settings, threadPool, networkService, bigArrays,
-            namedWriteableRegistry, circuitBreakerService));
+        return Collections.singletonMap(NETTY_TRANSPORT_NAME, () -> new Netty4Transport(settings, Version.CURRENT, threadPool,
+            networkService, bigArrays, namedWriteableRegistry, circuitBreakerService));
     }
 
     @Override
diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
index 1ace3953ebc78..1db97c3e4b1e6 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
@@ -39,6 +39,7 @@
 import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.apache.logging.log4j.util.Supplier;
 import org.elasticsearch.ExceptionsHelper;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.SuppressForbidden;
@@ -105,9 +106,9 @@ public class Netty4Transport extends TcpTransport {
     private volatile Bootstrap clientBootstrap;
     private final Map<String, ServerBootstrap> serverBootstraps = newConcurrentMap();
 
-    public Netty4Transport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
+    public Netty4Transport(Settings settings, Version version, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
                            NamedWriteableRegistry namedWriteableRegistry, CircuitBreakerService circuitBreakerService) {
-        super("netty", settings, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
+        super("netty", settings, version, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
         Netty4Utils.setAvailableProcessors(EsExecutors.PROCESSORS_SETTING.get(settings));
         this.workerCount = WORKER_COUNT.get(settings);
 
diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4ScheduledPingTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4ScheduledPingTests.java
index 0f3185add0833..bae0cb7cef980 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4ScheduledPingTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4ScheduledPingTests.java
@@ -18,6 +18,7 @@
  */
 package org.elasticsearch.transport.netty4;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
@@ -59,15 +60,15 @@ public void testScheduledPing() throws Exception {
         CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
 
         NamedWriteableRegistry registry = new NamedWriteableRegistry(Collections.emptyList());
-        final Netty4Transport nettyA = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
-            BigArrays.NON_RECYCLING_INSTANCE, registry, circuitBreakerService);
+        final Netty4Transport nettyA = new Netty4Transport(settings, Version.CURRENT, threadPool,
+            new NetworkService(Collections.emptyList()), BigArrays.NON_RECYCLING_INSTANCE, registry, circuitBreakerService);
         MockTransportService serviceA = new MockTransportService(settings, nettyA, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
                 null);
         serviceA.start();
         serviceA.acceptIncomingRequests();
 
-        final Netty4Transport nettyB = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
-            BigArrays.NON_RECYCLING_INSTANCE, registry, circuitBreakerService);
+        final Netty4Transport nettyB = new Netty4Transport(settings, Version.CURRENT, threadPool,
+            new NetworkService(Collections.emptyList()), BigArrays.NON_RECYCLING_INSTANCE, registry, circuitBreakerService);
         MockTransportService serviceB = new MockTransportService(settings, nettyB, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
                 null);
 
diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java
index 564cf61a39569..a711bb690e366 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java
@@ -19,6 +19,7 @@
 
 package org.elasticsearch.transport.netty4;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.Settings;
@@ -65,7 +66,7 @@ public void startThreadPool() {
         threadPool = new ThreadPool(settings);
         NetworkService networkService = new NetworkService(Collections.emptyList());
         BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
-        nettyTransport = new Netty4Transport(settings, threadPool, networkService, bigArrays,
+        nettyTransport = new Netty4Transport(settings, Version.CURRENT, threadPool, networkService, bigArrays,
             new NamedWriteableRegistry(Collections.emptyList()), new NoneCircuitBreakerService());
         nettyTransport.start();
 
diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java
index a49df3caaba4e..785c4cfb114bc 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyTransportMultiPortTests.java
@@ -18,6 +18,7 @@
  */
 package org.elasticsearch.transport.netty4;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.common.component.Lifecycle;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -118,7 +119,7 @@ public void testThatDefaultProfilePortOverridesGeneralConfiguration() throws Exc
 
     private TcpTransport startTransport(Settings settings, ThreadPool threadPool) {
         BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
-        TcpTransport transport = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
+        TcpTransport transport = new Netty4Transport(settings, Version.CURRENT, threadPool, new NetworkService(Collections.emptyList()),
             bigArrays, new NamedWriteableRegistry(Collections.emptyList()), new NoneCircuitBreakerService());
         transport.start();
 
diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
index fb7264313d732..4c651c31bee7e 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java
@@ -41,7 +41,6 @@
 import org.elasticsearch.transport.Transport;
 import org.elasticsearch.transport.TransportService;
 
-import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Collections;
@@ -55,22 +54,17 @@ public class SimpleNetty4TransportTests extends AbstractSimpleTransportTestCase
     public static MockTransportService nettyFromThreadPool(Settings settings, ThreadPool threadPool, final Version version,
                                                            ClusterSettings clusterSettings, boolean doHandshake) {
         NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
-        Transport transport = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
+        Transport transport = new Netty4Transport(settings, version, threadPool, new NetworkService(Collections.emptyList()),
             BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
-            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
                     super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
-
-            @Override
-            protected Version getCurrentVersion() {
-                return version;
-            }
         };
         MockTransportService mockTransportService =
             MockTransportService.createNewService(settings, transport, version, threadPool, clusterSettings, Collections.emptySet());
diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java
index 295df0c818a91..869edc33ef3ae 100644
--- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java
+++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java
@@ -74,8 +74,7 @@ public static void stopThreadPool() throws InterruptedException {
     public void createTransportService() {
         NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
         final Transport transport = new MockTcpTransport(Settings.EMPTY, threadPool, BigArrays.NON_RECYCLING_INSTANCE,
-            new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()),
-            Version.CURRENT) {
+            new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList())) {
             @Override
             public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException {
                 // we just need to ensure we don't resolve DNS here
diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java
index 129f0ada77d5d..f5275c1b98f3a 100644
--- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java
+++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java
@@ -20,6 +20,7 @@
 package org.elasticsearch.transport.nio;
 
 import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@@ -69,10 +70,10 @@ public class NioTransport extends TcpTransport {
     private volatile NioGroup nioGroup;
     private volatile TcpChannelFactory clientChannelFactory;
 
-    protected NioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
-                 PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry,
-                 CircuitBreakerService circuitBreakerService) {
-        super("nio", settings, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
+    protected NioTransport(Settings settings, Version version, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
+                           PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry,
+                           CircuitBreakerService circuitBreakerService) {
+        super("nio", settings, version, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
         this.pageCacheRecycler = pageCacheRecycler;
     }
 
diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java
index 1da8e909b2dd8..fd57ea20b1c8d 100644
--- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java
+++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransportPlugin.java
@@ -19,6 +19,7 @@
 
 package org.elasticsearch.transport.nio;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.Setting;
@@ -61,8 +62,8 @@ public Map<String, Supplier<Transport>> getTransports(Settings settings, ThreadP
                                                           NamedWriteableRegistry namedWriteableRegistry,
                                                           NetworkService networkService) {
         return Collections.singletonMap(NIO_TRANSPORT_NAME,
-            () -> new NioTransport(settings, threadPool, networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry,
-                circuitBreakerService));
+            () -> new NioTransport(settings, Version.CURRENT, threadPool, networkService, bigArrays, pageCacheRecycler,
+                namedWriteableRegistry, circuitBreakerService));
     }
 
     @Override
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java
index df53a4d79c7ad..0c1bad79ee8e6 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java
@@ -104,7 +104,8 @@ public Map<String, Supplier<Transport>> getTransports(Settings settings, ThreadP
         ExceptionThrowingNioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
                                       PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry,
                                       CircuitBreakerService circuitBreakerService) {
-            super(settings, threadPool, networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry, circuitBreakerService);
+            super(settings, Version.CURRENT, threadPool, networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry,
+                circuitBreakerService);
         }
 
         @Override
diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
index 552e88f95cd6b..8fc1dd04dd7b6 100644
--- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/SimpleNioTransportTests.java
@@ -58,9 +58,8 @@ public static MockTransportService nioFromThreadPool(Settings settings, ThreadPo
                                                          ClusterSettings clusterSettings, boolean doHandshake) {
         NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
         NetworkService networkService = new NetworkService(Collections.emptyList());
-        Transport transport = new NioTransport(settings, threadPool,
-            networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(settings), namedWriteableRegistry,
-            new NoneCircuitBreakerService()) {
+        Transport transport = new NioTransport(settings, version, threadPool, networkService, BigArrays.NON_RECYCLING_INSTANCE,
+            new MockPageCacheRecycler(settings), namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
             public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
@@ -70,11 +69,6 @@ public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue t
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
-
-            @Override
-            protected Version getCurrentVersion() {
-                return version;
-            }
         };
         MockTransportService mockTransportService =
             MockTransportService.createNewService(settings, transport, version, threadPool, clusterSettings, Collections.emptySet());
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
index bcab670894cf0..f4d265389d3d4 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpChannel.java
@@ -19,19 +19,12 @@
 
 package org.elasticsearch.transport;
 
-import org.elasticsearch.action.ActionFuture;
 import org.elasticsearch.action.ActionListener;
-import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.network.CloseableChannel;
-import org.elasticsearch.common.unit.TimeValue;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 
 /**
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 788320f65da7b..ed117f88f6bc2 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -178,6 +178,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
 
     protected final Settings settings;
     private final CircuitBreakerService circuitBreakerService;
+    private final Version version;
     protected final ThreadPool threadPool;
     private final BigArrays bigArrays;
     protected final NetworkService networkService;
@@ -194,7 +195,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
     // this lock is here to make sure we close this transport and disconnect all the client nodes
     // connections while no connect operations is going on
     private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
-    protected final boolean compress;;
+    protected final boolean compress;
     private volatile BoundTransportAddress boundAddress;
     private final String transportName;
 
@@ -207,12 +208,13 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
     private final BytesReference pingMessage;
     private final String nodeName;
 
-    public TcpTransport(String transportName, Settings settings, ThreadPool threadPool, BigArrays bigArrays,
+    public TcpTransport(String transportName, Settings settings,  Version version, ThreadPool threadPool, BigArrays bigArrays,
                         CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry,
                         NetworkService networkService) {
         super(settings);
         this.settings = settings;
         this.profileSettings = getProfileSettings(settings);
+        this.version = version;
         this.threadPool = threadPool;
         this.bigArrays = bigArrays;
         this.circuitBreakerService = circuitBreakerService;
@@ -221,10 +223,11 @@ public TcpTransport(String transportName, Settings settings, ThreadPool threadPo
         this.networkService = networkService;
         this.transportName = transportName;
         this.transportLogger = new TransportLogger();
-        this.handshaker = new TcpTransportHandshaker(getCurrentVersion(), threadPool,
-            (node, channel, requestId, version) -> sendRequestToChannel(node, channel, requestId, TcpTransportHandshaker.HANDSHAKE_ACTION_NAME,
-                TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, version, TransportStatus.setHandshake((byte) 0)),
-            (version, features, channel, response, requestId) -> sendResponse(version, features, channel, response, requestId,
+        this.handshaker = new TcpTransportHandshaker(version, threadPool,
+            (node, channel, requestId, v) -> sendRequestToChannel(node, channel, requestId,
+                TcpTransportHandshaker.HANDSHAKE_ACTION_NAME, TransportRequest.Empty.INSTANCE, TransportRequestOptions.EMPTY, v,
+                TransportStatus.setHandshake((byte) 0)),
+            (v, features, channel, response, requestId) -> sendResponse(v, features, channel, response, requestId,
                 TcpTransportHandshaker.HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY, TransportStatus.setHandshake((byte) 0)));
         this.nodeName = Node.NODE_NAME_SETTING.get(settings);
 
@@ -439,11 +442,6 @@ private void initiateConnection(DiscoveryNode node, ConnectionProfile connection
         threadPool.schedule(connectTimeout, ThreadPool.Names.GENERIC, channelsConnectedListener::onTimeout);
     }
 
-    protected Version getCurrentVersion() {
-        // this is just for tests to mock stuff like the nodes version - tests can override this internally
-        return Version.CURRENT;
-    }
-
     @Override
     public BoundTransportAddress boundAddress() {
         return this.boundAddress;
@@ -828,7 +826,7 @@ private void sendRequestToChannel(final DiscoveryNode node, final TcpChannel cha
             // we pick the smallest of the 2, to support both backward and forward compatibility
             // note, this is the only place we need to do this, since from here on, we use the serialized version
             // as the version to use also when the node receiving this request will send the response with
-            Version version = Version.min(getCurrentVersion(), channelVersion);
+            Version version = Version.min(this.version, channelVersion);
 
             stream.setVersion(version);
             threadPool.getThreadContext().writeTo(stream);
@@ -1199,7 +1197,7 @@ public final void messageReceived(BytesReference reference, TcpChannel channel)
                 streamIn = compressor.streamInput(streamIn);
             }
             final boolean isHandshake = TransportStatus.isHandshake(status);
-            ensureVersionCompatibility(version, getCurrentVersion(), isHandshake);
+            ensureVersionCompatibility(version, this.version, isHandshake);
             streamIn = new NamedWriteableAwareStreamInput(streamIn, namedWriteableRegistry);
             streamIn.setVersion(version);
             threadPool.getThreadContext().readHeaders(streamIn);
diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java
index ff868c3250aef..77080bfd2dc30 100644
--- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java
+++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java
@@ -21,7 +21,6 @@
 
 import org.apache.lucene.store.AlreadyClosedException;
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.UnavailableShardsException;
 import org.elasticsearch.action.admin.indices.close.CloseIndexRequest;
@@ -981,8 +980,7 @@ public void testRetryOnReplicaWithRealTransport() throws Exception {
         final ReplicationTask task = maybeTask();
         NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
         final Transport transport = new MockTcpTransport(Settings.EMPTY, threadPool, BigArrays.NON_RECYCLING_INSTANCE,
-                new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()),
-                Version.CURRENT);
+                new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()));
         transportService = new MockTransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
                 x -> clusterService.localNode(), null, Collections.emptySet());
         transportService.start();
diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
index 4ab738f5c7bc3..ed310ee305acf 100644
--- a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
+++ b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
@@ -377,8 +377,7 @@ public void testPortLimit() throws InterruptedException {
             BigArrays.NON_RECYCLING_INSTANCE,
             new NoneCircuitBreakerService(),
             new NamedWriteableRegistry(Collections.emptyList()),
-            networkService,
-            Version.CURRENT) {
+            networkService) {
 
             @Override
             public BoundTransportAddress boundAddress() {
@@ -419,8 +418,7 @@ public void testRemovingLocalAddresses() throws InterruptedException {
             BigArrays.NON_RECYCLING_INSTANCE,
             new NoneCircuitBreakerService(),
             new NamedWriteableRegistry(Collections.emptyList()),
-            networkService,
-            Version.CURRENT) {
+            networkService) {
 
             @Override
             public BoundTransportAddress boundAddress() {
@@ -465,8 +463,7 @@ public void testUnknownHost() throws InterruptedException {
             BigArrays.NON_RECYCLING_INSTANCE,
             new NoneCircuitBreakerService(),
             new NamedWriteableRegistry(Collections.emptyList()),
-            networkService,
-            Version.CURRENT) {
+            networkService) {
 
             @Override
             public BoundTransportAddress boundAddress() {
@@ -512,8 +509,7 @@ public void testResolveTimeout() throws InterruptedException {
             BigArrays.NON_RECYCLING_INSTANCE,
             new NoneCircuitBreakerService(),
             new NamedWriteableRegistry(Collections.emptyList()),
-            networkService,
-            Version.CURRENT) {
+            networkService) {
 
             @Override
             public BoundTransportAddress boundAddress() {
@@ -578,8 +574,7 @@ public void testResolveReuseExistingNodeConnections() throws ExecutionException,
             BigArrays.NON_RECYCLING_INSTANCE,
             new NoneCircuitBreakerService(),
             new NamedWriteableRegistry(Collections.emptyList()),
-            networkService,
-            v);
+            networkService);
 
         NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier, EnumSet.allOf(Role.class));
         closeables.push(handleA.transportService);
diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
index 5f0327e05527d..416eaf0251f9a 100644
--- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
@@ -188,7 +188,7 @@ public void testCompressRequest() throws IOException {
         AtomicReference<BytesReference> messageCaptor = new AtomicReference<>();
         try {
             TcpTransport transport = new TcpTransport(
-                "test", Settings.builder().put("transport.tcp.compress", compressed).build(), threadPool,
+                "test", Settings.builder().put("transport.tcp.compress", compressed).build(), Version.CURRENT, threadPool,
                 new BigArrays(new PageCacheRecycler(Settings.EMPTY), null), null, null, null) {
 
                 @Override
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
index 5417ec6ce2a01..624258adb0bae 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
@@ -88,7 +88,6 @@ public class MockTcpTransport extends TcpTransport {
     }
 
     private final ExecutorService executor;
-    private final Version mockVersion;
 
     public MockTcpTransport(Settings settings, ThreadPool threadPool, BigArrays bigArrays,
                             CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry,
@@ -100,11 +99,11 @@ public MockTcpTransport(Settings settings, ThreadPool threadPool, BigArrays bigA
     public MockTcpTransport(Settings settings, ThreadPool threadPool, BigArrays bigArrays,
                             CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry,
                             NetworkService networkService, Version mockVersion) {
-        super("mock-tcp-transport", settings, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
+        super("mock-tcp-transport", settings, mockVersion, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry,
+            networkService);
         // we have our own crazy cached threadpool this one is not bounded at all...
         // using the ES thread factory here is crucial for tests otherwise disruption tests won't block that thread
         executor = Executors.newCachedThreadPool(EsExecutors.daemonThreadFactory(settings, Transports.TEST_MOCK_TRANSPORT_THREAD_PREFIX));
-        this.mockVersion = mockVersion;
     }
 
     @Override
@@ -456,10 +455,5 @@ protected void stopInternal() {
             assert openChannels.isEmpty() : "there are still open channels: " + openChannels;
         }
     }
-
-    @Override
-    protected Version getCurrentVersion() {
-        return mockVersion;
-    }
 }
 
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
index b846fb6d3a1ef..0ba9501de24d3 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
@@ -21,6 +21,7 @@
 
 import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.bytes.BytesReference;
@@ -72,10 +73,10 @@ public class MockNioTransport extends TcpTransport {
     private volatile NioGroup nioGroup;
     private volatile MockTcpChannelFactory clientChannelFactory;
 
-    MockNioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
+    MockNioTransport(Settings settings, Version version, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
                      PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry,
                      CircuitBreakerService circuitBreakerService) {
-        super("mock-nio", settings, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
+        super("mock-nio", settings, version, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
         this.pageCacheRecycler = pageCacheRecycler;
     }
 
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransportPlugin.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransportPlugin.java
index 1acd947d5aad2..ceabe72ee4436 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransportPlugin.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransportPlugin.java
@@ -18,6 +18,7 @@
  */
 package org.elasticsearch.transport.nio;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.Settings;
@@ -44,7 +45,7 @@ public Map<String, Supplier<Transport>> getTransports(Settings settings, ThreadP
                                                           NamedWriteableRegistry namedWriteableRegistry,
                                                           NetworkService networkService) {
         return Collections.singletonMap(MOCK_NIO_TRANSPORT_NAME,
-            () -> new MockNioTransport(settings, threadPool, networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry,
-                circuitBreakerService));
+            () -> new MockNioTransport(settings, Version.CURRENT, threadPool, networkService, bigArrays, pageCacheRecycler,
+                namedWriteableRegistry, circuitBreakerService));
     }
 }
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
index 78a1bafb687de..1e5c6092687a6 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/MockTcpTransportTests.java
@@ -30,7 +30,6 @@
 import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
 import org.elasticsearch.test.transport.MockTransportService;
 
-import java.io.IOException;
 import java.util.Collections;
 
 public class MockTcpTransportTests extends AbstractSimpleTransportTestCase {
@@ -42,7 +41,7 @@ protected MockTransportService build(Settings settings, Version version, Cluster
             new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Collections.emptyList()), version) {
 
             @Override
-            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
                     super.executeHandshake(node, channel, timeout, listener);
                 } else {
diff --git a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
index 19fa319e777f6..c6ba13d4ca7b1 100644
--- a/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
+++ b/test/framework/src/test/java/org/elasticsearch/transport/nio/SimpleMockNioTransportTests.java
@@ -58,24 +58,17 @@ public static MockTransportService nioFromThreadPool(Settings settings, ThreadPo
                                                          ClusterSettings clusterSettings, boolean doHandshake) {
         NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
         NetworkService networkService = new NetworkService(Collections.emptyList());
-        Transport transport = new MockNioTransport(settings, threadPool,
-            networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(settings), namedWriteableRegistry,
-            new NoneCircuitBreakerService()) {
+        Transport transport = new MockNioTransport(settings, version, threadPool, networkService, BigArrays.NON_RECYCLING_INSTANCE,
+            new MockPageCacheRecycler(settings), namedWriteableRegistry, new NoneCircuitBreakerService()) {
 
             @Override
-            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
                     super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
-
-            @Override
-            protected Version getCurrentVersion() {
-                return version;
-            }
-
         };
         MockTransportService mockTransportService =
             MockTransportService.createNewService(settings, transport, version, threadPool, clusterSettings, Collections.emptySet());
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java
index e76302aebb058..d135506c1f427 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java
@@ -12,6 +12,7 @@
 import io.netty.channel.ChannelPromise;
 import io.netty.handler.ssl.SslHandler;
 import org.apache.logging.log4j.message.ParameterizedMessage;
+import org.elasticsearch.Version;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.CloseableChannel;
@@ -54,13 +55,14 @@ public class SecurityNetty4Transport extends Netty4Transport {
 
     public SecurityNetty4Transport(
             final Settings settings,
+            final Version version,
             final ThreadPool threadPool,
             final NetworkService networkService,
             final BigArrays bigArrays,
             final NamedWriteableRegistry namedWriteableRegistry,
             final CircuitBreakerService circuitBreakerService,
             final SSLService sslService) {
-        super(settings, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService);
+        super(settings, version, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService);
         this.sslService = sslService;
         this.sslEnabled = XPackSettings.TRANSPORT_SSL_ENABLED.get(settings);
         if (sslEnabled) {
diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java
index b77b17ff602bf..4629512776a3b 100644
--- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java
+++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java
@@ -900,9 +900,9 @@ public Map<String, Supplier<Transport>> getTransports(Settings settings, ThreadP
         }
 
         Map<String, Supplier<Transport>> transports = new HashMap<>();
-        transports.put(SecurityField.NAME4, () -> new SecurityNetty4ServerTransport(settings, threadPool,
+        transports.put(SecurityField.NAME4, () -> new SecurityNetty4ServerTransport(settings, Version.CURRENT, threadPool,
             networkService, bigArrays, namedWriteableRegistry, circuitBreakerService, ipFilter.get(), getSslService()));
-        transports.put(SecurityField.NIO, () -> new SecurityNioTransport(settings, threadPool,
+        transports.put(SecurityField.NIO, () -> new SecurityNioTransport(settings, Version.CURRENT, threadPool,
             networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry, circuitBreakerService, ipFilter.get(), getSslService()));
 
         return Collections.unmodifiableMap(transports);
diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransport.java
index e0794d037e33d..d74aa65e94bee 100644
--- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransport.java
+++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransport.java
@@ -7,6 +7,7 @@
 
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandler;
+import org.elasticsearch.Version;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -25,6 +26,7 @@ public class SecurityNetty4ServerTransport extends SecurityNetty4Transport {
 
     public SecurityNetty4ServerTransport(
             final Settings settings,
+            final Version version,
             final ThreadPool threadPool,
             final NetworkService networkService,
             final BigArrays bigArrays,
@@ -32,7 +34,7 @@ public SecurityNetty4ServerTransport(
             final CircuitBreakerService circuitBreakerService,
             @Nullable final IPFilter authenticator,
             final SSLService sslService) {
-        super(settings, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService, sslService);
+        super(settings, version, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService, sslService);
         this.authenticator = authenticator;
     }
 
diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java
index 71e14696a11ff..d9e4080865e44 100644
--- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java
+++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java
@@ -6,6 +6,7 @@
 package org.elasticsearch.xpack.security.transport.nio;
 
 import org.apache.logging.log4j.message.ParameterizedMessage;
+import org.elasticsearch.Version;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.CloseableChannel;
@@ -64,11 +65,11 @@ public class SecurityNioTransport extends NioTransport {
     private final Map<String, SSLConfiguration> profileConfiguration;
     private final boolean sslEnabled;
 
-    public SecurityNioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
-                                PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry,
+    public SecurityNioTransport(Settings settings, Version version, ThreadPool threadPool, NetworkService networkService,
+                                BigArrays bigArrays, PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry,
                                 CircuitBreakerService circuitBreakerService, @Nullable final IPFilter authenticator,
                                 SSLService sslService) {
-        super(settings, threadPool, networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry, circuitBreakerService);
+        super(settings, version, threadPool, networkService, bigArrays, pageCacheRecycler, namedWriteableRegistry, circuitBreakerService);
         this.authenticator = authenticator;
         this.sslService = sslService;
         this.sslEnabled = XPackSettings.TRANSPORT_SSL_ENABLED.get(settings);
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java
index e9d91f5bd2d6a..dc6bffe5c7271 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java
@@ -8,6 +8,7 @@
 import io.netty.channel.ChannelHandler;
 import io.netty.channel.embedded.EmbeddedChannel;
 import io.netty.handler.ssl.SslHandler;
+import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.MockSecureSettings;
@@ -68,6 +69,7 @@ private SecurityNetty4Transport createTransport(Settings additionalSettings) {
                         .build();
         return new SecurityNetty4ServerTransport(
                 settings,
+                Version.CURRENT,
                 mock(ThreadPool.class),
                 new NetworkService(Collections.emptyList()),
                 mock(BigArrays.class),
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
index a94f7f6d3836e..8c4dcf9e2fac5 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SimpleSecurityNetty4ServerTransportTests.java
@@ -40,7 +40,6 @@
 import javax.net.ssl.SNIServerName;
 import javax.net.ssl.SSLEngine;
 import javax.net.ssl.SSLParameters;
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -73,24 +72,18 @@ public MockTransportService nettyFromThreadPool(Settings settings, ThreadPool th
         Settings settings1 = Settings.builder()
             .put(settings)
             .put("xpack.security.transport.ssl.enabled", true).build();
-        Transport transport = new SecurityNetty4ServerTransport(settings1, threadPool,
+        Transport transport = new SecurityNetty4ServerTransport(settings1, version, threadPool,
             networkService, BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry,
             new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
 
             @Override
-            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
                     super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
-
-            @Override
-            protected Version getCurrentVersion() {
-                return version;
-            }
-
         };
         MockTransportService mockTransportService =
             MockTransportService.createNewService(settings, transport, version, threadPool, clusterSettings,
diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
index 4708b60edd4ec..5f336e2e5d38c 100644
--- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
+++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SimpleSecurityNioTransportTests.java
@@ -23,7 +23,6 @@
 import org.elasticsearch.transport.Transport;
 import org.elasticsearch.xpack.security.transport.AbstractSimpleSecurityTransportTestCase;
 
-import java.io.IOException;
 import java.util.Collections;
 
 public class SimpleSecurityNioTransportTests extends AbstractSimpleSecurityTransportTestCase {
@@ -35,24 +34,18 @@ public MockTransportService nioFromThreadPool(Settings settings, ThreadPool thre
         Settings settings1 = Settings.builder()
                 .put(settings)
                 .put("xpack.security.transport.ssl.enabled", true).build();
-        Transport transport = new SecurityNioTransport(settings1, threadPool,
-                networkService, BigArrays.NON_RECYCLING_INSTANCE, new MockPageCacheRecycler(settings), namedWriteableRegistry,
-                new NoneCircuitBreakerService(), null, createSSLService(settings1)) {
+        Transport transport = new SecurityNioTransport(settings1, version, threadPool, networkService, BigArrays.NON_RECYCLING_INSTANCE,
+            new MockPageCacheRecycler(settings), namedWriteableRegistry, new NoneCircuitBreakerService(), null,
+            createSSLService(settings1)) {
 
             @Override
-            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) throws IOException {
+            public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
                 if (doHandshake) {
                     super.executeHandshake(node, channel, timeout, listener);
                 } else {
                     listener.onResponse(version.minimumCompatibilityVersion());
                 }
             }
-
-            @Override
-            protected Version getCurrentVersion() {
-                return version;
-            }
-
         };
         MockTransportService mockTransportService =
                 MockTransportService.createNewService(settings, transport, version, threadPool, clusterSettings,

From 6db9478e3be54cca97c138d45ee2c4a4b8c57709 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 15:03:34 -0600
Subject: [PATCH 12/24] Fix tests

---
 .../transport/netty4/Netty4Transport.java     |  3 +--
 .../transport/nio/NioTransport.java           |  4 +--
 .../elasticsearch/transport/TcpTransport.java | 27 ++++++++++++-------
 .../transport/TcpTransportTests.java          |  2 +-
 .../transport/MockTcpTransport.java           |  4 +--
 .../transport/nio/MockNioTransport.java       |  6 ++---
 6 files changed, 25 insertions(+), 21 deletions(-)

diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
index 1db97c3e4b1e6..3d772696c6d95 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
@@ -40,7 +40,6 @@
 import org.apache.logging.log4j.util.Supplier;
 import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.Version;
-import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.SuppressForbidden;
 import org.elasticsearch.common.collect.Tuple;
@@ -222,7 +221,7 @@ protected ChannelHandler getClientChannelInitializer(DiscoveryNode node) {
     static final AttributeKey<Netty4TcpServerChannel> SERVER_CHANNEL_KEY = AttributeKey.newInstance("es-server-channel");
 
     @Override
-    protected Netty4TcpChannel initiateChannel(DiscoveryNode node, ActionListener<Void> listener) throws IOException {
+    protected Netty4TcpChannel initiateChannel(DiscoveryNode node) throws IOException {
         InetSocketAddress address = node.getAddress().address();
         Bootstrap bootstrapWithHandler = clientBootstrap.clone();
         bootstrapWithHandler.handler(getClientChannelInitializer(node));
diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java
index f5275c1b98f3a..39ef3f331e166 100644
--- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java
+++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java
@@ -21,7 +21,6 @@
 
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.Version;
-import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
@@ -84,10 +83,9 @@ protected NioTcpServerChannel bind(String name, InetSocketAddress address) throw
     }
 
     @Override
-    protected NioTcpChannel initiateChannel(DiscoveryNode node, ActionListener<Void> connectListener) throws IOException {
+    protected NioTcpChannel initiateChannel(DiscoveryNode node) throws IOException {
         InetSocketAddress address = node.getAddress().address();
         NioTcpChannel channel = nioGroup.openChannel(address, clientChannelFactory);
-        channel.addConnectListener(ActionListener.toBiConsumer(connectListener));
         return channel;
     }
 
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index ed117f88f6bc2..c4972e3e0c4cb 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -406,15 +406,24 @@ public NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connect
         try {
             ensureOpen();
             PlainActionFuture<NodeChannels> connectionFuture = PlainActionFuture.newFuture();
-            initiateConnection(node, connectionProfile, connectionFuture);
-            // TODO: This will not return the correct exceptions currently
-            return connectionFuture.actionGet();
+            List<TcpChannel> pendingChannels = initiateConnection(node, connectionProfile, connectionFuture);
+
+            try {
+                return connectionFuture.actionGet();
+            } catch (IllegalStateException e) {
+                // If the future was interrupted we can close the channels to improve the shutdown of the MockTcpTransport
+                if (e.getCause() instanceof InterruptedException) {
+                    CloseableChannel.closeChannels(pendingChannels, false);
+                }
+                throw e;
+            }
         } finally {
             closeLock.readLock().unlock();
         }
     }
 
-    private void initiateConnection(DiscoveryNode node, ConnectionProfile connectionProfile, ActionListener<NodeChannels> listener) {
+    private List<TcpChannel> initiateConnection(DiscoveryNode node, ConnectionProfile connectionProfile,
+                                                ActionListener<NodeChannels> listener) {
         int numConnections = connectionProfile.getNumConnections();
         assert numConnections > 0 : "A connection profile must be configured with at least one connection";
 
@@ -422,13 +431,13 @@ private void initiateConnection(DiscoveryNode node, ConnectionProfile connection
 
         for (int i = 0; i < numConnections; ++i) {
             try {
-                TcpChannel channel = initiateChannel(node, PlainActionFuture.newFuture());
+                TcpChannel channel = initiateChannel(node);
                 logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
                 channels.add(channel);
             } catch (Exception e) {
                 CloseableChannel.closeChannels(channels, false);
                 listener.onFailure(new ConnectTransportException(node, "general node connection failure", e));
-                return;
+                return channels;
             }
         }
 
@@ -440,6 +449,7 @@ private void initiateConnection(DiscoveryNode node, ConnectionProfile connection
 
         TimeValue connectTimeout = connectionProfile.getConnectTimeout();
         threadPool.schedule(connectTimeout, ThreadPool.Names.GENERIC, channelsConnectedListener::onTimeout);
+        return channels;
     }
 
     @Override
@@ -787,12 +797,11 @@ protected void serverAcceptedChannel(TcpChannel channel) {
     /**
      * Initiate a single tcp socket channel.
      *
-     * @param node            for the initiated connection
-     * @param connectListener listener to be called when connection complete
+     * @param node for the initiated connection
      * @return the pending connection
      * @throws IOException if an I/O exception occurs while opening the channel
      */
-    protected abstract TcpChannel initiateChannel(DiscoveryNode node, ActionListener<Void> connectListener) throws IOException;
+    protected abstract TcpChannel initiateChannel(DiscoveryNode node) throws IOException;
 
     /**
      * Called to tear down internal resources
diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
index 416eaf0251f9a..b9ce7d3be3700 100644
--- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java
@@ -197,7 +197,7 @@ protected FakeChannel bind(String name, InetSocketAddress address) throws IOExce
                 }
 
                 @Override
-                protected FakeChannel initiateChannel(DiscoveryNode node, ActionListener<Void> connectListener) throws IOException {
+                protected FakeChannel initiateChannel(DiscoveryNode node) throws IOException {
                     return new FakeChannel(messageCaptor);
                 }
 
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
index 624258adb0bae..2fddb42d57034 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java
@@ -162,7 +162,7 @@ private void readMessage(MockChannel mockChannel, StreamInput input) throws IOEx
 
     @Override
     @SuppressForbidden(reason = "real socket for mocking remote connections")
-    protected MockChannel initiateChannel(DiscoveryNode node, ActionListener<Void> connectListener) throws IOException {
+    protected MockChannel initiateChannel(DiscoveryNode node) throws IOException {
         InetSocketAddress address = node.getAddress().address();
         final MockSocket socket = new MockSocket();
         final MockChannel channel = new MockChannel(socket, address, "none");
@@ -181,8 +181,8 @@ protected MockChannel initiateChannel(DiscoveryNode node, ActionListener<Void> c
             try {
                 socket.connect(address);
                 socket.setSoLinger(false, 0);
-                channel.loopRead(executor);
                 channel.connectFuture.complete(null);
+                channel.loopRead(executor);
             } catch (Exception ex) {
                 channel.connectFuture.completeExceptionally(ex);
             }
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
index 0ba9501de24d3..2ff472e5a78f4 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java
@@ -87,11 +87,9 @@ protected MockServerChannel bind(String name, InetSocketAddress address) throws
     }
 
     @Override
-    protected MockSocketChannel initiateChannel(DiscoveryNode node, ActionListener<Void> connectListener) throws IOException {
+    protected MockSocketChannel initiateChannel(DiscoveryNode node) throws IOException {
         InetSocketAddress address = node.getAddress().address();
-        MockSocketChannel channel = nioGroup.openChannel(address, clientChannelFactory);
-        channel.addConnectListener(ActionListener.toBiConsumer(connectListener));
-        return channel;
+        return nioGroup.openChannel(address, clientChannelFactory);
     }
 
     @Override

From 562bc49d5230e58eb60a1931ac5684a8739370d9 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 15:41:03 -0600
Subject: [PATCH 13/24] Fix compile

---
 .../java/org/elasticsearch/xpack/core/XPackClientPlugin.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java
index 21bd005ac5b7c..57717b6fe859a 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java
@@ -5,6 +5,7 @@
  */
 package org.elasticsearch.xpack.core;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.action.Action;
 import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.cluster.ClusterState;
@@ -426,7 +427,7 @@ public Map<String, Supplier<Transport>> getTransports(
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
-        return Collections.singletonMap(SecurityField.NAME4, () -> new SecurityNetty4Transport(settings, threadPool,
+        return Collections.singletonMap(SecurityField.NAME4, () -> new SecurityNetty4Transport(settings, Version.CURRENT, threadPool,
                 networkService, bigArrays, namedWriteableRegistry, circuitBreakerService, sslService));
     }
 

From 46832ee51148460b9404d1aa4d83cdcccdd849ac Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 16:18:24 -0600
Subject: [PATCH 14/24] Fix checkstyle

---
 .../java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java  | 1 -
 1 file changed, 1 deletion(-)

diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java
index 869edc33ef3ae..ab0e3f2acf871 100644
--- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java
+++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java
@@ -20,7 +20,6 @@
 package org.elasticsearch.discovery.ec2;
 
 import com.amazonaws.services.ec2.model.Tag;
-import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.Settings;

From 7439bd1fe4fb4ffb868fd3060208c8ac18f2add0 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Wed, 31 Oct 2018 16:59:08 -0600
Subject: [PATCH 15/24] Fix compile

---
 .../org/elasticsearch/transport/netty4/Netty4TransportIT.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java
index b81c8efcb47ee..b93e09b53649e 100644
--- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java
+++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java
@@ -108,7 +108,7 @@ public ExceptionThrowingNetty4Transport(
                 BigArrays bigArrays,
                 NamedWriteableRegistry namedWriteableRegistry,
                 CircuitBreakerService circuitBreakerService) {
-            super(settings, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService);
+            super(settings, Version.CURRENT, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService);
         }
 
         @Override

From b3a751ccc84d06b28628420914602a3d9992607c Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Thu, 1 Nov 2018 06:41:42 -0600
Subject: [PATCH 16/24] Fix test

---
 .../main/java/org/elasticsearch/transport/TcpTransport.java   | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index c4972e3e0c4cb..d6f83f0f9c2c0 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -434,6 +434,10 @@ private List<TcpChannel> initiateConnection(DiscoveryNode node, ConnectionProfil
                 TcpChannel channel = initiateChannel(node);
                 logger.trace(() -> new ParameterizedMessage("Tcp transport client channel opened: {}", channel));
                 channels.add(channel);
+            } catch (ConnectTransportException e) {
+                CloseableChannel.closeChannels(channels, false);
+                listener.onFailure(e);
+                return channels;
             } catch (Exception e) {
                 CloseableChannel.closeChannels(channels, false);
                 listener.onFailure(new ConnectTransportException(node, "general node connection failure", e));

From 6581590cbc75925986db1dbcde102f79e406c582 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Thu, 1 Nov 2018 16:25:04 -0600
Subject: [PATCH 17/24] Fix concurrency

---
 .../elasticsearch/transport/TcpTransport.java | 19 +++++++++++++++++--
 1 file changed, 17 insertions(+), 2 deletions(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index d6f83f0f9c2c0..5166579494d77 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -1628,8 +1628,9 @@ private ChannelsConnectedListener(DiscoveryNode node, ConnectionProfile connecti
 
         @Override
         public void onResponse(Void v) {
-            assert pendingConnections.get() != 0 : "Should not called onResponse when the pending connections is 0.";
-            if (pendingConnections.get() != FAILED && pendingConnections.decrementAndGet() == 0) {
+            assert pendingConnections.get() != 0 : "Should not call onResponse when the pending connections is 0.";
+            // Returns true if all connections have completed successfully
+            if (setConnectSuccess()) {
                 final TcpChannel handshakeChannel = channels.get(0);
                 try {
                     executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
@@ -1674,6 +1675,20 @@ public void onTimeout() {
             }
         }
 
+        private boolean setConnectSuccess() {
+            while (true) {
+                int pendingConnections = this.pendingConnections.get();
+                if (pendingConnections == FAILED) {
+                    return false;
+                } else {
+                    int newValue = pendingConnections - 1;
+                    if (this.pendingConnections.compareAndSet(pendingConnections, newValue)) {
+                        return newValue == 0;
+                    }
+                }
+            }
+        }
+
         private boolean setFailed() {
             while (true) {
                 int pendingConnections = this.pendingConnections.get();

From e400c97086b775fb04538eda773a53e84267487d Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Thu, 1 Nov 2018 17:14:52 -0600
Subject: [PATCH 18/24] Changes

---
 .../org/elasticsearch/transport/TcpTransportHandshaker.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
index 27dd5117724cf..49908132fe55b 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
@@ -133,7 +133,7 @@ public void handleException(TransportException e) {
             }
         }
 
-        public void handleLocalException(TransportException e) {
+        void handleLocalException(TransportException e) {
             if (pendingHandshakes.remove(requestId) != null && isDone.compareAndSet(false, true)) {
                 listener.onFailure(e);
             }

From 15e6b747dbfd7fe27cd72eff60bc0a6f64bc7b61 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Thu, 1 Nov 2018 17:16:59 -0600
Subject: [PATCH 19/24] Call method

---
 .../org/elasticsearch/transport/TcpTransportHandshaker.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
index 49908132fe55b..ca786f18aa861 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
@@ -134,7 +134,7 @@ public void handleException(TransportException e) {
         }
 
         void handleLocalException(TransportException e) {
-            if (pendingHandshakes.remove(requestId) != null && isDone.compareAndSet(false, true)) {
+            if (removeHandlerForHandshake(requestId) != null && isDone.compareAndSet(false, true)) {
                 listener.onFailure(e);
             }
         }

From 2e71c4f6bd28df661674a345e53d5090966f7085 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 5 Nov 2018 11:31:08 -0700
Subject: [PATCH 20/24] Changes for review

---
 .../transport/netty4/Netty4TcpChannel.java    |  34 +++--
 .../transport/netty4/Netty4Transport.java     |  29 +---
 .../common/network/CloseableChannel.java      |   2 +-
 .../elasticsearch/transport/TcpTransport.java |  40 +-----
 .../transport/TcpTransportHandshaker.java     |  14 +-
 .../TcpTransportHandshakerTests.java          | 134 ++++++++++++++++++
 6 files changed, 178 insertions(+), 75 deletions(-)
 create mode 100644 server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java

diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
index d370c396e980d..38f49169d5a72 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
@@ -21,11 +21,13 @@
 
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelException;
+import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.ChannelPromise;
 import java.io.IOException;
 import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.concurrent.CompletableContext;
 import org.elasticsearch.transport.TcpChannel;
@@ -40,14 +42,10 @@ public class Netty4TcpChannel implements TcpChannel {
     private final CompletableContext<Void> connectContext;
     private final CompletableContext<Void> closeContext = new CompletableContext<>();
 
-    Netty4TcpChannel(Channel channel, String profile) {
-        this(channel, profile, completedConnectContext());
-    }
-
-    Netty4TcpChannel(Channel channel, String profile, CompletableContext<Void> connectContext) {
+    Netty4TcpChannel(Channel channel, String profile, @Nullable ChannelFuture connectFuture) {
         this.channel = channel;
         this.profile = profile;
-        this.connectContext = connectContext;
+        this.connectContext = new CompletableContext<>();
         this.channel.closeFuture().addListener(f -> {
             if (f.isSuccess()) {
                 closeContext.complete(null);
@@ -61,6 +59,24 @@ public class Netty4TcpChannel implements TcpChannel {
                 }
             }
         });
+
+        if (connectFuture != null) {
+            connectFuture.addListener(f -> {
+            if (f.isSuccess()) {
+                connectContext.complete(null);
+            } else {
+                Throwable cause = f.cause();
+                if (cause instanceof Error) {
+                    ExceptionsHelper.maybeDieOnAnotherThread(cause);
+                    connectContext.completeExceptionally(new Exception(cause));
+                } else {
+                    connectContext.completeExceptionally((Exception) cause);
+                }
+            }
+            });
+        } else {
+            connectContext.complete(null);
+        }
     }
 
     @Override
@@ -143,10 +159,4 @@ public String toString() {
             ", remoteAddress=" + channel.remoteAddress() +
             '}';
     }
-
-    private static CompletableContext<Void> completedConnectContext() {
-        CompletableContext<Void> connectContext = new CompletableContext<>();
-        connectContext.complete(null);
-        return connectContext;
-    }
 }
diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
index 2e917a338c674..9e9d94dc4208e 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
@@ -41,7 +41,6 @@
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.SuppressForbidden;
-import org.elasticsearch.common.concurrent.CompletableContext;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.network.NetworkService;
@@ -222,34 +221,18 @@ protected Netty4TcpChannel initiateChannel(DiscoveryNode node) throws IOExceptio
         Bootstrap bootstrapWithHandler = clientBootstrap.clone();
         bootstrapWithHandler.handler(getClientChannelInitializer(node));
         bootstrapWithHandler.remoteAddress(address);
-        ChannelFuture channelFuture = bootstrapWithHandler.connect();
+        ChannelFuture connectFuture = bootstrapWithHandler.connect();
 
-        Channel channel = channelFuture.channel();
+        Channel channel = connectFuture.channel();
         if (channel == null) {
-            ExceptionsHelper.maybeDieOnAnotherThread(channelFuture.cause());
-            throw new IOException(channelFuture.cause());
+            ExceptionsHelper.maybeDieOnAnotherThread(connectFuture.cause());
+            throw new IOException(connectFuture.cause());
         }
         addClosedExceptionLogger(channel);
 
-        CompletableContext<Void> connectContext = new CompletableContext<>();
-
-        Netty4TcpChannel nettyChannel = new Netty4TcpChannel(channel, "default", connectContext);
+        Netty4TcpChannel nettyChannel = new Netty4TcpChannel(channel, "default", connectFuture);
         channel.attr(CHANNEL_KEY).set(nettyChannel);
 
-        channelFuture.addListener(f -> {
-            if (f.isSuccess()) {
-                connectContext.complete(null);
-            } else {
-                Throwable cause = f.cause();
-                if (cause instanceof Error) {
-                    ExceptionsHelper.maybeDieOnAnotherThread(cause);
-                    connectContext.completeExceptionally(new Exception(cause));
-                } else {
-                    connectContext.completeExceptionally((Exception) cause);
-                }
-            }
-        });
-
         return nettyChannel;
     }
 
@@ -312,7 +295,7 @@ protected ServerChannelInitializer(String name) {
         @Override
         protected void initChannel(Channel ch) throws Exception {
             addClosedExceptionLogger(ch);
-            Netty4TcpChannel nettyTcpChannel = new Netty4TcpChannel(ch, name);
+            Netty4TcpChannel nettyTcpChannel = new Netty4TcpChannel(ch, name, null);
             ch.attr(CHANNEL_KEY).set(nettyTcpChannel);
             ch.pipeline().addLast("logging", new ESLoggingHandler());
             ch.pipeline().addLast("size", new Netty4SizeHeaderFrameDecoder());
diff --git a/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java b/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java
index 6b89a90aa2c77..36c0c881ea406 100644
--- a/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java
+++ b/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java
@@ -90,7 +90,7 @@ static <C extends CloseableChannel> void closeChannels(List<C> channels, boolean
             IOUtils.close(channels);
         } catch (IOException e) {
             // The CloseableChannel#close method does not throw IOException, so this should not occur.
-            throw new UncheckedIOException(e);
+            throw new AssertionError(e);
         }
         if (blocking) {
             ArrayList<ActionFuture<Void>> futures = new ArrayList<>(channels.size());
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 59ac0a57d884e..940de7496bb76 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -59,6 +59,7 @@
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.concurrent.AbstractRunnable;
+import org.elasticsearch.common.util.concurrent.CountDown;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
@@ -1608,13 +1609,11 @@ public final RequestHandlerRegistry<? extends TransportRequest> getRequestHandle
 
     private final class ChannelsConnectedListener implements ActionListener<Void> {
 
-        private static final int FAILED = -1;
-
         private final DiscoveryNode node;
         private final ConnectionProfile connectionProfile;
         private final List<TcpChannel> channels;
         private final ActionListener<NodeChannels> listener;
-        private final AtomicInteger pendingConnections;
+        private final CountDown countDown;
 
         private ChannelsConnectedListener(DiscoveryNode node, ConnectionProfile connectionProfile, List<TcpChannel> channels,
                                           ActionListener<NodeChannels> listener) {
@@ -1622,14 +1621,13 @@ private ChannelsConnectedListener(DiscoveryNode node, ConnectionProfile connecti
             this.connectionProfile = connectionProfile;
             this.channels = channels;
             this.listener = listener;
-            this.pendingConnections = new AtomicInteger(channels.size());
+            this.countDown = new CountDown(channels.size());
         }
 
         @Override
         public void onResponse(Void v) {
-            assert pendingConnections.get() != 0 : "Should not call onResponse when the pending connections is 0.";
             // Returns true if all connections have completed successfully
-            if (setConnectSuccess()) {
+            if (countDown.countDown()) {
                 final TcpChannel handshakeChannel = channels.get(0);
                 try {
                     executeHandshake(node, handshakeChannel, connectionProfile.getHandshakeTimeout(), new ActionListener<Version>() {
@@ -1660,43 +1658,17 @@ public void onFailure(Exception e) {
 
         @Override
         public void onFailure(Exception ex) {
-            assert pendingConnections.get() != 0 : "Should not receive non-timeout connection exception if no connections pending.";
-            if (setFailed()) {
+            if (countDown.fastForward()) {
                 CloseableChannel.closeChannels(channels, false);
                 listener.onFailure(new ConnectTransportException(node, "connect_exception", ex));
             }
         }
 
         public void onTimeout() {
-            if (setFailed()) {
+            if (countDown.fastForward()) {
                 CloseableChannel.closeChannels(channels, false);
                 listener.onFailure(new ConnectTransportException(node, "connect_timeout[" + connectionProfile.getConnectTimeout()  + "]"));
             }
         }
-
-        private boolean setConnectSuccess() {
-            while (true) {
-                int pendingConnections = this.pendingConnections.get();
-                if (pendingConnections == FAILED) {
-                    return false;
-                } else {
-                    int newValue = pendingConnections - 1;
-                    if (this.pendingConnections.compareAndSet(pendingConnections, newValue)) {
-                        return newValue == 0;
-                    }
-                }
-            }
-        }
-
-        private boolean setFailed() {
-            while (true) {
-                int pendingConnections = this.pendingConnections.get();
-                if (pendingConnections == 0 || pendingConnections == FAILED) {
-                    return false;
-                } else if (this.pendingConnections.compareAndSet(pendingConnections, FAILED)) {
-                    return true;
-                }
-            }
-        }
     }
 }
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
index ca786f18aa861..d1037d2bcb5bd 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransportHandshaker.java
@@ -33,7 +33,11 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-class TcpTransportHandshaker {
+/**
+ * Sends and receives transport-level connection handshakes. This class will send the initial handshake,
+ * manage state/timeouts while the handshake is in transit, and handle the eventual response.
+ */
+final class TcpTransportHandshaker {
 
     static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
     private final ConcurrentMap<Long, HandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
@@ -70,7 +74,7 @@ void sendHandshake(long requestId, DiscoveryNode node, TcpChannel channel, TimeV
                 () -> handler.handleLocalException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")));
             success = true;
         } catch (Exception e) {
-            handler.handleLocalException(new SendRequestTransportException(node, HANDSHAKE_ACTION_NAME, e));
+            handler.handleLocalException(new ConnectTransportException(node, "failure to send " + HANDSHAKE_ACTION_NAME, e));
         } finally {
             if (success == false) {
                 TransportResponseHandler<?> removed = pendingHandshakes.remove(requestId);
@@ -83,7 +87,7 @@ void handleHandshake(Version version, Set<String> features, TcpChannel channel,
         handshakeResponseSender.sendResponse(version, features, channel, new VersionHandshakeResponse(this.version), requestId);
     }
 
-    TransportResponseHandler<?> removeHandlerForHandshake(long requestId) {
+    TransportResponseHandler<VersionHandshakeResponse> removeHandlerForHandshake(long requestId) {
         return pendingHandshakes.remove(requestId);
     }
 
@@ -145,11 +149,11 @@ public String executor() {
         }
     }
 
-    private static final class VersionHandshakeResponse extends TransportResponse {
+    static final class VersionHandshakeResponse extends TransportResponse {
 
         private final Version version;
 
-        private VersionHandshakeResponse(Version version) {
+        VersionHandshakeResponse(Version version) {
             this.version = version;
         }
 
diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java
new file mode 100644
index 0000000000000..036895fd8354e
--- /dev/null
+++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.transport;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.action.support.PlainActionFuture;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.threadpool.TestThreadPool;
+import org.mockito.ArgumentCaptor;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class TcpTransportHandshakerTests extends ESTestCase {
+
+    private TcpTransportHandshaker handshaker;
+    private DiscoveryNode node;
+    private TcpChannel channel;
+    private TestThreadPool threadPool;
+    private TcpTransportHandshaker.HandshakeRequestSender requestSender;
+    private TcpTransportHandshaker.HandshakeResponseSender responseSender;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        String nodeId = "node-id";
+        channel = mock(TcpChannel.class);
+        requestSender = mock(TcpTransportHandshaker.HandshakeRequestSender.class);
+        responseSender = mock(TcpTransportHandshaker.HandshakeResponseSender.class);
+        node = new DiscoveryNode(nodeId, nodeId, nodeId, "host", "host_address", buildNewFakeTransportAddress(), Collections.emptyMap(),
+            Collections.emptySet(), Version.CURRENT);
+        threadPool = new TestThreadPool("thread-poll");
+        handshaker = new TcpTransportHandshaker(Version.CURRENT, threadPool, requestSender, responseSender);
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        threadPool.shutdown();
+        super.tearDown();
+    }
+
+    public void testHandshakeRequestAndResponse() throws IOException {
+        PlainActionFuture<Version> versionFuture = PlainActionFuture.newFuture();
+        long reqId = randomLongBetween(1, 10);
+        handshaker.sendHandshake(reqId, node, channel, new TimeValue(30, TimeUnit.SECONDS), versionFuture);
+
+        verify(requestSender).sendRequest(node, channel, reqId, Version.CURRENT.minimumCompatibilityVersion());
+
+        assertFalse(versionFuture.isDone());
+
+        TcpChannel mockChannel = mock(TcpChannel.class);
+        handshaker.handleHandshake(Version.CURRENT, Collections.emptySet(), mockChannel, reqId);
+
+
+        ArgumentCaptor<TransportResponse> responseCaptor = ArgumentCaptor.forClass(TransportResponse.class);
+        verify(responseSender).sendResponse(eq(Version.CURRENT), eq(Collections.emptySet()), eq(mockChannel), responseCaptor.capture(),
+            eq(reqId));
+
+        TransportResponseHandler<TcpTransportHandshaker.VersionHandshakeResponse> handler = handshaker.removeHandlerForHandshake(reqId);
+        handler.handleResponse((TcpTransportHandshaker.VersionHandshakeResponse) responseCaptor.getValue());
+
+        assertTrue(versionFuture.isDone());
+        assertEquals(Version.CURRENT, versionFuture.actionGet());
+    }
+
+    public void testHandshakeError() throws IOException {
+        PlainActionFuture<Version> versionFuture = PlainActionFuture.newFuture();
+        long reqId = randomLongBetween(1, 10);
+        handshaker.sendHandshake(reqId, node, channel, new TimeValue(30, TimeUnit.SECONDS), versionFuture);
+
+        verify(requestSender).sendRequest(node, channel, reqId, Version.CURRENT.minimumCompatibilityVersion());
+
+        assertFalse(versionFuture.isDone());
+
+        TransportResponseHandler<TcpTransportHandshaker.VersionHandshakeResponse> handler = handshaker.removeHandlerForHandshake(reqId);
+        handler.handleException(new TransportException("failed"));
+
+        assertTrue(versionFuture.isDone());
+        IllegalStateException ise = expectThrows(IllegalStateException.class, versionFuture::actionGet);
+        assertThat(ise.getMessage(), containsString("handshake failed"));
+    }
+
+    public void testSendRequestThrowsException() throws IOException {
+        PlainActionFuture<Version> versionFuture = PlainActionFuture.newFuture();
+        long reqId = randomLongBetween(1, 10);
+        doThrow(new IOException("boom")).when(requestSender).sendRequest(node, channel, reqId, Version.CURRENT.minimumCompatibilityVersion());
+
+        handshaker.sendHandshake(reqId, node, channel, new TimeValue(30, TimeUnit.SECONDS), versionFuture);
+
+
+        assertTrue(versionFuture.isDone());
+        ConnectTransportException cte = expectThrows(ConnectTransportException.class, versionFuture::actionGet);
+        assertThat(cte.getMessage(), containsString("failure to send internal:tcp/handshake"));
+        assertNull(handshaker.removeHandlerForHandshake(reqId));
+    }
+
+    public void testHandshakeTimeout() throws IOException {
+        PlainActionFuture<Version> versionFuture = PlainActionFuture.newFuture();
+        long reqId = randomLongBetween(1, 10);
+        handshaker.sendHandshake(reqId, node, channel, new TimeValue(100, TimeUnit.MILLISECONDS), versionFuture);
+
+        verify(requestSender).sendRequest(node, channel, reqId, Version.CURRENT.minimumCompatibilityVersion());
+
+        ConnectTransportException cte = expectThrows(ConnectTransportException.class, versionFuture::actionGet);
+        assertThat(cte.getMessage(), containsString("handshake_timeout"));
+
+        assertNull(handshaker.removeHandlerForHandshake(reqId));
+    }
+}

From 8406300daa81a911d1377cf5a7cbc504b45186eb Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 5 Nov 2018 12:08:51 -0700
Subject: [PATCH 21/24] Fix checkstyle

---
 .../src/main/java/org/elasticsearch/transport/TcpTransport.java  | 1 -
 1 file changed, 1 deletion(-)

diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index 940de7496bb76..eedd064bca7ca 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -95,7 +95,6 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;

From c58e7f052bab9bc6f7438bbe9732741ce7a25f2c Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 5 Nov 2018 12:52:14 -0700
Subject: [PATCH 22/24] Fix checkstyle

---
 .../java/org/elasticsearch/common/network/CloseableChannel.java  | 1 -
 1 file changed, 1 deletion(-)

diff --git a/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java b/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java
index 36c0c881ea406..4fc3a0f6bb6bd 100644
--- a/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java
+++ b/server/src/main/java/org/elasticsearch/common/network/CloseableChannel.java
@@ -26,7 +26,6 @@
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;

From 660877aec2f3a0eab6447ba82b97155b4daac0a1 Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Mon, 5 Nov 2018 13:48:39 -0700
Subject: [PATCH 23/24] Fix checkstyle

---
 .../elasticsearch/transport/TcpTransportHandshakerTests.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java
index 036895fd8354e..23e3870842e20 100644
--- a/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportHandshakerTests.java
@@ -108,7 +108,8 @@ public void testHandshakeError() throws IOException {
     public void testSendRequestThrowsException() throws IOException {
         PlainActionFuture<Version> versionFuture = PlainActionFuture.newFuture();
         long reqId = randomLongBetween(1, 10);
-        doThrow(new IOException("boom")).when(requestSender).sendRequest(node, channel, reqId, Version.CURRENT.minimumCompatibilityVersion());
+        Version compatibilityVersion = Version.CURRENT.minimumCompatibilityVersion();
+        doThrow(new IOException("boom")).when(requestSender).sendRequest(node, channel, reqId, compatibilityVersion);
 
         handshaker.sendHandshake(reqId, node, channel, new TimeValue(30, TimeUnit.SECONDS), versionFuture);
 

From f71c30c609965dc3feba083ad1a714bf790ffe7b Mon Sep 17 00:00:00 2001
From: Tim Brooks <tim@uncontended.net>
Date: Tue, 6 Nov 2018 13:29:06 -0700
Subject: [PATCH 24/24] Change from review

---
 .../transport/netty4/Netty4TcpChannel.java             | 10 ++++------
 .../transport/netty4/Netty4Transport.java              |  2 +-
 2 files changed, 5 insertions(+), 7 deletions(-)

diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
index 38f49169d5a72..af66b7c79881a 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4TcpChannel.java
@@ -24,7 +24,9 @@
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.ChannelPromise;
+
 import java.io.IOException;
+
 import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.common.Nullable;
@@ -60,8 +62,7 @@ public class Netty4TcpChannel implements TcpChannel {
             }
         });
 
-        if (connectFuture != null) {
-            connectFuture.addListener(f -> {
+        connectFuture.addListener(f -> {
             if (f.isSuccess()) {
                 connectContext.complete(null);
             } else {
@@ -73,10 +74,7 @@ public class Netty4TcpChannel implements TcpChannel {
                     connectContext.completeExceptionally((Exception) cause);
                 }
             }
-            });
-        } else {
-            connectContext.complete(null);
-        }
+        });
     }
 
     @Override
diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
index 9e9d94dc4208e..b34f50de0f041 100644
--- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
+++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java
@@ -295,7 +295,7 @@ protected ServerChannelInitializer(String name) {
         @Override
         protected void initChannel(Channel ch) throws Exception {
             addClosedExceptionLogger(ch);
-            Netty4TcpChannel nettyTcpChannel = new Netty4TcpChannel(ch, name, null);
+            Netty4TcpChannel nettyTcpChannel = new Netty4TcpChannel(ch, name, ch.newSucceededFuture());
             ch.attr(CHANNEL_KEY).set(nettyTcpChannel);
             ch.pipeline().addLast("logging", new ESLoggingHandler());
             ch.pipeline().addLast("size", new Netty4SizeHeaderFrameDecoder());