Skip to content

Commit 2e658c8

Browse files
committed
Check for closed connection while opening
While opening a connection to a node, a channel can subsequently close. If this happens, a future callback whose purpose is to close all other channels and disconnect from the node will fire. However, this future will not be ready to close all the channels because the connection will not be exposed to the future callback yet. Since this callback is run once, we will never try to disconnect from this node again and we will be left with a closed channel. This commit adds a check that all channels are open before exposing the channel and throws a general connection exception. In this case, the usual connection retry logic will take over.
1 parent 3898919 commit 2e658c8

File tree

11 files changed

+145
-35
lines changed

11 files changed

+145
-35
lines changed

core/src/main/java/org/elasticsearch/transport/TcpTransport.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -588,7 +588,10 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c
588588
}
589589
}
590590
};
591-
nodeChannels = connectToChannels(node, connectionProfile, onClose);
591+
nodeChannels = connectToChannels(node, connectionProfile, this::onChannelOpen, onClose);
592+
if (!Arrays.stream(nodeChannels.channels).allMatch(this::isOpen)) {
593+
throw new ConnectTransportException(node, "a channel closed while connecting");
594+
}
592595
final Channel channel = nodeChannels.getChannels().get(0); // one channel is guaranteed by the connection profile
593596
final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ?
594597
defaultConnectionProfile.getConnectTimeout() :
@@ -617,6 +620,10 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c
617620
}
618621
}
619622

623+
protected void onChannelOpen(final Channel channel) {
624+
625+
}
626+
620627
private void disconnectFromNodeCloseAndNotify(DiscoveryNode node, NodeChannels nodeChannels) {
621628
assert nodeChannels != null : "nodeChannels must not be null";
622629
try {
@@ -1034,7 +1041,9 @@ protected void innerOnFailure(Exception e) {
10341041
*/
10351042
protected abstract void sendMessage(Channel channel, BytesReference reference, ActionListener<Channel> listener);
10361043

1037-
protected abstract NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile connectionProfile,
1044+
protected abstract NodeChannels connectToChannels(DiscoveryNode node,
1045+
ConnectionProfile connectionProfile,
1046+
Consumer<Channel> onChannelOpen,
10381047
Consumer<Channel> onChannelClose) throws IOException;
10391048

10401049
/**

core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@
3737

3838
import java.io.IOException;
3939
import java.net.InetSocketAddress;
40+
import java.nio.channels.Channel;
4041
import java.util.List;
4142
import java.util.concurrent.TimeUnit;
4243
import java.util.concurrent.atomic.AtomicBoolean;
@@ -224,9 +225,16 @@ protected void sendMessage(Object o, BytesReference reference, ActionListener li
224225
}
225226

226227
@Override
227-
protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile profile,
228+
protected NodeChannels connectToChannels(DiscoveryNode node,
229+
ConnectionProfile profile,
230+
Consumer onChannelOpen,
228231
Consumer onChannelClose) throws IOException {
229-
return new NodeChannels(node, new Object[profile.getNumConnections()], profile);
232+
233+
final Object[] objects = new Object[profile.getNumConnections()];
234+
for (int i = 0; i < objects.length; i++) {
235+
onChannelOpen.accept(objects[i]);
236+
}
237+
return new NodeChannels(node, objects, profile);
230238
}
231239

232240
@Override

modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -252,7 +252,8 @@ public long getNumOpenServerConnections() {
252252
}
253253

254254
@Override
255-
protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile profile, Consumer<Channel> onChannelClose) {
255+
protected NodeChannels connectToChannels(
256+
DiscoveryNode node, ConnectionProfile profile, Consumer<Channel> onChannelOpen, Consumer<Channel> onChannelClose) {
256257
final Channel[] channels = new Channel[profile.getNumConnections()];
257258
final NodeChannels nodeChannels = new NodeChannels(node, channels, profile);
258259
boolean success = false;
@@ -283,6 +284,7 @@ protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile p
283284
if (!future.isSuccess()) {
284285
throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", future.cause());
285286
}
287+
onChannelOpen.accept(future.channel());
286288
channels[i] = future.channel();
287289
channels[i].closeFuture().addListener(closeListener);
288290
}

modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java

Lines changed: 21 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
package org.elasticsearch.transport.netty4;
2121

2222
import io.netty.channel.Channel;
23+
import io.netty.channel.ChannelFuture;
2324
import org.elasticsearch.Version;
2425
import org.elasticsearch.cluster.node.DiscoveryNode;
2526
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@@ -44,15 +45,16 @@
4445
import java.net.InetAddress;
4546
import java.net.UnknownHostException;
4647
import java.util.Collections;
48+
import java.util.function.Consumer;
4749

4850
import static java.util.Collections.emptyMap;
4951
import static java.util.Collections.emptySet;
5052
import static org.hamcrest.Matchers.containsString;
5153

52-
public class SimpleNetty4TransportTests extends AbstractSimpleTransportTestCase {
54+
public class SimpleNetty4TransportTests extends AbstractSimpleTransportTestCase<Channel> {
5355

5456
public static MockTransportService nettyFromThreadPool(Settings settings, ThreadPool threadPool, final Version version,
55-
ClusterSettings clusterSettings, boolean doHandshake) {
57+
ClusterSettings clusterSettings, boolean doHandshake, Consumer<Channel> onChannelOpen) {
5658
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
5759
Transport transport = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
5860
BigArrays.NON_RECYCLING_INSTANCE, namedWriteableRegistry, new NoneCircuitBreakerService()) {
@@ -67,6 +69,11 @@ protected Version executeHandshake(DiscoveryNode node, Channel channel, TimeValu
6769
}
6870
}
6971

72+
@Override
73+
protected void onChannelOpen(Channel channel) {
74+
onChannelOpen.accept(channel);
75+
}
76+
7077
@Override
7178
protected Version getCurrentVersion() {
7279
return version;
@@ -79,13 +86,21 @@ protected Version getCurrentVersion() {
7986
}
8087

8188
@Override
82-
protected MockTransportService build(Settings settings, Version version, ClusterSettings clusterSettings, boolean doHandshake) {
89+
protected MockTransportService build(
90+
Settings settings, Version version, ClusterSettings clusterSettings, boolean doHandshake, Consumer<Channel> onChannelOpen) {
8391
settings = Settings.builder().put(settings).put(TcpTransport.PORT.getKey(), "0").build();
84-
MockTransportService transportService = nettyFromThreadPool(settings, threadPool, version, clusterSettings, doHandshake);
92+
MockTransportService transportService =
93+
nettyFromThreadPool(settings, threadPool, version, clusterSettings, doHandshake, onChannelOpen);
8594
transportService.start();
8695
return transportService;
8796
}
8897

98+
@Override
99+
protected void close(Channel channel) {
100+
final ChannelFuture future = channel.close();
101+
future.awaitUninterruptibly();
102+
}
103+
89104
public void testConnectException() throws UnknownHostException {
90105
try {
91106
serviceA.connectToNode(new DiscoveryNode("C", new TransportAddress(InetAddress.getByName("localhost"), 9876),
@@ -108,7 +123,8 @@ public void testBindUnavailableAddress() {
108123
.build();
109124
ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
110125
BindTransportException bindTransportException = expectThrows(BindTransportException.class, () -> {
111-
MockTransportService transportService = nettyFromThreadPool(settings, threadPool, Version.CURRENT, clusterSettings, true);
126+
MockTransportService transportService =
127+
nettyFromThreadPool(settings, threadPool, Version.CURRENT, clusterSettings, true, c -> {});
112128
try {
113129
transportService.start();
114130
} finally {

test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java

Lines changed: 37 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -79,17 +79,20 @@
7979
import java.util.concurrent.atomic.AtomicBoolean;
8080
import java.util.concurrent.atomic.AtomicInteger;
8181
import java.util.concurrent.atomic.AtomicReference;
82+
import java.util.function.Consumer;
8283
import java.util.stream.Collectors;
8384

8485
import static java.util.Collections.emptyMap;
8586
import static java.util.Collections.emptySet;
87+
import static org.hamcrest.Matchers.containsString;
8688
import static org.hamcrest.Matchers.empty;
8789
import static org.hamcrest.Matchers.equalTo;
90+
import static org.hamcrest.Matchers.hasToString;
8891
import static org.hamcrest.Matchers.instanceOf;
8992
import static org.hamcrest.Matchers.notNullValue;
9093
import static org.hamcrest.Matchers.startsWith;
9194

92-
public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
95+
public abstract class AbstractSimpleTransportTestCase<Channel> extends ESTestCase {
9396

9497
protected ThreadPool threadPool;
9598
// we use always a non-alpha or beta version here otherwise minimumCompatibilityVersion will be different for the two used versions
@@ -105,7 +108,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
105108
protected volatile DiscoveryNode nodeB;
106109
protected volatile MockTransportService serviceB;
107110

108-
protected abstract MockTransportService build(Settings settings, Version version, ClusterSettings clusterSettings, boolean doHandshake);
111+
protected abstract MockTransportService build(
112+
Settings settings, Version version, ClusterSettings clusterSettings, boolean doHandshake, Consumer<Channel> onChannelOpen);
109113

110114
@Override
111115
@Before
@@ -146,6 +150,12 @@ public void onNodeDisconnected(DiscoveryNode node) {
146150

147151
private MockTransportService buildService(final String name, final Version version, ClusterSettings clusterSettings,
148152
Settings settings, boolean acceptRequests, boolean doHandshake) {
153+
return buildService(name, version, clusterSettings, settings, acceptRequests, doHandshake, c -> {});
154+
}
155+
156+
private MockTransportService buildService(final String name, final Version version, ClusterSettings clusterSettings,
157+
Settings settings, boolean acceptRequests, boolean doHandshake,
158+
Consumer<Channel> onChannelOpen) {
149159
MockTransportService service = build(
150160
Settings.builder()
151161
.put(settings)
@@ -154,7 +164,7 @@ private MockTransportService buildService(final String name, final Version versi
154164
.put(TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING")
155165
.build(),
156166
version,
157-
clusterSettings, doHandshake);
167+
clusterSettings, doHandshake, onChannelOpen);
158168
if (acceptRequests) {
159169
service.acceptIncomingRequests();
160170
}
@@ -1692,7 +1702,7 @@ public void testSendRandomRequests() throws InterruptedException {
16921702
.put(TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING")
16931703
.build(),
16941704
version0,
1695-
null, true);
1705+
null, true, c -> {});
16961706
DiscoveryNode nodeC = serviceC.getLocalNode();
16971707
serviceC.acceptIncomingRequests();
16981708

@@ -2125,7 +2135,7 @@ public String executor() {
21252135
public void testHandlerIsInvokedOnConnectionClose() throws IOException, InterruptedException {
21262136
List<String> executors = new ArrayList<>(ThreadPool.THREAD_POOL_TYPES.keySet());
21272137
CollectionUtil.timSort(executors); // makes sure it's reproducible
2128-
TransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
2138+
TransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true, c -> {});
21292139
serviceC.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
21302140
(request, channel) -> {
21312141
// do nothing
@@ -2183,7 +2193,7 @@ public String executor() {
21832193
}
21842194

21852195
public void testConcurrentDisconnectOnNonPublishedConnection() throws IOException, InterruptedException {
2186-
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
2196+
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true, c -> {});
21872197
CountDownLatch receivedLatch = new CountDownLatch(1);
21882198
CountDownLatch sendResponseLatch = new CountDownLatch(1);
21892199
serviceC.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
@@ -2251,7 +2261,7 @@ public String executor() {
22512261
}
22522262

22532263
public void testTransportStats() throws Exception {
2254-
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
2264+
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true, c -> {});
22552265
CountDownLatch receivedLatch = new CountDownLatch(1);
22562266
CountDownLatch sendResponseLatch = new CountDownLatch(1);
22572267
serviceB.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
@@ -2344,7 +2354,7 @@ public String executor() {
23442354
}
23452355

23462356
public void testTransportStatsWithException() throws Exception {
2347-
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
2357+
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true, c -> {});
23482358
CountDownLatch receivedLatch = new CountDownLatch(1);
23492359
CountDownLatch sendResponseLatch = new CountDownLatch(1);
23502360
Exception ex = new RuntimeException("boom");
@@ -2457,7 +2467,7 @@ public void testTransportProfilesWithPortAndHost() {
24572467
.put("transport.profiles.some_other_profile.port", "8700-8800")
24582468
.putArray("transport.profiles.some_other_profile.bind_host", hosts)
24592469
.putArray("transport.profiles.some_other_profile.publish_host", "_local:ipv4_")
2460-
.build(), version0, null, true)) {
2470+
.build(), version0, null, true, c -> {})) {
24612471

24622472
serviceC.start();
24632473
serviceC.acceptIncomingRequests();
@@ -2612,4 +2622,22 @@ public void testProfilesIncludesDefault() {
26122622
assertEquals(new HashSet<>(Arrays.asList("default", "test")), profileSettings.stream().map(s -> s.profileName).collect(Collectors
26132623
.toSet()));
26142624
}
2625+
2626+
public void testChannelCloseWhileConnecting() throws IOException {
2627+
final MockTransportService service = buildService("service", version0, clusterSettings, Settings.EMPTY, true, true, this::close);
2628+
final TcpTransport underlyingTransport = (TcpTransport) service.getOriginalTransport();
2629+
2630+
final String otherName = "other_service";
2631+
try (TransportService otherService = buildService(otherName, Version.CURRENT, null)) {
2632+
final DiscoveryNode node =
2633+
new DiscoveryNode(otherName, otherName, otherService.boundAddress().publishAddress(), emptyMap(), emptySet(), version0);
2634+
final ConnectTransportException e =
2635+
expectThrows(ConnectTransportException.class, () -> underlyingTransport.openConnection(node, null));
2636+
assertThat(e, hasToString(containsString("a channel closed while connecting")));
2637+
}
2638+
service.close();
2639+
}
2640+
2641+
protected abstract void close(Channel channel);
2642+
26152643
}

test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -176,7 +176,9 @@ private void readMessage(MockChannel mockChannel, StreamInput input) throws IOEx
176176
}
177177

178178
@Override
179-
protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile profile,
179+
protected NodeChannels connectToChannels(DiscoveryNode node,
180+
ConnectionProfile profile,
181+
Consumer<MockChannel> onChannelOpen,
180182
Consumer<MockChannel> onChannelClose) throws IOException {
181183
final MockChannel[] mockChannels = new MockChannel[1];
182184
final NodeChannels nodeChannels = new NodeChannels(node, mockChannels, LIGHT_PROFILE); // we always use light here
@@ -193,6 +195,7 @@ protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile p
193195
throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", ex);
194196
}
195197
MockChannel channel = new MockChannel(socket, address, "none", onChannelClose);
198+
onChannelOpen.accept(channel);
196199
channel.loopRead(executor);
197200
mockChannels[0] = channel;
198201
success = true;

test/framework/src/main/java/org/elasticsearch/transport/nio/NioClient.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,10 @@ public NioClient(Logger logger, OpenChannels openChannels, Supplier<SocketSelect
5656
this.channelFactory = channelFactory;
5757
}
5858

59-
public boolean connectToChannels(DiscoveryNode node, NioSocketChannel[] channels, TimeValue connectTimeout,
59+
public boolean connectToChannels(DiscoveryNode node,
60+
NioSocketChannel[] channels,
61+
TimeValue connectTimeout,
62+
Consumer<NioChannel> onChannelOpen,
6063
Consumer<NioChannel> closeListener) throws IOException {
6164
boolean allowedToConnect = semaphore.tryAcquire();
6265
if (allowedToConnect == false) {
@@ -70,6 +73,7 @@ public boolean connectToChannels(DiscoveryNode node, NioSocketChannel[] channels
7073
for (int i = 0; i < channels.length; i++) {
7174
SocketSelector selector = selectorSupplier.get();
7275
NioSocketChannel nioSocketChannel = channelFactory.openNioChannel(address, selector, closeListener);
76+
onChannelOpen.accept(nioSocketChannel);
7377
openChannels.clientChannelOpened(nioSocketChannel);
7478
connections.add(nioSocketChannel);
7579
}

test/framework/src/main/java/org/elasticsearch/transport/nio/NioTransport.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -151,11 +151,12 @@ protected void sendMessage(NioChannel channel, BytesReference reference, ActionL
151151
}
152152

153153
@Override
154-
protected NodeChannels connectToChannels(DiscoveryNode node, ConnectionProfile profile, Consumer<NioChannel> onChannelClose)
154+
protected NodeChannels connectToChannels(
155+
DiscoveryNode node, ConnectionProfile profile, Consumer<NioChannel> onChannelOpen, Consumer<NioChannel> onChannelClose)
155156
throws IOException {
156157
NioSocketChannel[] channels = new NioSocketChannel[profile.getNumConnections()];
157158
ClientChannelCloseListener closeListener = new ClientChannelCloseListener(onChannelClose);
158-
boolean connected = client.connectToChannels(node, channels, profile.getConnectTimeout(), closeListener);
159+
boolean connected = client.connectToChannels(node, channels, profile.getConnectTimeout(), onChannelOpen, closeListener);
159160
if (connected == false) {
160161
throw new ElasticsearchException("client is shutdown");
161162
}

0 commit comments

Comments
 (0)