Skip to content

Commit

Permalink
Open node connections asynchronously (elastic#35144)
Browse files Browse the repository at this point in the history
This is related to elastic#29023. Additionally at other points we have
discussed a preference for removing the need to unnecessarily block
threads for opening new node connections. This commit lays the groudwork
for this by opening connections asynchronously at the transport level.
We still block, however, this work will make it possible to eventually
remove all blocking on new connections out of the TransportService
and Transport.
  • Loading branch information
Tim-Brooks committed Nov 7, 2018
1 parent 723f57a commit 8f25ac6
Show file tree
Hide file tree
Showing 26 changed files with 745 additions and 442 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -83,8 +84,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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
Expand Down Expand Up @@ -101,9 +101,9 @@ public class Netty4Transport extends TcpTransport {
private volatile Bootstrap clientBootstrap;
private volatile NioEventLoopGroup eventLoopGroup;

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);

Expand Down Expand Up @@ -221,44 +221,31 @@ protected final void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
}

@Override
protected NettyTcpChannel initiateChannel(DiscoveryNode node, ActionListener<Void> listener) throws IOException {
protected NettyTcpChannel initiateChannel(DiscoveryNode node) throws IOException {
InetSocketAddress address = node.getAddress().address();
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);

NettyTcpChannel nettyChannel = new NettyTcpChannel(channel);
NettyTcpChannel nettyChannel = new NettyTcpChannel(channel, "default", connectFuture);
channel.attr(CHANNEL_KEY).set(nettyChannel);

channelFuture.addListener(f -> {
if (f.isSuccess()) {
listener.onResponse(null);
} else {
Throwable cause = f.cause();
if (cause instanceof Error) {
ExceptionsHelper.maybeDieOnAnotherThread(cause);
listener.onFailure(new Exception(cause));
} else {
listener.onFailure((Exception) cause);
}
}
});

return nettyChannel;
}

@Override
protected NettyTcpChannel bind(String name, InetSocketAddress address) {
Channel channel = serverBootstraps.get(name).bind(address).syncUninterruptibly().channel();
NettyTcpChannel esChannel = new NettyTcpChannel(channel);
// TODO: Switch to same server channels
NettyTcpChannel esChannel = new NettyTcpChannel(channel, "server", channel.newSucceededFuture());
channel.attr(CHANNEL_KEY).set(esChannel);
return esChannel;
}
Expand Down Expand Up @@ -314,7 +301,8 @@ protected ServerChannelInitializer(String name) {
@Override
protected void initChannel(Channel ch) throws Exception {
addClosedExceptionLogger(ch);
NettyTcpChannel nettyTcpChannel = new NettyTcpChannel(ch);
NettyTcpChannel nettyTcpChannel = new NettyTcpChannel(ch, name, ch.newSucceededFuture());

ch.attr(CHANNEL_KEY).set(nettyTcpChannel);
serverAcceptedChannel(nettyTcpChannel);
ch.pipeline().addLast("logging", new ESLoggingHandler());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,24 +20,33 @@
package org.elasticsearch.transport.netty4;

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 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;
import org.elasticsearch.transport.TransportException;

import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.concurrent.CompletableFuture;

public class NettyTcpChannel implements TcpChannel {

private final Channel channel;
private final CompletableFuture<Void> closeContext = new CompletableFuture<>();
private final CompletableContext<Void> connectContext;
private final String profile;

NettyTcpChannel(Channel channel) {
NettyTcpChannel(Channel channel, String profile, @Nullable ChannelFuture connectFuture) {
this.channel = channel;
this.profile = profile;
this.connectContext = new CompletableContext<>();
this.channel.closeFuture().addListener(f -> {
if (f.isSuccess()) {
closeContext.complete(null);
Expand All @@ -51,6 +60,20 @@ public class NettyTcpChannel implements TcpChannel {
}
}
});

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);
}
}
});
}

@Override
Expand All @@ -63,9 +86,19 @@ public void addCloseListener(ActionListener<Void> listener) {
closeContext.whenComplete(ActionListener.toBiConsumer(listener));
}

public void addConnectListener(ActionListener<Void> listener) {
connectContext.addListener(ActionListener.toBiConsumer(listener));
}

@Override
public void setSoLinger(int value) {
channel.config().setOption(ChannelOption.SO_LINGER, value);
public void setSoLinger(int value) throws IOException {
if (channel.isOpen()) {
try {
channel.config().setOption(ChannelOption.SO_LINGER, value);
} catch (ChannelException e) {
throw new IOException(e);
}
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -60,15 +61,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);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -40,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;
Expand All @@ -54,23 +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 Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout) throws IOException,
InterruptedException {
public void executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout, ActionListener<Version> listener) {
if (doHandshake) {
return super.executeHandshake(node, channel, timeout);
super.executeHandshake(node, channel, timeout, listener);
} else {
return version.minimumCompatibilityVersion();
listener.onResponse(version.minimumCompatibilityVersion());
}
}

@Override
protected Version getCurrentVersion() {
return version;
}
};
MockTransportService mockTransportService =
MockTransportService.createNewService(settings, transport, version, threadPool, clusterSettings, Collections.emptySet());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -74,8 +73,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
Expand Down
43 changes: 2 additions & 41 deletions server/src/main/java/org/elasticsearch/transport/TcpChannel.java
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,6 @@ public interface TcpChannel extends Releasable {
*/
void addCloseListener(ActionListener<Void> listener);


/**
* This sets the low level socket option {@link java.net.StandardSocketOptions} SO_LINGER on a channel.
*
Expand All @@ -71,7 +70,6 @@ public interface TcpChannel extends Releasable {
*/
void setSoLinger(int value) throws IOException;


/**
* Indicates whether a channel is currently open
*
Expand All @@ -95,6 +93,8 @@ public interface TcpChannel extends Releasable {
*/
void sendMessage(BytesReference reference, ActionListener<Void> listener);

void addConnectListener(ActionListener<Void> listener);

/**
* Closes the channel.
*
Expand Down Expand Up @@ -128,45 +128,6 @@ static <C extends TcpChannel> void closeChannels(List<C> channels, boolean block
}
}

/**
* 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);
}
}
}

static void blockOnFutures(List<ActionFuture<Void>> futures) {
for (ActionFuture<Void> future : futures) {
try {
Expand Down
Loading

0 comments on commit 8f25ac6

Please sign in to comment.