Skip to content

Commit

Permalink
Remove client connections from TcpTransport (#31886) (#32954)
Browse files Browse the repository at this point in the history
This is related to #31835. This commit adds a connection manager that
manages client connections to other nodes. This means that the
TcpTransport no longer maintains a map of nodes that it is connected
to.
  • Loading branch information
Tim-Brooks authored Aug 18, 2018
1 parent e5e1821 commit f955cb5
Show file tree
Hide file tree
Showing 43 changed files with 1,611 additions and 1,196 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -276,8 +276,12 @@ protected NettyTcpChannel bind(String name, InetSocketAddress address) {
return esChannel;
}

ScheduledPing getPing() {
return scheduledPing;
long successfulPingCount() {
return successfulPings.count();
}

long failedPingCount() {
return failedPings.count();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,8 @@
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponseHandler;
Expand All @@ -48,6 +46,7 @@
import static org.hamcrest.Matchers.greaterThan;

public class Netty4ScheduledPingTests extends ESTestCase {

public void testScheduledPing() throws Exception {
ThreadPool threadPool = new TestThreadPool(getClass().getName());

Expand All @@ -63,14 +62,14 @@ public void testScheduledPing() throws Exception {
final Netty4Transport nettyA = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
BigArrays.NON_RECYCLING_INSTANCE, registry, circuitBreakerService);
MockTransportService serviceA = new MockTransportService(settings, nettyA, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
null);
null);
serviceA.start();
serviceA.acceptIncomingRequests();

final Netty4Transport nettyB = new Netty4Transport(settings, threadPool, new NetworkService(Collections.emptyList()),
BigArrays.NON_RECYCLING_INSTANCE, registry, circuitBreakerService);
MockTransportService serviceB = new MockTransportService(settings, nettyB, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
null);
null);

serviceB.start();
serviceB.acceptIncomingRequests();
Expand All @@ -82,22 +81,19 @@ public void testScheduledPing() throws Exception {
serviceB.connectToNode(nodeA);

assertBusy(() -> {
assertThat(nettyA.getPing().getSuccessfulPings(), greaterThan(100L));
assertThat(nettyB.getPing().getSuccessfulPings(), greaterThan(100L));
assertThat(nettyA.successfulPingCount(), greaterThan(100L));
assertThat(nettyB.successfulPingCount(), greaterThan(100L));
});
assertThat(nettyA.getPing().getFailedPings(), equalTo(0L));
assertThat(nettyB.getPing().getFailedPings(), equalTo(0L));
assertThat(nettyA.failedPingCount(), equalTo(0L));
assertThat(nettyB.failedPingCount(), equalTo(0L));

serviceA.registerRequestHandler("internal:sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
new TransportRequestHandler<TransportRequest.Empty>() {
@Override
public void messageReceived(TransportRequest.Empty request, TransportChannel channel) {
try {
channel.sendResponse(TransportResponse.Empty.INSTANCE, TransportResponseOptions.EMPTY);
} catch (IOException e) {
logger.error("Unexpected failure", e);
fail(e.getMessage());
}
(request, channel) -> {
try {
channel.sendResponse(TransportResponse.Empty.INSTANCE, TransportResponseOptions.EMPTY);
} catch (IOException e) {
logger.error("Unexpected failure", e);
fail(e.getMessage());
}
});

Expand Down Expand Up @@ -129,11 +125,11 @@ public void handleException(TransportException exp) {
}

assertBusy(() -> {
assertThat(nettyA.getPing().getSuccessfulPings(), greaterThan(200L));
assertThat(nettyB.getPing().getSuccessfulPings(), greaterThan(200L));
assertThat(nettyA.successfulPingCount(), greaterThan(200L));
assertThat(nettyB.successfulPingCount(), greaterThan(200L));
});
assertThat(nettyA.getPing().getFailedPings(), equalTo(0L));
assertThat(nettyB.getPing().getFailedPings(), equalTo(0L));
assertThat(nettyA.failedPingCount(), equalTo(0L));
assertThat(nettyB.failedPingCount(), equalTo(0L));

Releasables.close(serviceA, serviceB);
terminate(threadPool);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ public void onFailure(Exception e) {
}

@Override
protected void doRun() throws Exception {
protected void doRun() {
try (Releasable ignored = nodeLocks.acquire(node)) {
validateAndConnectIfNeeded(node);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* 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.action.ActionListener;
import org.elasticsearch.common.concurrent.CompletableContext;


/**
* Abstract Transport.Connection that provides common close logic.
*/
public abstract class CloseableConnection implements Transport.Connection {

private final CompletableContext<Void> closeContext = new CompletableContext<>();

@Override
public void addCloseListener(ActionListener<Void> listener) {
closeContext.addListener(ActionListener.toBiConsumer(listener));
}

@Override
public boolean isClosed() {
return closeContext.isDone();
}

@Override
public void close() {
// This method is safe to call multiple times as the close context will provide concurrency
// protection and only be completed once. The attached listeners will only be notified once.
closeContext.complete(null);
}
}
Loading

0 comments on commit f955cb5

Please sign in to comment.