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 ef5a73c8c8a15..77b5ecb6e6211 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 @@ -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 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 4aed64459dba5..3b34e323e47b5 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 @@ -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; @@ -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()); @@ -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(); @@ -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() { - @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()); } }); @@ -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); diff --git a/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java b/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java index 998cd5ba0a870..2a9d960f8ccea 100644 --- a/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java +++ b/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java @@ -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); } diff --git a/server/src/main/java/org/elasticsearch/transport/CloseableConnection.java b/server/src/main/java/org/elasticsearch/transport/CloseableConnection.java new file mode 100644 index 0000000000000..cdc68dc10f42e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/CloseableConnection.java @@ -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 closeContext = new CompletableContext<>(); + + @Override + public void addCloseListener(ActionListener 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); + } +} diff --git a/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java b/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java new file mode 100644 index 0000000000000..f2b89ff59775c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java @@ -0,0 +1,282 @@ +/* + * 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.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.common.component.Lifecycle; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.AbstractLifecycleRunnable; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.common.util.concurrent.KeyedLock; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This class manages node connections. The connection is opened by the underlying transport. Once the + * connection is opened, this class manages the connection. This includes keep-alive pings and closing + * the connection when the connection manager is closed. + */ +public class ConnectionManager implements Closeable { + + private final ConcurrentMap connectedNodes = ConcurrentCollections.newConcurrentMap(); + private final KeyedLock connectionLock = new KeyedLock<>(); + private final Logger logger; + private final Transport transport; + private final ThreadPool threadPool; + private final TimeValue pingSchedule; + private final Lifecycle lifecycle = new Lifecycle(); + private final ReadWriteLock closeLock = new ReentrantReadWriteLock(); + private final DelegatingNodeConnectionListener connectionListener = new DelegatingNodeConnectionListener(); + + public ConnectionManager(Settings settings, Transport transport, ThreadPool threadPool) { + this.logger = Loggers.getLogger(getClass(), settings); + this.transport = transport; + this.threadPool = threadPool; + this.pingSchedule = TcpTransport.PING_SCHEDULE.get(settings); + this.lifecycle.moveToStarted(); + + if (pingSchedule.millis() > 0) { + threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, new ScheduledPing()); + } + } + + public void addListener(TransportConnectionListener listener) { + this.connectionListener.listeners.add(listener); + } + + public void removeListener(TransportConnectionListener listener) { + this.connectionListener.listeners.remove(listener); + } + + /** + * Connects to a node with the given connection profile. If the node is already connected this method has no effect. + * Once a successful is established, it can be validated before being exposed. + */ + public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, + CheckedBiConsumer connectionValidator) + throws ConnectTransportException { + if (node == null) { + throw new ConnectTransportException(null, "can't connect to a null node"); + } + closeLock.readLock().lock(); // ensure we don't open connections while we are closing + try { + ensureOpen(); + try (Releasable ignored = connectionLock.acquire(node.getId())) { + Transport.Connection connection = connectedNodes.get(node); + if (connection != null) { + return; + } + boolean success = false; + try { + connection = transport.openConnection(node, connectionProfile); + connectionValidator.accept(connection, connectionProfile); + // we acquire a connection lock, so no way there is an existing connection + connectedNodes.put(node, connection); + if (logger.isDebugEnabled()) { + logger.debug("connected to node [{}]", node); + } + try { + connectionListener.onNodeConnected(node); + } finally { + final Transport.Connection finalConnection = connection; + connection.addCloseListener(ActionListener.wrap(() -> { + connectedNodes.remove(node, finalConnection); + connectionListener.onNodeDisconnected(node); + })); + } + if (connection.isClosed()) { + throw new NodeNotConnectedException(node, "connection concurrently closed"); + } + success = true; + } catch (ConnectTransportException e) { + throw e; + } catch (Exception e) { + throw new ConnectTransportException(node, "general node connection failure", e); + } finally { + if (success == false) { // close the connection if there is a failure + logger.trace(() -> new ParameterizedMessage("failed to connect to [{}], cleaning dangling connections", node)); + IOUtils.closeWhileHandlingException(connection); + } + } + } + } finally { + closeLock.readLock().unlock(); + } + } + + /** + * Returns a connection for the given node if the node is connected. + * Connections returned from this method must not be closed. The lifecycle of this connection is + * maintained by this connection manager + * + * @throws NodeNotConnectedException if the node is not connected + * @see #connectToNode(DiscoveryNode, ConnectionProfile, CheckedBiConsumer) + */ + public Transport.Connection getConnection(DiscoveryNode node) { + Transport.Connection connection = connectedNodes.get(node); + if (connection == null) { + throw new NodeNotConnectedException(node, "Node not connected"); + } + return connection; + } + + /** + * Returns {@code true} if the node is connected. + */ + public boolean nodeConnected(DiscoveryNode node) { + return connectedNodes.containsKey(node); + } + + /** + * Disconnected from the given node, if not connected, will do nothing. + */ + public void disconnectFromNode(DiscoveryNode node) { + Transport.Connection nodeChannels = connectedNodes.remove(node); + if (nodeChannels != null) { + // if we found it and removed it we close + nodeChannels.close(); + } + } + + public int connectedNodeCount() { + return connectedNodes.size(); + } + + @Override + public void close() { + lifecycle.moveToStopped(); + CountDownLatch latch = new CountDownLatch(1); + + // TODO: Consider moving all read/write lock (in Transport and this class) to the TransportService + threadPool.generic().execute(() -> { + closeLock.writeLock().lock(); + try { + // we are holding a write lock so nobody modifies the connectedNodes / openConnections map - it's safe to first close + // all instances and then clear them maps + Iterator> iterator = connectedNodes.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry next = iterator.next(); + try { + IOUtils.closeWhileHandlingException(next.getValue()); + } finally { + iterator.remove(); + } + } + } finally { + closeLock.writeLock().unlock(); + latch.countDown(); + } + }); + + try { + try { + latch.await(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + // ignore + } + } finally { + lifecycle.moveToClosed(); + } + } + + private void ensureOpen() { + if (lifecycle.started() == false) { + throw new IllegalStateException("connection manager is closed"); + } + } + + private class ScheduledPing extends AbstractLifecycleRunnable { + + private ScheduledPing() { + super(lifecycle, logger); + } + + @Override + protected void doRunInLifecycle() { + for (Map.Entry entry : connectedNodes.entrySet()) { + Transport.Connection connection = entry.getValue(); + if (connection.sendPing() == false) { + logger.warn("attempted to send ping to connection without support for pings [{}]", connection); + } + } + } + + @Override + protected void onAfterInLifecycle() { + try { + threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, this); + } catch (EsRejectedExecutionException ex) { + if (ex.isExecutorShutdown()) { + logger.debug("couldn't schedule new ping execution, executor is shutting down", ex); + } else { + throw ex; + } + } + } + + @Override + public void onFailure(Exception e) { + if (lifecycle.stoppedOrClosed()) { + logger.trace("failed to send ping transport message", e); + } else { + logger.warn("failed to send ping transport message", e); + } + } + } + + private static final class DelegatingNodeConnectionListener implements TransportConnectionListener { + + private final List listeners = new CopyOnWriteArrayList<>(); + + @Override + public void onNodeDisconnected(DiscoveryNode key) { + for (TransportConnectionListener listener : listeners) { + listener.onNodeDisconnected(key); + } + } + + @Override + public void onNodeConnected(DiscoveryNode node) { + for (TransportConnectionListener listener : listeners) { + listener.onNodeConnected(node); + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java index 4edfa437eafaa..d43d50e35ac6d 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java @@ -293,11 +293,26 @@ public void sendRequest(long requestId, String action, TransportRequest request, TransportActionProxy.wrapRequest(targetNode, request), options); } + @Override + public boolean sendPing() { + return proxyConnection.sendPing(); + } + @Override public void close() { assert false: "proxy connections must not be closed"; } + @Override + public void addCloseListener(ActionListener listener) { + proxyConnection.addCloseListener(listener); + } + + @Override + public boolean isClosed() { + return proxyConnection.isClosed(); + } + @Override public Version getVersion() { return proxyConnection.getVersion(); diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index 118987e2fa099..6ca706aa63f0c 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -21,8 +21,6 @@ import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.hppc.IntSet; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.common.Booleans; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionFuture; @@ -30,7 +28,7 @@ import org.elasticsearch.action.NotifyOnceListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -49,7 +47,6 @@ 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.lease.Releasable; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.metrics.MeanMetric; import org.elasticsearch.common.network.NetworkAddress; @@ -63,11 +60,9 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.concurrent.AbstractLifecycleRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.rest.RestStatus; @@ -89,7 +84,6 @@ import java.util.EnumMap; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -186,17 +180,16 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements public static final Setting.AffixSetting PUBLISH_PORT_PROFILE = affixKeySetting("transport.profiles.", "publish_port", key -> intSetting(key, -1, -1, Setting.Property.NodeScope)); - private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.9); public static final int PING_DATA_SIZE = -1; + protected final CounterMetric successfulPings = new CounterMetric(); + protected final CounterMetric failedPings = new CounterMetric(); + private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.9); public static final String FEATURE_PREFIX = "transport.features"; public static final Setting DEFAULT_FEATURES_SETTING = Setting.groupSetting(FEATURE_PREFIX + ".", Setting.Property.NodeScope); private final String[] features; private final CircuitBreakerService circuitBreakerService; - // package visibility for tests - protected final ScheduledPing scheduledPing; - private final TimeValue pingSchedule; protected final ThreadPool threadPool; private final BigArrays bigArrays; protected final NetworkService networkService; @@ -205,16 +198,15 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final DelegatingTransportConnectionListener transportListener = new DelegatingTransportConnectionListener(); private final ConcurrentMap profileBoundAddresses = newConcurrentMap(); + // node id to actual channel - private final ConcurrentMap connectedNodes = newConcurrentMap(); private final Map> serverChannels = newConcurrentMap(); private final Set acceptedChannels = Collections.newSetFromMap(new ConcurrentHashMap<>()); - private final KeyedLock connectionLock = new KeyedLock<>(); private final NamedWriteableRegistry namedWriteableRegistry; // 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... (this might help with 100% CPU when stopping the transport?) + // connections while no connect operations is going on private final ReadWriteLock closeLock = new ReentrantReadWriteLock(); protected final boolean compress; private volatile BoundTransportAddress boundAddress; @@ -229,6 +221,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final MeanMetric transmittedBytesMetric = new MeanMetric(); private volatile Map requestHandlers = Collections.emptyMap(); private final ResponseHandlers responseHandlers = new ResponseHandlers(); + private final BytesReference pingMessage; public TcpTransport(String transportName, Settings settings, ThreadPool threadPool, BigArrays bigArrays, CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry, @@ -238,8 +231,6 @@ public TcpTransport(String transportName, Settings settings, ThreadPool threadPo this.threadPool = threadPool; this.bigArrays = bigArrays; this.circuitBreakerService = circuitBreakerService; - this.scheduledPing = new ScheduledPing(); - this.pingSchedule = PING_SCHEDULE.get(settings); this.namedWriteableRegistry = namedWriteableRegistry; this.compress = Transport.TRANSPORT_TCP_COMPRESS.get(settings); this.networkService = networkService; @@ -257,6 +248,15 @@ public TcpTransport(String transportName, Settings settings, ThreadPool threadPo // use a sorted set to present the features in a consistent order this.features = new TreeSet<>(defaultFeatures.names()).toArray(new String[defaultFeatures.names().size()]); } + + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.writeByte((byte) 'E'); + out.writeByte((byte) 'S'); + out.writeInt(TcpTransport.PING_DATA_SIZE); + pingMessage = out.bytes(); + } catch (IOException e) { + throw new AssertionError(e.getMessage(), e); // won't happen + } } static ConnectionProfile buildDefaultConnectionProfile(Settings settings) { @@ -280,9 +280,6 @@ static ConnectionProfile buildDefaultConnectionProfile(Settings settings) { @Override protected void doStart() { - if (pingSchedule.millis() > 0) { - threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, scheduledPing); - } } @Override @@ -344,93 +341,12 @@ public String executor() { } } - public class ScheduledPing extends AbstractLifecycleRunnable { - - /** - * The magic number (must be lower than 0) for a ping message. This is handled - * specifically in {@link TcpTransport#validateMessageHeader}. - */ - private final BytesReference pingHeader; - final CounterMetric successfulPings = new CounterMetric(); - final CounterMetric failedPings = new CounterMetric(); - - public ScheduledPing() { - super(lifecycle, logger); - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeByte((byte) 'E'); - out.writeByte((byte) 'S'); - out.writeInt(PING_DATA_SIZE); - pingHeader = out.bytes(); - } catch (IOException e) { - throw new IllegalStateException(e.getMessage(), e); // won't happen - } - } - - @Override - protected void doRunInLifecycle() throws Exception { - for (Map.Entry entry : connectedNodes.entrySet()) { - DiscoveryNode node = entry.getKey(); - NodeChannels channels = entry.getValue(); - for (TcpChannel channel : channels.getChannels()) { - internalSendMessage(channel, pingHeader, new SendMetricListener(pingHeader.length()) { - @Override - protected void innerInnerOnResponse(Void v) { - successfulPings.inc(); - } - - @Override - protected void innerOnFailure(Exception e) { - if (channel.isOpen()) { - logger.debug(() -> new ParameterizedMessage("[{}] failed to send ping transport message", node), e); - failedPings.inc(); - } else { - logger.trace(() -> - new ParameterizedMessage("[{}] failed to send ping transport message (channel closed)", node), e); - } - - } - }); - } - } - } - - public long getSuccessfulPings() { - return successfulPings.count(); - } - - public long getFailedPings() { - return failedPings.count(); - } - - @Override - protected void onAfterInLifecycle() { - try { - threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, this); - } catch (EsRejectedExecutionException ex) { - if (ex.isExecutorShutdown()) { - logger.debug("couldn't schedule new ping execution, executor is shutting down", ex); - } else { - throw ex; - } - } - } - - @Override - public void onFailure(Exception e) { - if (lifecycle.stoppedOrClosed()) { - logger.trace("failed to send ping transport message", e); - } else { - logger.warn("failed to send ping transport message", e); - } - } - } - - public final class NodeChannels implements Connection { + public final class NodeChannels extends CloseableConnection { private final Map typeMapping; private final List channels; private final DiscoveryNode node; - private final AtomicBoolean closed = new AtomicBoolean(false); private final Version version; + private final AtomicBoolean isClosing = new AtomicBoolean(false); NodeChannels(DiscoveryNode node, List channels, ConnectionProfile connectionProfile, Version handshakeVersion) { this.node = node; @@ -462,13 +378,38 @@ public TcpChannel channel(TransportRequestOptions.Type type) { return connectionTypeHandle.getChannel(channels); } - public boolean allChannelsOpen() { + boolean allChannelsOpen() { return channels.stream().allMatch(TcpChannel::isOpen); } + @Override + public boolean sendPing() { + for (TcpChannel channel : channels) { + internalSendMessage(channel, pingMessage, new SendMetricListener(pingMessage.length()) { + @Override + protected void innerInnerOnResponse(Void v) { + successfulPings.inc(); + } + + @Override + protected void innerOnFailure(Exception e) { + if (channel.isOpen()) { + logger.debug(() -> new ParameterizedMessage("[{}] failed to send ping transport message", node), e); + failedPings.inc(); + } else { + logger.trace(() -> + new ParameterizedMessage("[{}] failed to send ping transport message (channel closed)", node), e); + } + + } + }); + } + return true; + } + @Override public void close() { - if (closed.compareAndSet(false, true)) { + if (isClosing.compareAndSet(false, true)) { try { if (lifecycle.stopped()) { /* We set SO_LINGER timeout to 0 to ensure that when we shutdown the node we don't @@ -491,7 +432,8 @@ public void close() { boolean block = lifecycle.stopped() && Transports.isTransportThread(Thread.currentThread()) == false; TcpChannel.closeChannels(channels, block); } finally { - transportListener.onConnectionClosed(this); + // Call the super method to trigger listeners + super.close(); } } } @@ -504,81 +446,12 @@ public DiscoveryNode getNode() { @Override public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException { - if (closed.get()) { + if (isClosing.get()) { throw new NodeNotConnectedException(node, "connection already closed"); } TcpChannel channel = channel(options.type()); sendRequestToChannel(this.node, channel, requestId, action, request, options, getVersion(), (byte) 0); } - - boolean isClosed() { - return closed.get(); - } - } - - @Override - public boolean nodeConnected(DiscoveryNode node) { - return connectedNodes.containsKey(node); - } - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - connectionProfile = resolveConnectionProfile(connectionProfile); - if (node == null) { - throw new ConnectTransportException(null, "can't connect to a null node"); - } - closeLock.readLock().lock(); // ensure we don't open connections while we are closing - try { - ensureOpen(); - try (Releasable ignored = connectionLock.acquire(node.getId())) { - NodeChannels nodeChannels = connectedNodes.get(node); - if (nodeChannels != null) { - return; - } - boolean success = false; - try { - nodeChannels = openConnection(node, connectionProfile); - connectionValidator.accept(nodeChannels, connectionProfile); - // we acquire a connection lock, so no way there is an existing connection - connectedNodes.put(node, nodeChannels); - if (logger.isDebugEnabled()) { - logger.debug("connected to node [{}]", node); - } - try { - transportListener.onNodeConnected(node); - } finally { - if (nodeChannels.isClosed()) { - // we got closed concurrently due to a disconnect or some other event on the channel. - // the close callback will close the NodeChannel instance first and then try to remove - // the connection from the connected nodes. It will NOT acquire the connectionLock for - // the node to prevent any blocking calls on network threads. Yet, we still establish a happens - // before relationship to the connectedNodes.put since we check if we can remove the - // (DiscoveryNode, NodeChannels) tuple from the map after we closed. Here we check if it's closed an if so we - // try to remove it first either way one of the two wins even if the callback has run before we even added the - // tuple to the map since in that case we remove it here again - if (connectedNodes.remove(node, nodeChannels)) { - transportListener.onNodeDisconnected(node); - } - throw new NodeNotConnectedException(node, "connection concurrently closed"); - } - } - success = true; - } catch (ConnectTransportException e) { - throw e; - } catch (Exception e) { - throw new ConnectTransportException(node, "general node connection failure", e); - } finally { - if (success == false) { // close the connection if there is a failure - logger.trace(() -> new ParameterizedMessage("failed to connect to [{}], cleaning dangling connections", node)); - IOUtils.closeWhileHandlingException(nodeChannels); - } - } - } - } finally { - closeLock.readLock().unlock(); - } } /** @@ -609,7 +482,7 @@ protected ConnectionProfile resolveConnectionProfile(ConnectionProfile connectio } @Override - public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) { + public NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) { if (node == null) { throw new ConnectTransportException(null, "can't open connection to a null node"); } @@ -661,16 +534,16 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c // 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); - transportListener.onConnectionOpened(nodeChannels); final NodeChannels finalNodeChannels = nodeChannels; - final AtomicBoolean runOnce = new AtomicBoolean(false); + try { + transportListener.onConnectionOpened(nodeChannels); + } finally { + nodeChannels.addCloseListener(ActionListener.wrap(() -> transportListener.onConnectionClosed(finalNodeChannels))); + } + Consumer onClose = c -> { assert c.isOpen() == false : "channel is still open when onClose is called"; - // we only need to disconnect from the nodes once since all other channels - // will also try to run this we protect it from running multiple times. - if (runOnce.compareAndSet(false, true)) { - disconnectFromNodeCloseAndNotify(node, finalNodeChannels); - } + finalNodeChannels.close(); }; nodeChannels.channels.forEach(ch -> ch.addCloseListener(ActionListener.wrap(() -> onClose.accept(ch)))); @@ -696,46 +569,6 @@ public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile c } } - private void disconnectFromNodeCloseAndNotify(DiscoveryNode node, NodeChannels nodeChannels) { - assert nodeChannels != null : "nodeChannels must not be null"; - try { - IOUtils.closeWhileHandlingException(nodeChannels); - } finally { - if (closeLock.readLock().tryLock()) { - try { - if (connectedNodes.remove(node, nodeChannels)) { - transportListener.onNodeDisconnected(node); - } - } finally { - closeLock.readLock().unlock(); - } - } - } - } - - @Override - public NodeChannels getConnection(DiscoveryNode node) { - NodeChannels nodeChannels = connectedNodes.get(node); - if (nodeChannels == null) { - throw new NodeNotConnectedException(node, "Node not connected"); - } - return nodeChannels; - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - closeLock.readLock().lock(); - NodeChannels nodeChannels = null; - try (Releasable ignored = connectionLock.acquire(node.getId())) { - nodeChannels = connectedNodes.remove(node); - } finally { - closeLock.readLock().unlock(); - if (nodeChannels != null) { // if we found it and removed it we close and notify - IOUtils.closeWhileHandlingException(nodeChannels, () -> transportListener.onNodeDisconnected(node)); - } - } - } - protected Version getCurrentVersion() { // this is just for tests to mock stuff like the nodes version - tests can override this internally return Version.CURRENT; @@ -980,19 +813,6 @@ protected final void doStop() { TcpChannel.closeChannels(new ArrayList<>(acceptedChannels), true); acceptedChannels.clear(); - - // we are holding a write lock so nobody modifies the connectedNodes / openConnections map - it's safe to first close - // all instances and then clear them maps - Iterator> iterator = connectedNodes.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry next = iterator.next(); - try { - IOUtils.closeWhileHandlingException(next.getValue()); - transportListener.onNodeDisconnected(next.getKey()); - } finally { - iterator.remove(); - } - } stopInternal(); } finally { closeLock.writeLock().unlock(); @@ -1098,8 +918,7 @@ public boolean canCompress(TransportRequest request) { private void sendRequestToChannel(final DiscoveryNode node, final TcpChannel channel, final long requestId, final String action, final TransportRequest request, TransportRequestOptions options, Version channelVersion, - byte status) throws IOException, - TransportException { + byte status) throws IOException, TransportException { if (compress) { options = TransportRequestOptions.builder(options).withCompress(true).build(); } diff --git a/server/src/main/java/org/elasticsearch/transport/Transport.java b/server/src/main/java/org/elasticsearch/transport/Transport.java index 74235479657bf..9538119f43b8a 100644 --- a/server/src/main/java/org/elasticsearch/transport/Transport.java +++ b/server/src/main/java/org/elasticsearch/transport/Transport.java @@ -20,8 +20,8 @@ package org.elasticsearch.transport; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.component.LifecycleComponent; @@ -85,23 +85,6 @@ public interface Transport extends LifecycleComponent { */ TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException; - /** - * Returns {@code true} if the node is connected. - */ - boolean nodeConnected(DiscoveryNode node); - - /** - * Connects to a node with the given connection profile. If the node is already connected this method has no effect. - * Once a successful is established, it can be validated before being exposed. - */ - void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) throws ConnectTransportException; - - /** - * Disconnected from the given node, if not connected, will do nothing. - */ - void disconnectFromNode(DiscoveryNode node); - /** * Returns a list of all local adresses for this transport */ @@ -112,23 +95,10 @@ default CircuitBreaker getInFlightRequestBreaker() { } /** - * Returns a connection for the given node if the node is connected. - * Connections returned from this method must not be closed. The lifecycle of this connection is maintained by the Transport - * implementation. - * - * @throws NodeNotConnectedException if the node is not connected - * @see #connectToNode(DiscoveryNode, ConnectionProfile, CheckedBiConsumer) - */ - Connection getConnection(DiscoveryNode node); - - /** - * Opens a new connection to the given node and returns it. In contrast to - * {@link #connectToNode(DiscoveryNode, ConnectionProfile, CheckedBiConsumer)} the returned connection is not managed by + * Opens a new connection to the given node and returns it. The returned connection is not managed by * the transport implementation. This connection must be closed once it's not needed anymore. - * This connection type can be used to execute a handshake between two nodes before the node will be published via - * {@link #connectToNode(DiscoveryNode, ConnectionProfile, CheckedBiConsumer)}. */ - Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException; + Connection openConnection(DiscoveryNode node, ConnectionProfile profile); TransportStats getStats(); @@ -154,6 +124,21 @@ interface Connection extends Closeable { void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException; + default boolean sendPing() { + return false; + } + + /** + * The listener's {@link ActionListener#onResponse(Object)} method will be called when this + * connection is closed. No implementations currently throw an exception during close, so + * {@link ActionListener#onFailure(Exception)} will not be called. + * + * @param listener to be called + */ + void addCloseListener(ActionListener listener); + + boolean isClosed(); + /** * Returns the version of the node this connection was established with. */ @@ -168,6 +153,9 @@ default Version getVersion() { default Object getCacheKey() { return this; } + + @Override + void close(); } /** diff --git a/server/src/main/java/org/elasticsearch/transport/TransportService.java b/server/src/main/java/org/elasticsearch/transport/TransportService.java index d887519ee968b..aa1aaaf3b97a5 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportService.java @@ -21,11 +21,11 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; @@ -47,6 +47,7 @@ import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.tasks.TaskManager; @@ -79,6 +80,7 @@ public class TransportService extends AbstractLifecycleComponent implements Tran private final CountDownLatch blockIncomingRequestsLatch = new CountDownLatch(1); protected final Transport transport; + protected final ConnectionManager connectionManager; protected final ThreadPool threadPool; protected final ClusterName clusterName; protected final TaskManager taskManager; @@ -109,6 +111,7 @@ protected boolean removeEldestEntry(Map.Entry eldest) { Function.identity(), Property.Dynamic, Property.NodeScope); private final Logger tracerLog; + private final ConnectionProfile defaultConnectionProfile; volatile String[] tracerLogInclude; volatile String[] tracerLogExclude; @@ -131,6 +134,15 @@ public void sendRequest(long requestId, String action, TransportRequest request, sendLocalRequest(requestId, action, request, options); } + @Override + public void addCloseListener(ActionListener listener) { + } + + @Override + public boolean isClosed() { + return false; + } + @Override public void close() { } @@ -145,6 +157,13 @@ public void close() { public TransportService(Settings settings, Transport transport, ThreadPool threadPool, TransportInterceptor transportInterceptor, Function localNodeFactory, @Nullable ClusterSettings clusterSettings, Set taskHeaders) { + this(settings, transport, threadPool, transportInterceptor, localNodeFactory, clusterSettings, taskHeaders, + new ConnectionManager(settings, transport, threadPool)); + } + + public TransportService(Settings settings, Transport transport, ThreadPool threadPool, TransportInterceptor transportInterceptor, + Function localNodeFactory, @Nullable ClusterSettings clusterSettings, + Set taskHeaders, ConnectionManager connectionManager) { super(settings); // The only time we do not want to validate node connections is when this is a transport client using the simple node sampler this.validateConnections = TransportClient.CLIENT_TYPE.equals(settings.get(Client.CLIENT_TYPE_SETTING_S.getKey())) == false || @@ -152,6 +171,7 @@ public TransportService(Settings settings, Transport transport, ThreadPool threa this.transport = transport; this.threadPool = threadPool; this.localNodeFactory = localNodeFactory; + this.connectionManager = connectionManager; this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); setTracerLogInclude(TRACE_LOG_INCLUDE_SETTING.get(settings)); setTracerLogExclude(TRACE_LOG_EXCLUDE_SETTING.get(settings)); @@ -162,6 +182,7 @@ public TransportService(Settings settings, Transport transport, ThreadPool threa this.connectToRemoteCluster = RemoteClusterService.ENABLE_REMOTE_CLUSTERS.get(settings); remoteClusterService = new RemoteClusterService(settings, this); responseHandlers = transport.getResponseHandlers(); + defaultConnectionProfile = TcpTransport.buildDefaultConnectionProfile(settings); if (clusterSettings != null) { clusterSettings.addSettingsUpdateConsumer(TRACE_LOG_INCLUDE_SETTING, this::setTracerLogInclude); clusterSettings.addSettingsUpdateConsumer(TRACE_LOG_EXCLUDE_SETTING, this::setTracerLogExclude); @@ -231,6 +252,7 @@ protected void doStart() { @Override protected void doStop() { try { + connectionManager.close(); transport.stop(); } finally { // in case the transport is not connected to our local node (thus cleaned on node disconnect) @@ -304,7 +326,7 @@ public List getLocalAddresses() { * Returns true iff the given node is already connected. */ public boolean nodeConnected(DiscoveryNode node) { - return isLocalNode(node) || transport.nodeConnected(node); + return isLocalNode(node) || connectionManager.nodeConnected(node); } /** @@ -326,7 +348,9 @@ public void connectToNode(final DiscoveryNode node, ConnectionProfile connection if (isLocalNode(node)) { return; } - transport.connectToNode(node, connectionProfile, (newConnection, actualProfile) -> { + + ConnectionProfile resolvedProfile = TcpTransport.resolveConnectionProfile(connectionProfile, defaultConnectionProfile); + connectionManager.connectToNode(node, resolvedProfile, (newConnection, actualProfile) -> { // We don't validate cluster names to allow for CCS connections. final DiscoveryNode remote = handshake(newConnection, actualProfile.getHandshakeTimeout().millis(), cn -> true).discoveryNode; if (validateConnections && node.equals(remote) == false) { @@ -377,12 +401,11 @@ public DiscoveryNode handshake( * @param handshakeTimeout handshake timeout * @param clusterNamePredicate cluster name validation predicate * @return the handshake response - * @throws ConnectTransportException if the connection failed * @throws IllegalStateException if the handshake failed */ public HandshakeResponse handshake( final Transport.Connection connection, - final long handshakeTimeout, Predicate clusterNamePredicate) throws ConnectTransportException { + final long handshakeTimeout, Predicate clusterNamePredicate) { final HandshakeResponse response; final DiscoveryNode node = connection.getNode(); try { @@ -409,6 +432,10 @@ public HandshakeResponse newInstance() { return response; } + public ConnectionManager getConnectionManager() { + return connectionManager; + } + static class HandshakeRequest extends TransportRequest { public static final HandshakeRequest INSTANCE = new HandshakeRequest(); @@ -461,15 +488,17 @@ public void disconnectFromNode(DiscoveryNode node) { if (isLocalNode(node)) { return; } - transport.disconnectFromNode(node); + connectionManager.disconnectFromNode(node); } public void addConnectionListener(TransportConnectionListener listener) { transport.addConnectionListener(listener); + connectionManager.addListener(listener); } public void removeConnectionListener(TransportConnectionListener listener) { transport.removeConnectionListener(listener); + connectionManager.removeListener(listener); } public TransportFuture submitRequest(DiscoveryNode node, String action, TransportRequest request, @@ -532,7 +561,7 @@ public Transport.Connection getConnection(DiscoveryNode node) { if (isLocalNode(node)) { return localNodeConnection; } else { - return transport.getConnection(node); + return connectionManager.getConnection(node); } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java index 731397e043918..aefcebde9223a 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java @@ -63,7 +63,7 @@ public void setUp() throws Exception { clusterService = getInstanceFromNode(ClusterService.class); indicesService = getInstanceFromNode(IndicesService.class); CapturingTransport capturingTransport = new CapturingTransport(); - transportService = new TransportService(clusterService.getSettings(), capturingTransport, threadPool, + transportService = capturingTransport.createCapturingTransportService(clusterService.getSettings(), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); @@ -81,12 +81,10 @@ public void tearDown() throws Exception { public void testIncludeDefaults() { GetIndexRequest defaultsRequest = new GetIndexRequest().indices(indexName).includeDefaults(true); getIndexAction.execute(null, defaultsRequest, ActionListener.wrap( - defaultsResponse -> { - assertNotNull( - "index.refresh_interval should be set as we are including defaults", - defaultsResponse.getSetting(indexName, "index.refresh_interval") - ); - }, exception -> { + defaultsResponse -> assertNotNull( + "index.refresh_interval should be set as we are including defaults", + defaultsResponse.getSetting(indexName, "index.refresh_interval") + ), exception -> { throw new AssertionError(exception); }) ); @@ -95,12 +93,10 @@ public void testIncludeDefaults() { public void testDoNotIncludeDefaults() { GetIndexRequest noDefaultsRequest = new GetIndexRequest().indices(indexName); getIndexAction.execute(null, noDefaultsRequest, ActionListener.wrap( - noDefaultsResponse -> { - assertNull( - "index.refresh_interval should be null as it was never set", - noDefaultsResponse.getSetting(indexName, "index.refresh_interval") - ); - }, exception -> { + noDefaultsResponse -> assertNull( + "index.refresh_interval should be null as it was never set", + noDefaultsResponse.getSetting(indexName, "index.refresh_interval") + ), exception -> { throw new AssertionError(exception); }) ); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java index 11f0188c8c0b0..03ccebba10dbd 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java @@ -75,7 +75,7 @@ public void setUp() throws Exception { threadPool = new TestThreadPool("GetSettingsActionTests"); clusterService = createClusterService(threadPool); CapturingTransport capturingTransport = new CapturingTransport(); - transportService = new TransportService(clusterService.getSettings(), capturingTransport, threadPool, + transportService = capturingTransport.createCapturingTransportService(clusterService.getSettings(), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java index 3bd66af1bab05..a1abd4d61f7fd 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTests.java @@ -80,8 +80,8 @@ public void setUp() throws Exception { threadPool = new TestThreadPool("TransportBulkActionTookTests"); clusterService = createClusterService(threadPool); CapturingTransport capturingTransport = new CapturingTransport(); - transportService = new TransportService(clusterService.getSettings(), capturingTransport, threadPool, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, + transportService = capturingTransport.createCapturingTransportService(clusterService.getSettings(), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index af8289f0c45b1..af4346baa949a 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -91,8 +91,8 @@ public void tearDown() throws Exception { private TransportBulkAction createAction(boolean controlled, AtomicLong expected) { CapturingTransport capturingTransport = new CapturingTransport(); - TransportService transportService = new TransportService(clusterService.getSettings(), capturingTransport, threadPool, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, + TransportService transportService = capturingTransport.createCapturingTransportService(clusterService.getSettings(), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index 82e0fcaf5d667..5bea70bef17e0 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -442,7 +442,17 @@ public void sendRequest(long requestId, String action, TransportRequest request, } @Override - public void close() throws IOException { + public void addCloseListener(ActionListener listener) { + + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public void close() { throw new UnsupportedOperationException(); } } diff --git a/server/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java b/server/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java index 6a7d443553888..9bed5e7901704 100644 --- a/server/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java @@ -191,7 +191,7 @@ public void setUp() throws Exception { super.setUp(); transport = new CapturingTransport(); clusterService = createClusterService(THREAD_POOL); - final TransportService transportService = new TransportService(clusterService.getSettings(), transport, THREAD_POOL, + TransportService transportService = transport.createCapturingTransportService(clusterService.getSettings(), THREAD_POOL, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java b/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java index b27bc9ad79432..f592b3d803af5 100644 --- a/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java @@ -87,8 +87,8 @@ public void setUp() throws Exception { super.setUp(); transport = new CapturingTransport(); clusterService = createClusterService(threadPool); - transportService = new TransportService(clusterService.getSettings(), transport, threadPool, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); + transportService = transport.createCapturingTransportService(clusterService.getSettings(), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); localNode = new DiscoveryNode("local_node", buildNewFakeTransportAddress(), Collections.emptyMap(), diff --git a/server/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java b/server/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java index 60a46876a7126..3c5d3d3e1d2d6 100644 --- a/server/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java @@ -180,7 +180,7 @@ public void setUp() throws Exception { super.setUp(); transport = new CapturingTransport(); clusterService = createClusterService(THREAD_POOL); - transportService = new TransportService(clusterService.getSettings(), transport, THREAD_POOL, + transportService = transport.createCapturingTransportService(clusterService.getSettings(), THREAD_POOL, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); 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 6756c00ea84aa..7f1b4adf8df1e 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 @@ -163,7 +163,7 @@ public void setUp() throws Exception { super.setUp(); transport = new CapturingTransport(); clusterService = createClusterService(threadPool); - transportService = new TransportService(clusterService.getSettings(), transport, threadPool, + transportService = transport.createCapturingTransportService(clusterService.getSettings(), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index d305630f83ed7..9a571f1e9d8eb 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -255,8 +255,8 @@ public void testDocumentFailureInShardOperationOnReplica() throws Exception { public void testReplicaProxy() throws InterruptedException, ExecutionException { CapturingTransport transport = new CapturingTransport(); - TransportService transportService = new TransportService(clusterService.getSettings(), transport, threadPool, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); + TransportService transportService = transport.createCapturingTransportService(clusterService.getSettings(), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); ShardStateAction shardStateAction = new ShardStateAction(Settings.EMPTY, clusterService, transportService, null, null, threadPool); diff --git a/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java index 8db45cc5508ef..0505143592497 100644 --- a/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.ClusterStateCreationUtils; @@ -144,16 +143,15 @@ public void setUp() throws Exception { super.setUp(); transport = new CapturingTransport(); clusterService = createClusterService(THREAD_POOL); - transportService = new TransportService(clusterService.getSettings(), transport, THREAD_POOL, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet() - ); + transportService = transport.createCapturingTransportService(clusterService.getSettings(), THREAD_POOL, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); action = new TestTransportInstanceSingleOperationAction( Settings.EMPTY, "indices:admin/test", transportService, - new ActionFilters(new HashSet()), + new ActionFilters(new HashSet<>()), new MyResolver(), Request::new ); diff --git a/server/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java b/server/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java index 669a678b77e31..513f07b733cda 100644 --- a/server/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java +++ b/server/src/test/java/org/elasticsearch/client/transport/FailAndRetryMockTransport.java @@ -19,6 +19,7 @@ package org.elasticsearch.client.transport; +import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.node.liveness.LivenessResponse; import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; @@ -26,13 +27,13 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.transport.CloseableConnection; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.RequestHandlerRegistry; @@ -43,9 +44,9 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportStats; -import java.io.IOException; import java.net.UnknownHostException; import java.util.Collections; import java.util.Map; @@ -59,7 +60,7 @@ abstract class FailAndRetryMockTransport imp private final Random random; private final ClusterName clusterName; private volatile Map requestHandlers = Collections.emptyMap(); - final Object requestHandlerMutex = new Object(); + private final Object requestHandlerMutex = new Object(); private final ResponseHandlers responseHandlers = new ResponseHandlers(); private TransportConnectionListener listener; @@ -78,8 +79,9 @@ abstract class FailAndRetryMockTransport imp protected abstract ClusterState getMockClusterState(DiscoveryNode node); @Override - public Connection getConnection(DiscoveryNode node) { - return new Connection() { + public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) { + return new CloseableConnection() { + @Override public DiscoveryNode getNode() { return node; @@ -87,19 +89,22 @@ public DiscoveryNode getNode() { @Override public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws IOException, TransportException { - + throws TransportException { //we make sure that nodes get added to the connected ones when calling addTransportAddress, by returning proper nodes info if (connectMode) { if (TransportLivenessAction.NAME.equals(action)) { TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener); - transportResponseHandler.handleResponse(new LivenessResponse(ClusterName.CLUSTER_NAME_SETTING. - getDefault(Settings.EMPTY), - node)); + ClusterName clusterName = ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY); + transportResponseHandler.handleResponse(new LivenessResponse(clusterName, node)); } else if (ClusterStateAction.NAME.equals(action)) { TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener); ClusterState clusterState = getMockClusterState(node); transportResponseHandler.handleResponse(new ClusterStateResponse(clusterName, clusterState, 0L)); + } else if (TransportService.HANDSHAKE_ACTION_NAME.equals(action)) { + TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener); + Version version = node.getVersion(); + transportResponseHandler.handleResponse(new TransportService.HandshakeResponse(node, clusterName, version)); + } else { throw new UnsupportedOperationException("Mock transport does not understand action " + action); } @@ -129,19 +134,9 @@ public void sendRequest(long requestId, String action, TransportRequest request, } } } - - @Override - public void close() throws IOException { - - } }; } - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - return getConnection(node); - } - protected abstract Response newResponse(); public void endConnectMode() { @@ -175,23 +170,6 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi throw new UnsupportedOperationException(); } - @Override - public boolean nodeConnected(DiscoveryNode node) { - return false; - } - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - - } - @Override public Lifecycle.State lifecycleState() { return null; diff --git a/server/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/server/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java index 88db9f18d5c79..5fa0903febc11 100644 --- a/server/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; @@ -50,7 +49,6 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; import org.hamcrest.CustomMatcher; import java.io.Closeable; @@ -82,7 +80,7 @@ public class TransportClientNodesServiceTests extends ESTestCase { private static class TestIteration implements Closeable { private final ThreadPool threadPool; private final FailAndRetryMockTransport transport; - private final TransportService transportService; + private final MockTransportService transportService; private final TransportClientNodesService transportClientNodesService; private final int listNodesCount; private final int sniffNodesCount; @@ -142,7 +140,8 @@ protected ClusterState getMockClusterState(DiscoveryNode node) { return ClusterState.builder(clusterName).nodes(TestIteration.this.nodeMap.get(node.getAddress())).build(); } }; - transportService = new TransportService(settings, transport, threadPool, new TransportInterceptor() { + + transportService = new MockTransportService(settings, transport, threadPool, new TransportInterceptor() { @Override public AsyncSender interceptSender(AsyncSender sender) { return new AsyncSender() { @@ -164,6 +163,11 @@ public void sendRequest(Transport.Connection conne assert addr == null : "boundAddress: " + addr; return DiscoveryNode.createLocal(settings, buildNewFakeTransportAddress(), UUIDs.randomBase64UUID()); }, null, Collections.emptySet()); + transportService.addNodeConnectedBehavior((connectionManager, discoveryNode) -> false); + transportService.addGetConnectionBehavior((connectionManager, discoveryNode) -> { + // The FailAndRetryTransport does not use the connection profile + return transport.openConnection(discoveryNode, null); + }); transportService.start(); transportService.acceptIncomingRequests(); transportClientNodesService = @@ -355,25 +359,15 @@ public void testSniffNodesSamplerClosesConnections() throws Exception { .build(); try (MockTransportService clientService = createNewService(clientSettings, Version.CURRENT, threadPool, null)) { - final List establishedConnections = new CopyOnWriteArrayList<>(); - final List reusedConnections = new CopyOnWriteArrayList<>(); - - clientService.addDelegate(remoteService, new MockTransportService.DelegateTransport(clientService.original()) { - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - MockConnection connection = new MockConnection(super.openConnection(node, profile)); - establishedConnections.add(connection); - return connection; - } + final List establishedConnections = new CopyOnWriteArrayList<>(); - @Override - public Connection getConnection(DiscoveryNode node) { - MockConnection connection = new MockConnection(super.getConnection(node)); - reusedConnections.add(connection); - return connection; - } + clientService.addConnectBehavior(remoteService, (transport, discoveryNode, profile) -> { + Transport.Connection connection = transport.openConnection(discoveryNode, profile); + establishedConnections.add(connection); + return connection; }); + clientService.start(); clientService.acceptIncomingRequests(); @@ -381,27 +375,26 @@ public Connection getConnection(DiscoveryNode node) { new TransportClientNodesService(clientSettings, clientService, threadPool, (a, b) -> {})) { assertEquals(0, transportClientNodesService.connectedNodes().size()); assertEquals(0, establishedConnections.size()); - assertEquals(0, reusedConnections.size()); transportClientNodesService.addTransportAddresses(remoteService.getLocalDiscoNode().getAddress()); assertEquals(1, transportClientNodesService.connectedNodes().size()); - assertClosedConnections(establishedConnections, 1); + assertEquals(1, clientService.connectionManager().connectedNodeCount()); transportClientNodesService.doSample(); - assertClosedConnections(establishedConnections, 2); - assertOpenConnections(reusedConnections, 1); + assertEquals(1, clientService.connectionManager().connectedNodeCount()); + establishedConnections.clear(); handler.blockRequest(); Thread thread = new Thread(transportClientNodesService::doSample); thread.start(); - assertBusy(() -> assertEquals(3, establishedConnections.size())); - assertFalse("Temporary ping connection must be opened", establishedConnections.get(2).isClosed()); + assertBusy(() -> assertTrue(establishedConnections.size() >= 1)); + assertFalse("Temporary ping connection must be opened", establishedConnections.get(0).isClosed()); handler.releaseRequest(); thread.join(); - assertClosedConnections(establishedConnections, 3); + assertTrue(establishedConnections.get(0).isClosed()); } } } finally { @@ -409,56 +402,6 @@ public Connection getConnection(DiscoveryNode node) { } } - private void assertClosedConnections(final List connections, final int size) { - assertEquals("Expecting " + size + " closed connections but got " + connections.size(), size, connections.size()); - connections.forEach(c -> assertConnection(c, true)); - } - - private void assertOpenConnections(final List connections, final int size) { - assertEquals("Expecting " + size + " open connections but got " + connections.size(), size, connections.size()); - connections.forEach(c -> assertConnection(c, false)); - } - - private static void assertConnection(final MockConnection connection, final boolean closed) { - assertEquals("Connection [" + connection + "] must be " + (closed ? "closed" : "open"), closed, connection.isClosed()); - } - - class MockConnection implements Transport.Connection { - private final AtomicBoolean closed = new AtomicBoolean(false); - private final Transport.Connection connection; - - private MockConnection(Transport.Connection connection) { - this.connection = connection; - } - - @Override - public DiscoveryNode getNode() { - return connection.getNode(); - } - - @Override - public Version getVersion() { - return connection.getVersion(); - } - - @Override - public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws IOException, TransportException { - connection.sendRequest(requestId, action, request, options); - } - - @Override - public void close() throws IOException { - if (closed.compareAndSet(false, true)) { - connection.close(); - } - } - - boolean isClosed() { - return closed.get(); - } - } - class MockHandler implements TransportRequestHandler { private final AtomicBoolean block = new AtomicBoolean(false); private final CountDownLatch release = new CountDownLatch(1); diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterInfoServiceIT.java b/server/src/test/java/org/elasticsearch/cluster/ClusterInfoServiceIT.java index d7d232a08d43c..b17a0cc5418e9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterInfoServiceIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterInfoServiceIT.java @@ -46,12 +46,9 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import org.hamcrest.Matchers; -import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -116,23 +113,23 @@ public void blockActions(String... actions) { @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() - // manual collection or upon cluster forming. - .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2) - .put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.getKey(), "1s") - .build(); + // manual collection or upon cluster forming. + .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2) + .put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.getKey(), "1s") + .build(); } @Override protected Collection> nodePlugins() { return Arrays.asList(TestPlugin.class, - MockTransportService.TestPlugin.class); + MockTransportService.TestPlugin.class); } public void testClusterInfoServiceCollectsInformation() throws Exception { internalCluster().startNodes(2); assertAcked(prepareCreate("test").setSettings(Settings.builder() - .put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING.getKey(), 0) - .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE).build())); + .put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING.getKey(), 0) + .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE).build())); ensureGreen("test"); InternalTestCluster internalTestCluster = internalCluster(); // Get the cluster info service on the master node @@ -177,8 +174,8 @@ public void testClusterInfoServiceCollectsInformation() throws Exception { public void testClusterInfoServiceInformationClearOnError() throws InterruptedException, ExecutionException { internalCluster().startNodes(2, - // manually control publishing - Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(), "60m").build()); + // manually control publishing + Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(), "60m").build()); prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)).get(); ensureGreen("test"); InternalTestCluster internalTestCluster = internalCluster(); @@ -196,19 +193,16 @@ public void testClusterInfoServiceInformationClearOnError() throws InterruptedEx final Set blockedActions = newHashSet(NodesStatsAction.NAME, NodesStatsAction.NAME + "[n]", IndicesStatsAction.NAME, IndicesStatsAction.NAME + "[n]"); // drop all outgoing stats requests to force a timeout. for (DiscoveryNode node : internalTestCluster.clusterService().state().getNodes()) { - mockTransportService.addDelegate(internalTestCluster.getInstance(TransportService.class, node.getName()), new MockTransportService.DelegateTransport(mockTransportService.original()) { - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { + mockTransportService.addSendBehavior(internalTestCluster.getInstance(TransportService.class, node.getName()), + (connection, requestId, action, request, options) -> { if (blockedActions.contains(action)) { if (timeout.get()) { logger.info("dropping [{}] to [{}]", action, node); return; } } - super.sendRequest(connection, requestId, action, request, options); - } - }); + connection.sendRequest(requestId, action, request, options); + }); } // timeouts shouldn't clear the info diff --git a/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java index eb93dad5db8a0..92c1016d3e615 100644 --- a/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java @@ -20,16 +20,16 @@ package org.elasticsearch.cluster; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.LifecycleListener; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -39,6 +39,7 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; @@ -46,8 +47,6 @@ import org.junit.After; import org.junit.Before; -import java.io.IOException; -import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -56,6 +55,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Predicate; import static org.hamcrest.Matchers.equalTo; @@ -70,7 +71,7 @@ private List generateNodes() { for (int i = randomIntBetween(20, 50); i > 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))); nodes.add(new DiscoveryNode("node_" + i, "" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), - roles, Version.CURRENT)); + roles, Version.CURRENT)); } return nodes; } @@ -121,7 +122,7 @@ public void testReconnect() { for (int i = 0; i < 3; i++) { // simulate disconnects for (DiscoveryNode node : randomSubsetOf(nodes)) { - transport.disconnectFromNode(node); + transportService.disconnectFromNode(node); } service.new ConnectionChecker().run(); } @@ -134,18 +135,12 @@ public void testReconnect() { private void assertConnectedExactlyToNodes(ClusterState state) { assertConnected(state.nodes()); - assertThat(transport.connectedNodes.size(), equalTo(state.nodes().getSize())); + assertThat(transportService.getConnectionManager().connectedNodeCount(), equalTo(state.nodes().getSize())); } private void assertConnected(Iterable nodes) { for (DiscoveryNode node : nodes) { - assertTrue("not connected to " + node, transport.connectedNodes.contains(node)); - } - } - - private void assertNotConnected(Iterable nodes) { - for (DiscoveryNode node : nodes) { - assertFalse("still connected to " + node, transport.connectedNodes.contains(node)); + assertTrue("not connected to " + node, transportService.nodeConnected(node)); } } @@ -155,7 +150,8 @@ public void setUp() throws Exception { super.setUp(); this.threadPool = new TestThreadPool(getClass().getName()); this.transport = new MockTransport(); - transportService = new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, + transportService = new NoHandshakeTransportService(Settings.EMPTY, transport, threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> DiscoveryNode.createLocal(Settings.EMPTY, buildNewFakeTransportAddress(), UUIDs.randomBase64UUID()), null, Collections.emptySet()); transportService.start(); @@ -171,11 +167,29 @@ public void tearDown() throws Exception { super.tearDown(); } - final class MockTransport implements Transport { - Set connectedNodes = ConcurrentCollections.newConcurrentSet(); - volatile boolean randomConnectionExceptions = false; + private final class NoHandshakeTransportService extends TransportService { + + private NoHandshakeTransportService(Settings settings, + Transport transport, + ThreadPool threadPool, + TransportInterceptor transportInterceptor, + Function localNodeFactory, + ClusterSettings clusterSettings, + Set taskHeaders) { + super(settings, transport, threadPool, transportInterceptor, localNodeFactory, clusterSettings, taskHeaders); + } + + @Override + public HandshakeResponse handshake(Transport.Connection connection, long timeout, Predicate clusterNamePredicate) { + return new HandshakeResponse(connection.getNode(), new ClusterName(""), Version.CURRENT); + } + } + + private final class MockTransport implements Transport { private ResponseHandlers responseHandlers = new ResponseHandlers(); - private TransportConnectionListener listener = new TransportConnectionListener() {}; + private volatile boolean randomConnectionExceptions = false; + private TransportConnectionListener listener = new TransportConnectionListener() { + }; @Override public void registerRequestHandler(RequestHandlerRegistry reg) { @@ -207,37 +221,19 @@ public Map profileBoundAddresses() { } @Override - public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { + public TransportAddress[] addressesFromString(String address, int perAddressLimit) { return new TransportAddress[0]; } @Override - public boolean nodeConnected(DiscoveryNode node) { - return connectedNodes.contains(node); - } - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { + public Connection openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) { if (connectionProfile == null) { - if (connectedNodes.contains(node) == false && randomConnectionExceptions && randomBoolean()) { + if (randomConnectionExceptions && randomBoolean()) { throw new ConnectTransportException(node, "simulated"); } - connectedNodes.add(node); listener.onNodeConnected(node); } - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - connectedNodes.remove(node); - listener.onNodeDisconnected(node); - } - - @Override - public Connection getConnection(DiscoveryNode node) { - return new Connection() { + Connection connection = new Connection() { @Override public DiscoveryNode getNode() { return node; @@ -249,16 +245,21 @@ public void sendRequest(long requestId, String action, TransportRequest request, } + @Override + public void addCloseListener(ActionListener listener) { + + } + @Override public void close() { } - }; - } - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) { - Connection connection = getConnection(node); + @Override + public boolean isClosed() { + return false; + } + }; listener.onConnectionOpened(connection); return connection; } @@ -282,13 +283,16 @@ public void removeLifecycleListener(LifecycleListener listener) { } @Override - public void start() {} + public void start() { + } @Override - public void stop() {} + public void stop() { + } @Override - public void close() {} + public void close() { + } @Override public TransportStats getStats() { diff --git a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java index 1d78cdeb98374..64fa51d159a54 100644 --- a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java @@ -124,8 +124,8 @@ public void setUp() throws Exception { super.setUp(); this.transport = new CapturingTransport(); clusterService = createClusterService(THREAD_POOL); - transportService = new TransportService(clusterService.getSettings(), transport, THREAD_POOL, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); + transportService = transport.createCapturingTransportService(clusterService.getSettings(), THREAD_POOL, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); shardStateAction = new TestShardStateAction(Settings.EMPTY, clusterService, transportService, null, null); diff --git a/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index 0f5f4870ae1bb..32a8c06bb7020 100644 --- a/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -94,7 +94,8 @@ public static void setupThreadPool() { public void setUp() throws Exception { super.setUp(); clusterService = createClusterService(threadPool); - transportService = new TransportService(clusterService.getSettings(), new CapturingTransport(), threadPool, + CapturingTransport transport = new CapturingTransport(); + transportService = transport.createCapturingTransportService(clusterService.getSettings(), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java index f688291237813..4d190921f2da9 100644 --- a/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java @@ -37,13 +37,9 @@ import org.elasticsearch.test.disruption.SlowClusterStateProcessing; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -175,24 +171,15 @@ public void testClusterJoinDespiteOfPublishingIssues() throws Exception { logger.info("allowing requests from non master [{}] to master [{}], waiting for two join request", nonMasterNode, masterNode); final CountDownLatch countDownLatch = new CountDownLatch(2); - nonMasterTransportService.addDelegate(masterTranspotService, new MockTransportService.DelegateTransport(nonMasterTransportService - .original()) { - @Override - protected void sendRequest(Transport.Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - if (action.equals(MembershipAction.DISCOVERY_JOIN_ACTION_NAME)) { - countDownLatch.countDown(); - } - super.sendRequest(connection, requestId, action, request, options); + nonMasterTransportService.addSendBehavior(masterTransportService, (connection, requestId, action, request, options) -> { + if (action.equals(MembershipAction.DISCOVERY_JOIN_ACTION_NAME)) { + countDownLatch.countDown(); } - - @Override - public Transport.Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - return super.openConnection(node, profile); - } - + connection.sendRequest(requestId, action, request, options); }); + nonMasterTransportService.addConnectBehavior(masterTransportService, Transport::openConnection); + countDownLatch.await(); logger.info("waiting for cluster to reform"); 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 eef926a1e1238..4ab738f5c7bc3 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java @@ -21,7 +21,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.util.Constants; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -29,7 +28,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode.Role; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.network.NetworkAddress; @@ -42,14 +40,13 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.MockTcpTransport; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transport; @@ -152,14 +149,7 @@ public void testSimplePings() throws IOException, InterruptedException, Executio new NoneCircuitBreakerService(), new NamedWriteableRegistry(Collections.emptyList()), networkService, - v) { - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - throw new AssertionError("zen pings should never connect to node (got [" + node + "])"); - } - }; + v); NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier); closeables.push(handleA.transportService); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java index 596575abc3025..0888dfd3c40c5 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java @@ -17,12 +17,12 @@ package org.elasticsearch.index.seqno; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.shard.IndexShard; @@ -33,7 +33,6 @@ import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; import java.util.Collections; @@ -47,7 +46,7 @@ public class GlobalCheckpointSyncActionTests extends ESTestCase { private ThreadPool threadPool; - private Transport transport; + private CapturingTransport transport; private ClusterService clusterService; private TransportService transportService; private ShardStateAction shardStateAction; @@ -57,7 +56,7 @@ public void setUp() throws Exception { threadPool = new TestThreadPool(getClass().getName()); transport = new CapturingTransport(); clusterService = createClusterService(threadPool); - transportService = new TransportService(clusterService.getSettings(), transport, threadPool, + transportService = transport.createCapturingTransportService(clusterService.getSettings(), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> clusterService.localNode(), null, Collections.emptySet()); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java index b2c828cb73f0c..4f4f39c614687 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncIT.java @@ -34,11 +34,8 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -85,23 +82,14 @@ public void testBackgroundGlobalCheckpointSync() throws Exception { (MockTransportService) internalCluster().getInstance(TransportService.class, node.getName()); final MockTransportService receiverTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, other.getName()); - - senderTransportService.addDelegate(receiverTransportService, - new MockTransportService.DelegateTransport(senderTransportService.original()) { - @Override - protected void sendRequest( - final Connection connection, - final long requestId, - final String action, - final TransportRequest request, - final TransportRequestOptions options) throws IOException { - if ("indices:admin/seq_no/global_checkpoint_sync[r]".equals(action)) { - throw new IllegalStateException("blocking indices:admin/seq_no/global_checkpoint_sync[r]"); - } else { - super.sendRequest(connection, requestId, action, request, options); - } - } - }); + senderTransportService.addSendBehavior(receiverTransportService, + (connection, requestId, action, request, options) -> { + if ("indices:admin/seq_no/global_checkpoint_sync[r]".equals(action)) { + throw new IllegalStateException("blocking indices:admin/seq_no/global_checkpoint_sync[r]"); + } else { + connection.sendRequest(requestId, action, request, options); + } + }); } } }, diff --git a/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java b/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java index 8aaf6de1d6a00..3b5f77d72b6bb 100644 --- a/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java +++ b/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java @@ -70,8 +70,6 @@ import org.elasticsearch.test.MockIndexEventListener; import org.elasticsearch.test.store.MockFSIndexStore; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -338,19 +336,15 @@ public void testCorruptionOnNetworkLayerFinalizingRecovery() throws ExecutionExc final CountDownLatch hasCorrupted = new CountDownLatch(1); for (NodeStats dataNode : dataNodeStats) { MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance(TransportService.class, dataNode.getNode().getName())); - mockTransportService.addDelegate(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), new MockTransportService.DelegateTransport(mockTransportService.original()) { - - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException { - if (corrupt.get() && action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) { - RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request; - byte[] array = BytesRef.deepCopyOf(req.content().toBytesRef()).bytes; - int i = randomIntBetween(0, req.content().length() - 1); - array[i] = (byte) ~array[i]; // flip one byte in the content - hasCorrupted.countDown(); - } - super.sendRequest(connection, requestId, action, request, options); + mockTransportService.addSendBehavior(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), (connection, requestId, action, request, options) -> { + if (corrupt.get() && action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) { + RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request; + byte[] array = BytesRef.deepCopyOf(req.content().toBytesRef()).bytes; + int i = randomIntBetween(0, req.content().length() - 1); + array[i] = (byte) ~array[i]; // flip one byte in the content + hasCorrupted.countDown(); } + connection.sendRequest(requestId, action, request, options); }); } @@ -410,25 +404,21 @@ public void testCorruptionOnNetworkLayer() throws ExecutionException, Interrupte final boolean truncate = randomBoolean(); for (NodeStats dataNode : dataNodeStats) { MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance(TransportService.class, dataNode.getNode().getName())); - mockTransportService.addDelegate(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), new MockTransportService.DelegateTransport(mockTransportService.original()) { - - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException { - if (action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) { - RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request; - if (truncate && req.length() > 1) { - BytesRef bytesRef = req.content().toBytesRef(); - BytesArray array = new BytesArray(bytesRef.bytes, bytesRef.offset, (int) req.length() - 1); - request = new RecoveryFileChunkRequest(req.recoveryId(), req.shardId(), req.metadata(), req.position(), array, req.lastChunk(), req.totalTranslogOps(), req.sourceThrottleTimeInNanos()); - } else { - assert req.content().toBytesRef().bytes == req.content().toBytesRef().bytes : "no internal reference!!"; - final byte[] array = req.content().toBytesRef().bytes; - int i = randomIntBetween(0, req.content().length() - 1); - array[i] = (byte) ~array[i]; // flip one byte in the content - } + mockTransportService.addSendBehavior(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), (connection, requestId, action, request, options) -> { + if (action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) { + RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request; + if (truncate && req.length() > 1) { + BytesRef bytesRef = req.content().toBytesRef(); + BytesArray array = new BytesArray(bytesRef.bytes, bytesRef.offset, (int) req.length() - 1); + request = new RecoveryFileChunkRequest(req.recoveryId(), req.shardId(), req.metadata(), req.position(), array, req.lastChunk(), req.totalTranslogOps(), req.sourceThrottleTimeInNanos()); + } else { + assert req.content().toBytesRef().bytes == req.content().toBytesRef().bytes : "no internal reference!!"; + final byte[] array = req.content().toBytesRef().bytes; + int i = randomIntBetween(0, req.content().length() - 1); + array[i] = (byte) ~array[i]; // flip one byte in the content } - super.sendRequest(connection, requestId, action, request, options); } + connection.sendRequest(requestId, action, request, options); }); } diff --git a/server/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java b/server/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java index 24825b8a99ac2..25448d92d7e95 100644 --- a/server/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java +++ b/server/src/test/java/org/elasticsearch/index/store/ExceptionRetryIT.java @@ -38,8 +38,6 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -95,18 +93,14 @@ public void testRetryDueToExceptionOnNetworkLayer() throws ExecutionException, I for (NodeStats dataNode : nodeStats.getNodes()) { MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance(TransportService.class, dataNode.getNode().getName())); - mockTransportService.addDelegate(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), - new MockTransportService.DelegateTransport(mockTransportService.original()) { - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - super.sendRequest(connection, requestId, action, request, options); + mockTransportService.addSendBehavior(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), + (connection, requestId, action, request, options) -> { + connection.sendRequest(requestId, action, request, options); if (action.equals(TransportShardBulkAction.ACTION_NAME) && exceptionThrown.compareAndSet(false, true)) { logger.debug("Throw ConnectTransportException"); throw new ConnectTransportException(connection.getNode(), action); } - } - }); + }); } BulkRequestBuilder bulkBuilder = client.prepareBulk(); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index cf1449fecd6a5..89a8813e3e07b 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -61,6 +61,7 @@ import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSIndexStore; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.test.transport.StubbableTransport; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; @@ -598,8 +599,8 @@ public void testDisconnectsWhileRecovering() throws Exception { TransportService blueTransportService = internalCluster().getInstance(TransportService.class, blueNodeName); final CountDownLatch requestBlocked = new CountDownLatch(1); - blueMockTransportService.addDelegate(redTransportService, new RecoveryActionBlocker(dropRequests, recoveryActionToBlock, blueMockTransportService.original(), requestBlocked)); - redMockTransportService.addDelegate(blueTransportService, new RecoveryActionBlocker(dropRequests, recoveryActionToBlock, redMockTransportService.original(), requestBlocked)); + blueMockTransportService.addSendBehavior(redTransportService, new RecoveryActionBlocker(dropRequests, recoveryActionToBlock, requestBlocked)); + redMockTransportService.addSendBehavior(blueTransportService, new RecoveryActionBlocker(dropRequests, recoveryActionToBlock, requestBlocked)); logger.info("--> starting recovery from blue to red"); client().admin().indices().prepareUpdateSettings(indexName).setSettings( @@ -620,21 +621,20 @@ public void testDisconnectsWhileRecovering() throws Exception { } - private class RecoveryActionBlocker extends MockTransportService.DelegateTransport { + private class RecoveryActionBlocker implements StubbableTransport.SendRequestBehavior { private final boolean dropRequests; private final String recoveryActionToBlock; private final CountDownLatch requestBlocked; - RecoveryActionBlocker(boolean dropRequests, String recoveryActionToBlock, Transport delegate, CountDownLatch requestBlocked) { - super(delegate); + RecoveryActionBlocker(boolean dropRequests, String recoveryActionToBlock, CountDownLatch requestBlocked) { this.dropRequests = dropRequests; this.recoveryActionToBlock = recoveryActionToBlock; this.requestBlocked = requestBlocked; } @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { + public void sendRequest(Transport.Connection connection, long requestId, String action, TransportRequest request, + TransportRequestOptions options) throws IOException { if (recoveryActionToBlock.equals(action) || requestBlocked.getCount() == 0) { logger.info("--> preventing {} request", action); requestBlocked.countDown(); @@ -643,7 +643,7 @@ protected void sendRequest(Connection connection, long requestId, String action, } throw new ConnectTransportException(connection.getNode(), "DISCONNECT: prevented " + action + " request"); } - super.sendRequest(connection, requestId, action, request, options); + connection.sendRequest(requestId, action, request, options); } } @@ -686,12 +686,12 @@ public void testDisconnectsDuringRecovery() throws Exception { MockTransportService blueMockTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName); MockTransportService redMockTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, redNodeName); - redMockTransportService.addDelegate(blueMockTransportService, new MockTransportService.DelegateTransport(redMockTransportService.original()) { + redMockTransportService.addSendBehavior(blueMockTransportService, new StubbableTransport.SendRequestBehavior() { private final AtomicInteger count = new AtomicInteger(); @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { + public void sendRequest(Transport.Connection connection, long requestId, String action, TransportRequest request, + TransportRequestOptions options) throws IOException { logger.info("--> sending request {} on {}", action, connection.getNode()); if (PeerRecoverySourceService.Actions.START_RECOVERY.equals(action) && count.incrementAndGet() == 1) { // ensures that it's considered as valid recovery attempt by source @@ -701,7 +701,7 @@ protected void sendRequest(Connection connection, long requestId, String action, } catch (InterruptedException e) { throw new RuntimeException(e); } - super.sendRequest(connection, requestId, action, request, options); + connection.sendRequest(requestId, action, request, options); try { Thread.sleep(disconnectAfterDelay.millis()); } catch (InterruptedException e) { @@ -709,35 +709,27 @@ protected void sendRequest(Connection connection, long requestId, String action, } throw new ConnectTransportException(connection.getNode(), "DISCONNECT: simulation disconnect after successfully sending " + action + " request"); } else { - super.sendRequest(connection, requestId, action, request, options); + connection.sendRequest(requestId, action, request, options); } } }); final AtomicBoolean finalized = new AtomicBoolean(); - blueMockTransportService.addDelegate(redMockTransportService, new MockTransportService.DelegateTransport(blueMockTransportService.original()) { - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - logger.info("--> sending request {} on {}", action, connection.getNode()); - if (action.equals(PeerRecoveryTargetService.Actions.FINALIZE)) { - finalized.set(true); - } - super.sendRequest(connection, requestId, action, request, options); + blueMockTransportService.addSendBehavior(redMockTransportService, (connection, requestId, action, request, options) -> { + logger.info("--> sending request {} on {}", action, connection.getNode()); + if (action.equals(PeerRecoveryTargetService.Actions.FINALIZE)) { + finalized.set(true); } + connection.sendRequest(requestId, action, request, options); }); for (MockTransportService mockTransportService : Arrays.asList(redMockTransportService, blueMockTransportService)) { - mockTransportService.addDelegate(masterTransportService, new MockTransportService.DelegateTransport(mockTransportService.original()) { - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - logger.info("--> sending request {} on {}", action, connection.getNode()); - if ((primaryRelocation && finalized.get()) == false) { - assertNotEquals(action, ShardStateAction.SHARD_FAILED_ACTION_NAME); - } - super.sendRequest(connection, requestId, action, request, options); + mockTransportService.addSendBehavior(masterTransportService, (connection, requestId, action, request, options) -> { + logger.info("--> sending request {} on {}", action, connection.getNode()); + if ((primaryRelocation && finalized.get()) == false) { + assertNotEquals(action, ShardStateAction.SHARD_FAILED_ACTION_NAME); } + connection.sendRequest(requestId, action, request, options); }); } diff --git a/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java b/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java index 5c1c0d91e608b..7e99ccbbe6117 100644 --- a/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java +++ b/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java @@ -55,8 +55,6 @@ import org.elasticsearch.test.disruption.BlockClusterStateProcessing; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -234,16 +232,13 @@ public void testShardCleanupIfShardDeletionAfterRelocationFailedAndIndexDeleted( MockTransportService transportServiceNode_1 = (MockTransportService) internalCluster().getInstance(TransportService.class, node_1); TransportService transportServiceNode_2 = internalCluster().getInstance(TransportService.class, node_2); final CountDownLatch shardActiveRequestSent = new CountDownLatch(1); - transportServiceNode_1.addDelegate(transportServiceNode_2, new MockTransportService.DelegateTransport(transportServiceNode_1.original()) { - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException { - if (action.equals("internal:index/shard/exists") && shardActiveRequestSent.getCount() > 0) { - shardActiveRequestSent.countDown(); - logger.info("prevent shard active request from being sent"); - throw new ConnectTransportException(connection.getNode(), "DISCONNECT: simulated"); - } - super.sendRequest(connection, requestId, action, request, options); + transportServiceNode_1.addSendBehavior(transportServiceNode_2, (connection, requestId, action, request, options) -> { + if (action.equals("internal:index/shard/exists") && shardActiveRequestSent.getCount() > 0) { + shardActiveRequestSent.countDown(); + logger.info("prevent shard active request from being sent"); + throw new ConnectTransportException(connection.getNode(), "DISCONNECT: simulated"); } + connection.sendRequest(requestId, action, request, options); }); logger.info("--> move shard from {} to {}, and wait for relocation to finish", node_1, node_2); diff --git a/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 738cd00a43d2e..cb93d803bb7c6 100644 --- a/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -58,6 +58,7 @@ import org.elasticsearch.test.MockIndexEventListener; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.test.transport.StubbableTransport; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; @@ -372,7 +373,7 @@ public void testCancellationCleansTempFiles() throws Exception { MockTransportService mockTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, p_node); for (DiscoveryNode node : clusterService.state().nodes()) { if (!node.equals(clusterService.localNode())) { - mockTransportService.addDelegate(internalCluster().getInstance(TransportService.class, node.getName()), new RecoveryCorruption(mockTransportService.original(), corruptionCount)); + mockTransportService.addSendBehavior(internalCluster().getInstance(TransportService.class, node.getName()), new RecoveryCorruption(corruptionCount)); } } @@ -485,17 +486,16 @@ public void testIndexAndRelocateConcurrently() throws ExecutionException, Interr } - class RecoveryCorruption extends MockTransportService.DelegateTransport { + class RecoveryCorruption implements StubbableTransport.SendRequestBehavior { private final CountDownLatch corruptionCount; - RecoveryCorruption(Transport transport, CountDownLatch corruptionCount) { - super(transport); + RecoveryCorruption(CountDownLatch corruptionCount) { this.corruptionCount = corruptionCount; } @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException { + public void sendRequest(Transport.Connection connection, long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException { if (action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) { RecoveryFileChunkRequest chunkRequest = (RecoveryFileChunkRequest) request; if (chunkRequest.name().startsWith(IndexFileNames.SEGMENTS)) { @@ -506,9 +506,9 @@ protected void sendRequest(Connection connection, long requestId, String action, array[0] = (byte) ~array[0]; // flip one byte in the content corruptionCount.countDown(); } - super.sendRequest(connection, requestId, action, request, options); + connection.sendRequest(requestId, action, request, options); } else { - super.sendRequest(connection, requestId, action, request, options); + connection.sendRequest(requestId, action, request, options); } } } diff --git a/server/src/test/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java b/server/src/test/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java index b33cfb508b930..ac8688c9847d3 100644 --- a/server/src/test/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/recovery/TruncatedRecoveryIT.java @@ -35,11 +35,8 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -70,8 +67,8 @@ protected Collection> nodePlugins() { */ public void testCancelRecoveryAndResume() throws Exception { assertTrue(client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .put(CHUNK_SIZE_SETTING.getKey(), new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES))) - .get().isAcknowledged()); + .put(CHUNK_SIZE_SETTING.getKey(), new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES))) + .get().isAcknowledged()); NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().get(); List dataNodeStats = new ArrayList<>(); @@ -91,9 +88,9 @@ public void testCancelRecoveryAndResume() throws Exception { // create the index and prevent allocation on any other nodes than the lucky one // we have no replicas so far and make sure that we allocate the primary on the lucky node assertAcked(prepareCreate("test") - .addMapping("type1", "field1", "type=text", "the_id", "type=text") - .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards()) - .put("index.routing.allocation.include._name", primariesNode.getNode().getName()))); // only allocate on the lucky node + .addMapping("type1", "field1", "type=text", "the_id", "type=text") + .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards()) + .put("index.routing.allocation.include._name", primariesNode.getNode().getName()))); // only allocate on the lucky node // index some docs and check if they are coming back int numDocs = randomIntBetween(100, 200); @@ -116,11 +113,8 @@ public void testCancelRecoveryAndResume() throws Exception { final AtomicBoolean truncate = new AtomicBoolean(true); for (NodeStats dataNode : dataNodeStats) { MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance(TransportService.class, dataNode.getNode().getName())); - mockTransportService.addDelegate(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), new MockTransportService.DelegateTransport(mockTransportService.original()) { - - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { + mockTransportService.addSendBehavior(internalCluster().getInstance(TransportService.class, unluckyNode.getNode().getName()), + (connection, requestId, action, request, options) -> { if (action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) { RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request; logger.debug("file chunk [{}] lastChunk: {}", req, req.lastChunk()); @@ -129,16 +123,15 @@ protected void sendRequest(Connection connection, long requestId, String action, throw new RuntimeException("Caused some truncated files for fun and profit"); } } - super.sendRequest(connection, requestId, action, request, options); - } - }); + connection.sendRequest(requestId, action, request, options); + }); } logger.info("--> bumping replicas to 1"); // client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) - .put("index.routing.allocation.include._name", // now allow allocation on all nodes - primariesNode.getNode().getName() + "," + unluckyNode.getNode().getName())).get(); + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put("index.routing.allocation.include._name", // now allow allocation on all nodes + primariesNode.getNode().getName() + "," + unluckyNode.getNode().getName())).get(); latch.await(); diff --git a/server/src/test/java/org/elasticsearch/transport/ConnectionManagerTests.java b/server/src/test/java/org/elasticsearch/transport/ConnectionManagerTests.java new file mode 100644 index 0000000000000..64e8a42600458 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/transport/ConnectionManagerTests.java @@ -0,0 +1,164 @@ +/* + * 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.cluster.node.DiscoveryNode; +import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.net.InetAddress; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConnectionManagerTests extends ESTestCase { + + private ConnectionManager connectionManager; + private ThreadPool threadPool; + private Transport transport; + private ConnectionProfile connectionProfile; + + @Before + public void createConnectionManager() { + Settings settings = Settings.builder() + .put("node.name", ConnectionManagerTests.class.getSimpleName()) + .build(); + threadPool = new ThreadPool(settings); + transport = mock(Transport.class); + connectionManager = new ConnectionManager(settings, transport, threadPool); + TimeValue oneSecond = new TimeValue(1000); + connectionProfile = ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, oneSecond, oneSecond); + } + + @After + public void stopThreadPool() { + threadPool.shutdown(); + } + + public void testConnectAndDisconnect() { + AtomicInteger nodeConnectedCount = new AtomicInteger(); + AtomicInteger nodeDisconnectedCount = new AtomicInteger(); + connectionManager.addListener(new TransportConnectionListener() { + @Override + public void onNodeConnected(DiscoveryNode node) { + nodeConnectedCount.incrementAndGet(); + } + + @Override + public void onNodeDisconnected(DiscoveryNode node) { + nodeDisconnectedCount.incrementAndGet(); + } + }); + + + DiscoveryNode node = new DiscoveryNode("", new TransportAddress(InetAddress.getLoopbackAddress(), 0), Version.CURRENT); + Transport.Connection connection = new TestConnect(node); + when(transport.openConnection(node, connectionProfile)).thenReturn(connection); + + assertFalse(connectionManager.nodeConnected(node)); + + AtomicReference connectionRef = new AtomicReference<>(); + CheckedBiConsumer validator = (c, p) -> connectionRef.set(c); + connectionManager.connectToNode(node, connectionProfile, validator); + + assertFalse(connection.isClosed()); + assertTrue(connectionManager.nodeConnected(node)); + assertSame(connection, connectionManager.getConnection(node)); + assertEquals(1, connectionManager.connectedNodeCount()); + assertEquals(1, nodeConnectedCount.get()); + assertEquals(0, nodeDisconnectedCount.get()); + + if (randomBoolean()) { + connectionManager.disconnectFromNode(node); + } else { + connection.close(); + } + assertTrue(connection.isClosed()); + assertEquals(0, connectionManager.connectedNodeCount()); + assertEquals(1, nodeConnectedCount.get()); + assertEquals(1, nodeDisconnectedCount.get()); + } + + public void testConnectFails() { + AtomicInteger nodeConnectedCount = new AtomicInteger(); + AtomicInteger nodeDisconnectedCount = new AtomicInteger(); + connectionManager.addListener(new TransportConnectionListener() { + @Override + public void onNodeConnected(DiscoveryNode node) { + nodeConnectedCount.incrementAndGet(); + } + + @Override + public void onNodeDisconnected(DiscoveryNode node) { + nodeDisconnectedCount.incrementAndGet(); + } + }); + + + DiscoveryNode node = new DiscoveryNode("", new TransportAddress(InetAddress.getLoopbackAddress(), 0), Version.CURRENT); + Transport.Connection connection = new TestConnect(node); + when(transport.openConnection(node, connectionProfile)).thenReturn(connection); + + assertFalse(connectionManager.nodeConnected(node)); + + CheckedBiConsumer validator = (c, p) -> { + throw new ConnectTransportException(node, ""); + }; + + expectThrows(ConnectTransportException.class, () -> connectionManager.connectToNode(node, connectionProfile, validator)); + + assertTrue(connection.isClosed()); + assertFalse(connectionManager.nodeConnected(node)); + expectThrows(NodeNotConnectedException.class, () -> connectionManager.getConnection(node)); + assertEquals(0, connectionManager.connectedNodeCount()); + assertEquals(0, nodeConnectedCount.get()); + assertEquals(0, nodeDisconnectedCount.get()); + } + + private static class TestConnect extends CloseableConnection { + + private final DiscoveryNode node; + + private TestConnect(DiscoveryNode node) { + this.node = node; + } + + @Override + public DiscoveryNode getNode() { + return node; + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws TransportException { + + } + } +} diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java index f124d765bbf23..cbe5e4c7a3fa2 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java @@ -112,11 +112,11 @@ public static MockTransportService startTransport(String id, List } public static MockTransportService startTransport( - final String id, - final List knownNodes, - final Version version, - final ThreadPool threadPool, - final Settings settings) { + final String id, + final List knownNodes, + final Version version, + final ThreadPool threadPool, + final Settings settings) { boolean success = false; final Settings s = Settings.builder().put(settings).put("node.name", id).build(); ClusterName clusterName = ClusterName.CLUSTER_NAME_SETTING.get(s); @@ -443,19 +443,28 @@ public void sendRequest(long requestId, String action, TransportRequest request, } @Override - public void close() throws IOException { + public void addCloseListener(ActionListener listener) { // no-op } - }; - service.addDelegate(seedNode.getAddress(), new MockTransportService.DelegateTransport(service.getOriginalTransport()) { + @Override - public Connection getConnection(DiscoveryNode node) { - if (node == seedNode) { - return seedConnection; - } - return super.getConnection(node); + public boolean isClosed() { + return false; + } + + @Override + public void close() { + // no-op + } + }; + + service.addGetConnectionBehavior(seedNode.getAddress(), (connectionManager, discoveryNode) -> { + if (discoveryNode == seedNode) { + return seedConnection; } + return connectionManager.getConnection(discoveryNode); }); + service.start(); service.acceptIncomingRequests(); try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", @@ -518,7 +527,9 @@ public void run() { closeRemote.countDown(); listenerCalled.await(); assertNotNull(exceptionReference.get()); - expectThrows(CancellableThreads.ExecutionCancelledException.class, () -> {throw exceptionReference.get();}); + expectThrows(CancellableThreads.ExecutionCancelledException.class, () -> { + throw exceptionReference.get(); + }); } } @@ -552,7 +563,7 @@ public void testFetchShards() throws Exception { .indicesOptions(request.indicesOptions()).local(true).preference(request.preference()) .routing(request.routing()); connection.fetchSearchShards(searchShardsRequest, - new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); + new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); responseLatch.await(); assertNull(failReference.get()); assertNotNull(reference.get()); @@ -631,18 +642,18 @@ public void testFetchShardsSkipUnavailable() throws Exception { service.start(); service.acceptIncomingRequests(); try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", - Collections.singletonList(seedNode), service, Integer.MAX_VALUE, n -> true)) { + Collections.singletonList(seedNode), service, Integer.MAX_VALUE, n -> true)) { SearchRequest request = new SearchRequest("test-index"); ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest("test-index") - .indicesOptions(request.indicesOptions()).local(true).preference(request.preference()) - .routing(request.routing()); + .indicesOptions(request.indicesOptions()).local(true).preference(request.preference()) + .routing(request.routing()); { CountDownLatch responseLatch = new CountDownLatch(1); AtomicReference reference = new AtomicReference<>(); AtomicReference failReference = new AtomicReference<>(); connection.fetchSearchShards(searchShardsRequest, - new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); + new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); assertTrue(responseLatch.await(5, TimeUnit.SECONDS)); assertNull(failReference.get()); assertNotNull(reference.get()); @@ -671,7 +682,9 @@ public void onNodeDisconnected(DiscoveryNode node) { AtomicReference reference = new AtomicReference<>(); AtomicReference failReference = new AtomicReference<>(); connection.fetchSearchShards(searchShardsRequest, - new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); + new LatchedActionListener<>(ActionListener.wrap((s) -> { + reference.set(s); + }, failReference::set), responseLatch)); assertTrue(responseLatch.await(1, TimeUnit.SECONDS)); assertNotNull(failReference.get()); assertNull(reference.get()); @@ -684,7 +697,7 @@ public void onNodeDisconnected(DiscoveryNode node) { AtomicReference reference = new AtomicReference<>(); AtomicReference failReference = new AtomicReference<>(); connection.fetchSearchShards(searchShardsRequest, - new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); + new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); assertTrue(responseLatch.await(1, TimeUnit.SECONDS)); assertNull(failReference.get()); assertNotNull(reference.get()); @@ -707,7 +720,7 @@ public void onNodeDisconnected(DiscoveryNode node) { AtomicReference reference = new AtomicReference<>(); AtomicReference failReference = new AtomicReference<>(); connection.fetchSearchShards(searchShardsRequest, - new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); + new LatchedActionListener<>(ActionListener.wrap(reference::set, failReference::set), responseLatch)); assertTrue(responseLatch.await(1, TimeUnit.SECONDS)); assertNull(failReference.get()); assertNotNull(reference.get()); @@ -754,27 +767,28 @@ public void run() { for (int i = 0; i < numConnectionAttempts; i++) { AtomicBoolean executed = new AtomicBoolean(false); ActionListener listener = ActionListener.wrap( - x -> { - assertTrue(executed.compareAndSet(false, true)); - latch.countDown();}, - x -> { - /* - * This can occur on a thread submitted to the thread pool while we are closing the - * remote cluster connection at the end of the test. - */ - if (x instanceof CancellableThreads.ExecutionCancelledException) { - // we should already be shutting down - assertTrue(executed.get()); - return; - } + x -> { + assertTrue(executed.compareAndSet(false, true)); + latch.countDown(); + }, + x -> { + /* + * This can occur on a thread submitted to the thread pool while we are closing the + * remote cluster connection at the end of the test. + */ + if (x instanceof CancellableThreads.ExecutionCancelledException) { + // we should already be shutting down + assertTrue(executed.get()); + return; + } - assertTrue(executed.compareAndSet(false, true)); - latch.countDown(); + assertTrue(executed.compareAndSet(false, true)); + latch.countDown(); - if (!(x instanceof RejectedExecutionException)) { - throw new AssertionError(x); - } - }); + if (!(x instanceof RejectedExecutionException)) { + throw new AssertionError(x); + } + }); connection.updateSeedNodes(seedNodes, listener); } latch.await(); @@ -861,7 +875,7 @@ public void run() { } else { throw new AssertionError(x); } - }); + }); try { connection.updateSeedNodes(seedNodes, listener); } catch (Exception e) { @@ -954,6 +968,30 @@ public void testGetConnectionInfo() throws Exception { } } + private RemoteConnectionInfo getRemoteConnectionInfo(RemoteClusterConnection connection) throws Exception { + AtomicReference statsRef = new AtomicReference<>(); + AtomicReference exceptionRef = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + connection.getConnectionInfo(new ActionListener() { + @Override + public void onResponse(RemoteConnectionInfo remoteConnectionInfo) { + statsRef.set(remoteConnectionInfo); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + latch.countDown(); + } + }); + latch.await(); + if (exceptionRef.get() != null) { + throw exceptionRef.get(); + } + return statsRef.get(); + } + public void testRemoteConnectionInfo() throws IOException { RemoteConnectionInfo stats = new RemoteConnectionInfo("test_cluster", Arrays.asList(new TransportAddress(TransportAddress.META_ADDRESS, 1)), @@ -1013,7 +1051,7 @@ private static RemoteConnectionInfo assertSerialization(RemoteConnectionInfo inf RemoteConnectionInfo remoteConnectionInfo = new RemoteConnectionInfo(in); assertEquals(info, remoteConnectionInfo); assertEquals(info.hashCode(), remoteConnectionInfo.hashCode()); - return randomBoolean() ? info : remoteConnectionInfo; + return randomBoolean() ? info : remoteConnectionInfo; } } @@ -1078,34 +1116,10 @@ public void testRenderConnectionInfoXContent() throws IOException { "\"skip_unavailable\":false}}", Strings.toString(builder)); } - private RemoteConnectionInfo getRemoteConnectionInfo(RemoteClusterConnection connection) throws Exception { - AtomicReference statsRef = new AtomicReference<>(); - AtomicReference exceptionRef = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - connection.getConnectionInfo(new ActionListener() { - @Override - public void onResponse(RemoteConnectionInfo remoteConnectionInfo) { - statsRef.set(remoteConnectionInfo); - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - exceptionRef.set(e); - latch.countDown(); - } - }); - latch.await(); - if (exceptionRef.get() != null) { - throw exceptionRef.get(); - } - return statsRef.get(); - } - public void testEnsureConnected() throws IOException, InterruptedException { List knownNodes = new CopyOnWriteArrayList<>(); try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT); - MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) { + MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) { DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode(); knownNodes.add(seedTransport.getLocalDiscoNode()); @@ -1200,7 +1214,7 @@ public void testConnectedNodesConcurrentAccess() throws IOException, Interrupted try { final int numDiscoverableNodes = randomIntBetween(5, 20); List discoverableNodes = new ArrayList<>(numDiscoverableNodes); - for (int i = 0; i < numDiscoverableNodes; i++ ) { + for (int i = 0; i < numDiscoverableNodes; i++) { MockTransportService transportService = startTransport("discoverable_node" + i, knownNodes, Version.CURRENT); discoverableNodes.add(transportService.getLocalDiscoNode()); discoverableTransports.add(transportService); @@ -1285,12 +1299,12 @@ public void testClusterNameIsChecked() throws Exception { Settings settings = Settings.builder().put("cluster.name", "testClusterNameIsChecked").build(); try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT, threadPool, settings); - MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT, threadPool, - settings); - MockTransportService otherClusterTransport = startTransport("other_cluster_discoverable_node", otherClusterKnownNodes, - Version.CURRENT, threadPool, Settings.builder().put("cluster.name", "otherCluster").build()); - MockTransportService otherClusterDiscoverable= startTransport("other_cluster_discoverable_node", otherClusterKnownNodes, - Version.CURRENT, threadPool, Settings.builder().put("cluster.name", "otherCluster").build())) { + MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT, threadPool, + settings); + MockTransportService otherClusterTransport = startTransport("other_cluster_discoverable_node", otherClusterKnownNodes, + Version.CURRENT, threadPool, Settings.builder().put("cluster.name", "otherCluster").build()); + MockTransportService otherClusterDiscoverable = startTransport("other_cluster_discoverable_node", otherClusterKnownNodes, + Version.CURRENT, threadPool, Settings.builder().put("cluster.name", "otherCluster").build())) { DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode(); knownNodes.add(seedTransport.getLocalDiscoNode()); @@ -1350,33 +1364,39 @@ public DiscoveryNode getNode() { @Override public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws TransportException { + throws TransportException { // no-op } @Override - public void close() { + public void addCloseListener(ActionListener listener) { // no-op } - }; - service.addDelegate(connectedNode.getAddress(), new MockTransportService.DelegateTransport(service.getOriginalTransport()) { + @Override - public Connection getConnection(DiscoveryNode node) { - if (node == connectedNode) { - return seedConnection; - } - return super.getConnection(node); + public boolean isClosed() { + return false; } @Override - public boolean nodeConnected(DiscoveryNode node) { - return node.equals(connectedNode); + public void close() { + // no-op + } + }; + + service.addNodeConnectedBehavior(connectedNode.getAddress(), (connectionManager, discoveryNode) + -> discoveryNode.equals(connectedNode)); + + service.addGetConnectionBehavior(connectedNode.getAddress(), (connectionManager, discoveryNode) -> { + if (discoveryNode == connectedNode) { + return seedConnection; } + return connectionManager.getConnection(discoveryNode); }); service.start(); service.acceptIncomingRequests(); try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", - Collections.singletonList(connectedNode), service, Integer.MAX_VALUE, n -> true)) { + Collections.singletonList(connectedNode), service, Integer.MAX_VALUE, n -> true)) { connection.addConnectedNode(connectedNode); for (int i = 0; i < 10; i++) { //always a direct connection as the remote node is already connected diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java index e361dc7894308..a1d078211d103 100644 --- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java +++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java @@ -192,7 +192,11 @@ protected FakeChannel initiateChannel(DiscoveryNode node, TimeValue connectTimeo } @Override - public NodeChannels getConnection(DiscoveryNode node) { + protected void stopInternal() { + } + + @Override + public NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) { int numConnections = MockTcpTransport.LIGHT_PROFILE.getNumConnections(); ArrayList fakeChannels = new ArrayList<>(numConnections); for (int i = 0; i < numConnections; ++i) { @@ -203,7 +207,7 @@ public NodeChannels getConnection(DiscoveryNode node) { }; DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT); - Transport.Connection connection = transport.getConnection(node); + Transport.Connection connection = transport.openConnection(node, null); connection.sendRequest(42, "foobar", request, TransportRequestOptions.EMPTY); BytesReference reference = messageCaptor.get(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java b/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java index ffdf79c0636b2..a7d3e85d3009d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/CapturingTransport.java @@ -20,18 +20,22 @@ package org.elasticsearch.test.transport; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.transport.ConnectTransportException; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.ConnectionManager; import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.RequestHandlerRegistry; @@ -39,13 +43,14 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportStats; import java.io.IOException; -import java.io.UncheckedIOException; import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; @@ -53,9 +58,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; import static org.apache.lucene.util.LuceneTestCase.rarely; @@ -84,6 +91,45 @@ public CapturedRequest(DiscoveryNode node, long requestId, String action, Transp private ConcurrentMap> requests = new ConcurrentHashMap<>(); private BlockingQueue capturedRequests = ConcurrentCollections.newBlockingQueue(); + public TransportService createCapturingTransportService(Settings settings, ThreadPool threadPool, TransportInterceptor interceptor, + Function localNodeFactory, + @Nullable ClusterSettings clusterSettings, Set taskHeaders) { + StubbableConnectionManager connectionManager = new StubbableConnectionManager(new ConnectionManager(settings, this, threadPool), + settings, this, threadPool); + connectionManager.setDefaultNodeConnectedBehavior((cm, discoveryNode) -> true); + connectionManager.setDefaultConnectBehavior((cm, discoveryNode) -> new Connection() { + @Override + public DiscoveryNode getNode() { + return discoveryNode; + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws TransportException { + requests.put(requestId, Tuple.tuple(discoveryNode, action)); + capturedRequests.add(new CapturedRequest(discoveryNode, requestId, action, request)); + } + + @Override + public void addCloseListener(ActionListener listener) { + + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public void close() { + + } + }); + return new TransportService(settings, this, threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders, + connectionManager); + + } + /** returns all requests captured so far. Doesn't clear the captured request list. See {@link #clear()} */ public CapturedRequest[] capturedRequests() { return capturedRequests.toArray(new CapturedRequest[0]); @@ -195,7 +241,7 @@ public void handleError(final long requestId, final TransportException e) { } @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { + public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) { return new Connection() { @Override public DiscoveryNode getNode() { @@ -204,13 +250,23 @@ public DiscoveryNode getNode() { @Override public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws IOException, TransportException { + throws TransportException { requests.put(requestId, Tuple.tuple(node, action)); capturedRequests.add(new CapturedRequest(node, requestId, action, request)); } @Override - public void close() throws IOException { + public void addCloseListener(ActionListener listener) { + + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public void close() { } }; @@ -236,23 +292,6 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi return new TransportAddress[0]; } - @Override - public boolean nodeConnected(DiscoveryNode node) { - return true; - } - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - - } - @Override public Lifecycle.State lifecycleState() { return null; @@ -282,14 +321,6 @@ public List getLocalAddresses() { return Collections.emptyList(); } - public Connection getConnection(DiscoveryNode node) { - try { - return openConnection(node, null); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - @Override public void registerRequestHandler(RequestHandlerRegistry reg) { synchronized (requestHandlerMutex) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index 7f818de29d430..40a6ad6476d4a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -20,15 +20,12 @@ package org.elasticsearch.test.transport; import com.carrotsearch.randomizedtesting.SysGlobals; - import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.component.Lifecycle; -import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; @@ -40,7 +37,6 @@ 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.ConcurrentCollections; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; @@ -48,21 +44,18 @@ import org.elasticsearch.test.tasks.MockTaskManager; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; +import org.elasticsearch.transport.ConnectionManager; import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.MockTcpTransport; import org.elasticsearch.transport.RequestHandlerRegistry; import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportConnectionListener; -import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.TransportStats; import java.io.IOException; -import java.net.UnknownHostException; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -71,18 +64,18 @@ import java.util.Map; import java.util.Queue; import java.util.Set; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.function.Supplier; /** - * A mock transport service that allows to simulate different network topology failures. + * A mock delegate service that allows to simulate different network topology failures. * Internally it maps TransportAddress objects to rules that inject failures. * Adding rules for a node is done by adding rules for all bound addresses of a node * (and the publish address, if different). - * Matching requests to rules is based on the transport address associated with the + * Matching requests to rules is based on the delegate address associated with the * discovery node of the request, namely by DiscoveryNode.getAddress(). * This address is usually the publish address of the node but can also be a different one * (for example, @see org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing, which constructs @@ -147,9 +140,15 @@ public MockTransportService(Settings settings, Transport transport, ThreadPool t public MockTransportService(Settings settings, Transport transport, ThreadPool threadPool, TransportInterceptor interceptor, Function localNodeFactory, @Nullable ClusterSettings clusterSettings, Set taskHeaders) { - super(settings, new LookupTestTransport(transport), threadPool, interceptor, localNodeFactory, clusterSettings, - taskHeaders); - this.original = transport; + this(settings, new StubbableTransport(transport), threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders); + } + + private MockTransportService(Settings settings, StubbableTransport transport, ThreadPool threadPool, TransportInterceptor interceptor, + Function localNodeFactory, + @Nullable ClusterSettings clusterSettings, Set taskHeaders) { + super(settings, transport, threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders, + new StubbableConnectionManager(new ConnectionManager(settings, transport, threadPool), settings, transport, threadPool)); + this.original = transport.getDelegate(); } public static TransportAddress[] extractTransportAddresses(TransportService transportService) { @@ -173,11 +172,12 @@ protected TaskManager createTaskManager(Settings settings, ThreadPool threadPool * Clears all the registered rules. */ public void clearAllRules() { - transport().transports.clear(); + transport().clearBehaviors(); + connectionManager().clearBehaviors(); } /** - * Clears the rule associated with the provided transport service. + * Clears the rule associated with the provided delegate service. */ public void clearRule(TransportService transportService) { for (TransportAddress transportAddress : extractTransportAddresses(transportService)) { @@ -186,20 +186,11 @@ public void clearRule(TransportService transportService) { } /** - * Clears the rule associated with the provided transport address. + * Clears the rule associated with the provided delegate address. */ public void clearRule(TransportAddress transportAddress) { - Transport transport = transport().transports.remove(transportAddress); - if (transport instanceof ClearableTransport) { - ((ClearableTransport) transport).clearRule(); - } - } - - /** - * Returns the original Transport service wrapped by this mock transport service. - */ - public Transport original() { - return original; + transport().clearBehavior(transportAddress); + connectionManager().clearBehavior(transportAddress); } /** @@ -217,30 +208,14 @@ public void addFailToSendNoConnectRule(TransportService transportService) { * is added to fail as well. */ public void addFailToSendNoConnectRule(TransportAddress transportAddress) { - addDelegate(transportAddress, new DelegateTransport(original) { - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - if (original.nodeConnected(node) == false) { - // connecting to an already connected node is a no-op - throw new ConnectTransportException(node, "DISCONNECT: simulated"); - } - } - - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - throw new ConnectTransportException(node, "DISCONNECT: simulated"); - } + transport().addConnectBehavior(transportAddress, (transport, discoveryNode, profile) -> { + throw new ConnectTransportException(discoveryNode, "DISCONNECT: simulated"); + }); - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - connection.close(); - // send the request, which will blow up - connection.sendRequest(requestId, action, request, options); - } + transport().addSendBehavior(transportAddress, (connection, requestId, action, request, options) -> { + connection.close(); + // send the request, which will blow up + connection.sendRequest(requestId, action, request, options); }); } @@ -271,18 +246,12 @@ public void addFailToSendNoConnectRule(TransportService transportService, final * Adds a rule that will cause matching operations to throw ConnectTransportExceptions */ public void addFailToSendNoConnectRule(TransportAddress transportAddress, final Set blockedActions) { - - addDelegate(transportAddress, new DelegateTransport(original) { - - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - if (blockedActions.contains(action)) { - logger.info("--> preventing {} request", action); - connection.close(); - } - connection.sendRequest(requestId, action, request, options); + transport().addSendBehavior(transportAddress, (connection, requestId, action, request, options) -> { + if (blockedActions.contains(action)) { + logger.info("--> preventing {} request", action); + connection.close(); } + connection.sendRequest(requestId, action, request, options); }); } @@ -301,28 +270,12 @@ public void addUnresponsiveRule(TransportService transportService) { * and failing to connect once the rule was added. */ public void addUnresponsiveRule(TransportAddress transportAddress) { - addDelegate(transportAddress, new DelegateTransport(original) { - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - if (original.nodeConnected(node) == false) { - // connecting to an already connected node is a no-op - throw new ConnectTransportException(node, "UNRESPONSIVE: simulated"); - } - } - - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - throw new ConnectTransportException(node, "UNRESPONSIVE: simulated"); - } + transport().addConnectBehavior(transportAddress, (transport, discoveryNode, profile) -> { + throw new ConnectTransportException(discoveryNode, "UNRESPONSIVE: simulated"); + }); - @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - // don't send anything, the receiving node is unresponsive - } + transport().addSendBehavior(transportAddress, (connection, requestId, action, request, options) -> { + // don't send anything, the receiving node is unresponsive }); } @@ -347,70 +300,38 @@ public void addUnresponsiveRule(TransportService transportService, final TimeVal public void addUnresponsiveRule(TransportAddress transportAddress, final TimeValue duration) { final long startTime = System.currentTimeMillis(); - addDelegate(transportAddress, new ClearableTransport(original) { - private final Queue requestsToSendWhenCleared = new LinkedBlockingDeque<>(); - private boolean cleared = false; + Supplier delaySupplier = () -> new TimeValue(duration.millis() - (System.currentTimeMillis() - startTime)); - TimeValue getDelay() { - return new TimeValue(duration.millis() - (System.currentTimeMillis() - startTime)); + transport().addConnectBehavior(transportAddress, (transport, discoveryNode, profile) -> { + TimeValue delay = delaySupplier.get(); + if (delay.millis() <= 0) { + return original.openConnection(discoveryNode, profile); } - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - if (original.nodeConnected(node)) { - // connecting to an already connected node is a no-op - return; - } - TimeValue delay = getDelay(); - if (delay.millis() <= 0) { - original.connectToNode(node, connectionProfile, connectionValidator); - return; - } - - // TODO: Replace with proper setting - TimeValue connectingTimeout = TcpTransport.TCP_CONNECT_TIMEOUT.getDefault(Settings.EMPTY); - try { - if (delay.millis() < connectingTimeout.millis()) { - Thread.sleep(delay.millis()); - original.connectToNode(node, connectionProfile, connectionValidator); - } else { - Thread.sleep(connectingTimeout.millis()); - throw new ConnectTransportException(node, "UNRESPONSIVE: simulated"); - } - } catch (InterruptedException e) { - throw new ConnectTransportException(node, "UNRESPONSIVE: simulated"); + // TODO: Replace with proper setting + TimeValue connectingTimeout = TcpTransport.TCP_CONNECT_TIMEOUT.getDefault(Settings.EMPTY); + try { + if (delay.millis() < connectingTimeout.millis()) { + Thread.sleep(delay.millis()); + return original.openConnection(discoveryNode, profile); + } else { + Thread.sleep(connectingTimeout.millis()); + throw new ConnectTransportException(discoveryNode, "UNRESPONSIVE: simulated"); } + } catch (InterruptedException e) { + throw new ConnectTransportException(discoveryNode, "UNRESPONSIVE: simulated"); } + }); - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - TimeValue delay = getDelay(); - if (delay.millis() <= 0) { - return original.openConnection(node, profile); - } - - // TODO: Replace with proper setting - TimeValue connectingTimeout = TcpTransport.TCP_CONNECT_TIMEOUT.getDefault(Settings.EMPTY); - try { - if (delay.millis() < connectingTimeout.millis()) { - Thread.sleep(delay.millis()); - return original.openConnection(node, profile); - } else { - Thread.sleep(connectingTimeout.millis()); - throw new ConnectTransportException(node, "UNRESPONSIVE: simulated"); - } - } catch (InterruptedException e) { - throw new ConnectTransportException(node, "UNRESPONSIVE: simulated"); - } - } + transport().addSendBehavior(transportAddress, new StubbableTransport.SendRequestBehavior() { + private final Queue requestsToSendWhenCleared = new LinkedBlockingDeque<>(); + private boolean cleared = false; @Override - protected void sendRequest(Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { + public void sendRequest(Transport.Connection connection, long requestId, String action, TransportRequest request, + TransportRequestOptions options) throws IOException { // delayed sending - even if larger then the request timeout to simulated a potential late response from target node - TimeValue delay = getDelay(); + TimeValue delay = delaySupplier.get(); if (delay.millis() <= 0) { connection.sendRequest(requestId, action, request, options); return; @@ -450,7 +371,7 @@ protected void doRun() throws IOException { } @Override - public void clearRule() { + public void clearCallback() { synchronized (this) { assert cleared == false; cleared = true; @@ -461,234 +382,128 @@ public void clearRule() { } /** - * Adds a new delegate transport that is used for communication with the given transport service. + * Adds a new send behavior that is used for communication with the given delegate service. * - * @return {@code true} iff no other delegate was registered for any of the addresses bound by transport service. + * @return {@code true} if no other send behavior was registered for any of the addresses bound by delegate service. */ - public boolean addDelegate(TransportService transportService, DelegateTransport transport) { + public boolean addSendBehavior(TransportService transportService, StubbableTransport.SendRequestBehavior sendBehavior) { boolean noRegistered = true; for (TransportAddress transportAddress : extractTransportAddresses(transportService)) { - noRegistered &= addDelegate(transportAddress, transport); + noRegistered &= addSendBehavior(transportAddress, sendBehavior); } return noRegistered; } /** - * Adds a new delegate transport that is used for communication with the given transport address. + * Adds a new send behavior that is used for communication with the given delegate address. * - * @return {@code true} iff no other delegate was registered for this address before. + * @return {@code true} if no other send behavior was registered for this address before. */ - public boolean addDelegate(TransportAddress transportAddress, DelegateTransport transport) { - return transport().transports.put(transportAddress, transport) == null; - } - - private LookupTestTransport transport() { - return (LookupTestTransport) transport; + public boolean addSendBehavior(TransportAddress transportAddress, StubbableTransport.SendRequestBehavior sendBehavior) { + return transport().addSendBehavior(transportAddress, sendBehavior); } /** - * A lookup transport that has a list of potential Transport implementations to delegate to for node operations, - * if none is registered, then the default one is used. + * Adds a send behavior that is the default send behavior. + * + * @return {@code true} if no default send behavior was registered */ - private static class LookupTestTransport extends DelegateTransport { - - final ConcurrentMap transports = ConcurrentCollections.newConcurrentMap(); - - LookupTestTransport(Transport transport) { - super(transport); - } - - private Transport getTransport(DiscoveryNode node) { - Transport transport = transports.get(node.getAddress()); - if (transport != null) { - return transport; - } - return this.transport; - } - - @Override - public boolean nodeConnected(DiscoveryNode node) { - return getTransport(node).nodeConnected(node); - } - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - getTransport(node).connectToNode(node, connectionProfile, connectionValidator); - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - getTransport(node).disconnectFromNode(node); - } + public boolean addSendBehavior(StubbableTransport.SendRequestBehavior behavior) { + return transport().setDefaultSendBehavior(behavior); + } - @Override - public Connection getConnection(DiscoveryNode node) { - return getTransport(node).getConnection(node); - } - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - return getTransport(node).openConnection(node, profile); + /** + * Adds a new connect behavior that is used for creating connections with the given delegate service. + * + * @return {@code true} if no other send behavior was registered for any of the addresses bound by delegate service. + */ + public boolean addConnectBehavior(TransportService transportService, StubbableTransport.OpenConnectionBehavior connectBehavior) { + boolean noRegistered = true; + for (TransportAddress transportAddress : extractTransportAddresses(transportService)) { + noRegistered &= addConnectBehavior(transportAddress, connectBehavior); } + return noRegistered; } /** - * A pure delegate transport. - * Can be extracted to a common class if needed in other places in the codebase. + * Adds a new connect behavior that is used for creating connections with the given delegate address. + * + * @return {@code true} if no other send behavior was registered for this address before. */ - public static class DelegateTransport implements Transport { - - protected final Transport transport; - - - public DelegateTransport(Transport transport) { - this.transport = transport; - } - - @Override - public void addConnectionListener(TransportConnectionListener listener) { - transport.addConnectionListener(listener); - } - - @Override - public boolean removeConnectionListener(TransportConnectionListener listener) { - return transport.removeConnectionListener(listener); - } - - @Override - public void registerRequestHandler(RequestHandlerRegistry reg) { - transport.registerRequestHandler(reg); - } - - @Override - public RequestHandlerRegistry getRequestHandler(String action) { - return transport.getRequestHandler(action); - } - - @Override - public BoundTransportAddress boundAddress() { - return transport.boundAddress(); - } - - @Override - public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { - return transport.addressesFromString(address, perAddressLimit); - } - - @Override - public boolean nodeConnected(DiscoveryNode node) { - return transport.nodeConnected(node); - } - - @Override - public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, - CheckedBiConsumer connectionValidator) - throws ConnectTransportException { - transport.connectToNode(node, connectionProfile, connectionValidator); - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - transport.disconnectFromNode(node); - } - - @Override - public List getLocalAddresses() { - return transport.getLocalAddresses(); - } - - @Override - public Connection getConnection(DiscoveryNode node) { - return new FilteredConnection(transport.getConnection(node)) { - @Override - public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws IOException, TransportException { - DelegateTransport.this.sendRequest(connection, requestId, action, request, options); - } - }; - } - - @Override - public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - return new FilteredConnection(transport.openConnection(node, profile)) { - @Override - public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws IOException, TransportException { - DelegateTransport.this.sendRequest(connection, requestId, action, request, options); - } - }; - } - - @Override - public TransportStats getStats() { - return transport.getStats(); - } - - @Override - public ResponseHandlers getResponseHandlers() { - return transport.getResponseHandlers(); - } - - @Override - public Lifecycle.State lifecycleState() { - return transport.lifecycleState(); - } - - @Override - public void addLifecycleListener(LifecycleListener listener) { - transport.addLifecycleListener(listener); - } - - @Override - public void removeLifecycleListener(LifecycleListener listener) { - transport.removeLifecycleListener(listener); - } - - @Override - public void start() { - transport.start(); - } + public boolean addConnectBehavior(TransportAddress transportAddress, StubbableTransport.OpenConnectionBehavior connectBehavior) { + return transport().addConnectBehavior(transportAddress, connectBehavior); + } - @Override - public void stop() { - transport.stop(); + /** + * Adds a new get connection behavior that is used for communication with the given delegate service. + * + * @return {@code true} if no other get connection behavior was registered for any of the addresses bound by delegate service. + */ + public boolean addGetConnectionBehavior(TransportService transportService, StubbableConnectionManager.GetConnectionBehavior behavior) { + boolean noRegistered = true; + for (TransportAddress transportAddress : extractTransportAddresses(transportService)) { + noRegistered &= addGetConnectionBehavior(transportAddress, behavior); } + return noRegistered; + } - @Override - public void close() { - transport.close(); - } + /** + * Adds a get connection behavior that is used for communication with the given delegate address. + * + * @return {@code true} if no other get connection behavior was registered for this address before. + */ + public boolean addGetConnectionBehavior(TransportAddress transportAddress, StubbableConnectionManager.GetConnectionBehavior behavior) { + return connectionManager().addConnectBehavior(transportAddress, behavior); + } - @Override - public Map profileBoundAddresses() { - return transport.profileBoundAddresses(); - } + /** + * Adds a get connection behavior that is the default get connection behavior. + * + * @return {@code true} if no default get connection behavior was registered. + */ + public boolean addGetConnectionBehavior(StubbableConnectionManager.GetConnectionBehavior behavior) { + return connectionManager().setDefaultConnectBehavior(behavior); + } - protected void sendRequest(Transport.Connection connection, long requestId, String action, TransportRequest request, - TransportRequestOptions options) throws IOException { - connection.sendRequest(requestId, action, request, options); + /** + * Adds a node connected behavior that is used for the given delegate service. + * + * @return {@code true} if no other node connected behavior was registered for any of the addresses bound by delegate service. + */ + public boolean addNodeConnectedBehavior(TransportService transportService, StubbableConnectionManager.NodeConnectedBehavior behavior) { + boolean noRegistered = true; + for (TransportAddress transportAddress : extractTransportAddresses(transportService)) { + noRegistered &= addNodeConnectedBehavior(transportAddress, behavior); } + return noRegistered; } /** - * The delegate transport instances defined in this class mock various kinds of disruption types. This subclass adds a method - * {@link #clearRule()} so that when the disruptions are cleared (see {@link #clearRule(TransportService)}) this gives the - * disruption a possibility to run clean-up actions. + * Adds a node connected behavior that is used for the given delegate address. + * + * @return {@code true} if no other node connected behavior was registered for this address before. */ - public abstract static class ClearableTransport extends DelegateTransport { + public boolean addNodeConnectedBehavior(TransportAddress transportAddress, StubbableConnectionManager.NodeConnectedBehavior behavior) { + return connectionManager().addNodeConnectedBehavior(transportAddress, behavior); + } - public ClearableTransport(Transport transport) { - super(transport); - } + /** + * Adds a node connected behavior that is the default node connected behavior. + * + * @return {@code true} if no default node connected behavior was registered. + */ + public boolean addNodeConnectedBehavior(StubbableConnectionManager.NodeConnectedBehavior behavior) { + return connectionManager().setDefaultNodeConnectedBehavior(behavior); + } - /** - * Called by {@link #clearRule(TransportService)} - */ - public abstract void clearRule(); + public StubbableTransport transport() { + return (StubbableTransport) transport; } + public StubbableConnectionManager connectionManager() { + return (StubbableConnectionManager) connectionManager; + } List activeTracers = new CopyOnWriteArrayList<>(); @@ -766,78 +581,36 @@ protected void traceRequestSent(DiscoveryNode node, long requestId, String actio } } - private static class FilteredConnection implements Transport.Connection { - protected final Transport.Connection connection; - - private FilteredConnection(Transport.Connection connection) { - this.connection = connection; - } - - @Override - public DiscoveryNode getNode() { - return connection.getNode(); - } - - @Override - public Version getVersion() { - return connection.getVersion(); - } - - @Override - public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) - throws IOException, TransportException { - connection.sendRequest(requestId, action, request, options); - } - - @Override - public void close() throws IOException { - connection.close(); - } - - @Override - public Object getCacheKey() { - return connection.getCacheKey(); - } - } - public Transport getOriginalTransport() { Transport transport = transport(); - while (transport instanceof DelegateTransport) { - transport = ((DelegateTransport) transport).transport; + while (transport instanceof StubbableTransport) { + transport = ((StubbableTransport) transport).getDelegate(); } return transport; } @Override public Transport.Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { - FilteredConnection filteredConnection = new FilteredConnection(super.openConnection(node, profile)) { - final AtomicBoolean closed = new AtomicBoolean(false); - - @Override - public void close() throws IOException { - try { - super.close(); - } finally { - if (closed.compareAndSet(false, true)) { - synchronized (openConnections) { - List connections = openConnections.get(node); - boolean remove = connections.remove(this); - assert remove; - if (connections.isEmpty()) { - openConnections.remove(node); - } - } - } - } + Transport.Connection connection = super.openConnection(node, profile); - } - }; synchronized (openConnections) { List connections = openConnections.computeIfAbsent(node, (n) -> new CopyOnWriteArrayList<>()); - connections.add(filteredConnection); + connections.add(connection); } - return filteredConnection; + + connection.addCloseListener(ActionListener.wrap(() -> { + synchronized (openConnections) { + List connections = openConnections.get(node); + boolean remove = connections.remove(connection); + assert remove : "Should have removed connection"; + if (connections.isEmpty()) { + openConnections.remove(node); + } + } + })); + + return connection; } @Override @@ -851,4 +624,5 @@ protected void doClose() throws IOException { public DiscoveryNode getLocalDiscoNode() { return this.getLocalNode(); } + } diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableConnectionManager.java b/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableConnectionManager.java new file mode 100644 index 0000000000000..a74cb2752c284 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableConnectionManager.java @@ -0,0 +1,136 @@ +/* + * 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.test.transport; + +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.CheckedBiConsumer; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.ConnectTransportException; +import org.elasticsearch.transport.ConnectionManager; +import org.elasticsearch.transport.ConnectionProfile; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportConnectionListener; + +import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class StubbableConnectionManager extends ConnectionManager { + + private final ConnectionManager delegate; + private final ConcurrentMap getConnectionBehaviors; + private final ConcurrentMap nodeConnectedBehaviors; + private volatile GetConnectionBehavior defaultGetConnectionBehavior = ConnectionManager::getConnection; + private volatile NodeConnectedBehavior defaultNodeConnectedBehavior = ConnectionManager::nodeConnected; + + public StubbableConnectionManager(ConnectionManager delegate, Settings settings, Transport transport, ThreadPool threadPool) { + super(settings, transport, threadPool); + this.delegate = delegate; + this.getConnectionBehaviors = new ConcurrentHashMap<>(); + this.nodeConnectedBehaviors = new ConcurrentHashMap<>(); + } + + public boolean addConnectBehavior(TransportAddress transportAddress, GetConnectionBehavior connectBehavior) { + return getConnectionBehaviors.put(transportAddress, connectBehavior) == null; + } + + public boolean setDefaultConnectBehavior(GetConnectionBehavior behavior) { + GetConnectionBehavior prior = defaultGetConnectionBehavior; + defaultGetConnectionBehavior = behavior; + return prior == null; + } + + public boolean addNodeConnectedBehavior(TransportAddress transportAddress, NodeConnectedBehavior behavior) { + return nodeConnectedBehaviors.put(transportAddress, behavior) == null; + } + + public boolean setDefaultNodeConnectedBehavior(NodeConnectedBehavior behavior) { + NodeConnectedBehavior prior = defaultNodeConnectedBehavior; + defaultNodeConnectedBehavior = behavior; + return prior == null; + } + + public void clearBehaviors() { + getConnectionBehaviors.clear(); + nodeConnectedBehaviors.clear(); + } + + public void clearBehavior(TransportAddress transportAddress) { + getConnectionBehaviors.remove(transportAddress); + nodeConnectedBehaviors.remove(transportAddress); + } + + @Override + public Transport.Connection getConnection(DiscoveryNode node) { + TransportAddress address = node.getAddress(); + GetConnectionBehavior behavior = getConnectionBehaviors.getOrDefault(address, defaultGetConnectionBehavior); + return behavior.getConnection(delegate, node); + } + + @Override + public boolean nodeConnected(DiscoveryNode node) { + TransportAddress address = node.getAddress(); + NodeConnectedBehavior behavior = nodeConnectedBehaviors.getOrDefault(address, defaultNodeConnectedBehavior); + return behavior.nodeConnected(delegate, node); + } + + @Override + public void addListener(TransportConnectionListener listener) { + delegate.addListener(listener); + } + + @Override + public void removeListener(TransportConnectionListener listener) { + delegate.removeListener(listener); + } + + @Override + public void connectToNode(DiscoveryNode node, ConnectionProfile connectionProfile, + CheckedBiConsumer connectionValidator) + throws ConnectTransportException { + delegate.connectToNode(node, connectionProfile, connectionValidator); + } + + @Override + public void disconnectFromNode(DiscoveryNode node) { + delegate.disconnectFromNode(node); + } + + @Override + public int connectedNodeCount() { + return delegate.connectedNodeCount(); + } + + @Override + public void close() { + delegate.close(); + } + + @FunctionalInterface + public interface GetConnectionBehavior { + Transport.Connection getConnection(ConnectionManager connectionManager, DiscoveryNode discoveryNode); + } + + @FunctionalInterface + public interface NodeConnectedBehavior { + boolean nodeConnected(ConnectionManager connectionManager, DiscoveryNode discoveryNode); + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java b/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java new file mode 100644 index 0000000000000..5a0dd3b7f6d53 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java @@ -0,0 +1,252 @@ +/* + * 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.test.transport; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.component.Lifecycle; +import org.elasticsearch.common.component.LifecycleListener; +import org.elasticsearch.common.transport.BoundTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.transport.ConnectionProfile; +import org.elasticsearch.transport.RequestHandlerRegistry; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportConnectionListener; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportStats; + +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class StubbableTransport implements Transport { + + private final ConcurrentHashMap sendBehaviors = new ConcurrentHashMap<>(); + private final ConcurrentHashMap connectBehaviors = new ConcurrentHashMap<>(); + private volatile SendRequestBehavior defaultSendRequest = null; + private volatile OpenConnectionBehavior defaultConnectBehavior = null; + private final Transport delegate; + + + public StubbableTransport(Transport transport) { + this.delegate = transport; + } + + boolean setDefaultSendBehavior(SendRequestBehavior sendBehavior) { + SendRequestBehavior prior = defaultSendRequest; + defaultSendRequest = sendBehavior; + return prior == null; + } + + boolean addSendBehavior(TransportAddress transportAddress, SendRequestBehavior sendBehavior) { + return sendBehaviors.put(transportAddress, sendBehavior) == null; + } + + boolean addConnectBehavior(TransportAddress transportAddress, OpenConnectionBehavior connectBehavior) { + return connectBehaviors.put(transportAddress, connectBehavior) == null; + } + + void clearBehaviors() { + sendBehaviors.clear(); + connectBehaviors.clear(); + } + + void clearBehavior(TransportAddress transportAddress) { + SendRequestBehavior behavior = sendBehaviors.remove(transportAddress); + if (behavior != null) { + behavior.clearCallback(); + } + connectBehaviors.remove(transportAddress); + } + + Transport getDelegate() { + return delegate; + } + + @Override + public void addConnectionListener(TransportConnectionListener listener) { + delegate.addConnectionListener(listener); + } + + @Override + public boolean removeConnectionListener(TransportConnectionListener listener) { + return delegate.removeConnectionListener(listener); + } + + @Override + public void registerRequestHandler(RequestHandlerRegistry reg) { + delegate.registerRequestHandler(reg); + } + + @Override + public RequestHandlerRegistry getRequestHandler(String action) { + return delegate.getRequestHandler(action); + } + + @Override + public BoundTransportAddress boundAddress() { + return delegate.boundAddress(); + } + + @Override + public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { + return delegate.addressesFromString(address, perAddressLimit); + } + + @Override + public List getLocalAddresses() { + return delegate.getLocalAddresses(); + } + + @Override + public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) { + TransportAddress address = node.getAddress(); + OpenConnectionBehavior behavior = connectBehaviors.getOrDefault(address, defaultConnectBehavior); + Connection connection; + if (behavior == null) { + connection = delegate.openConnection(node, profile); + } else { + connection = behavior.openConnection(delegate, node, profile); + } + + return new WrappedConnection(connection); + } + + @Override + public TransportStats getStats() { + return delegate.getStats(); + } + + @Override + public Transport.ResponseHandlers getResponseHandlers() { + return delegate.getResponseHandlers(); + } + + @Override + public Lifecycle.State lifecycleState() { + return delegate.lifecycleState(); + } + + @Override + public void addLifecycleListener(LifecycleListener listener) { + delegate.addLifecycleListener(listener); + } + + @Override + public void removeLifecycleListener(LifecycleListener listener) { + delegate.removeLifecycleListener(listener); + } + + @Override + public void start() { + delegate.start(); + } + + @Override + public void stop() { + delegate.stop(); + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public Map profileBoundAddresses() { + return delegate.profileBoundAddresses(); + } + + private class WrappedConnection implements Transport.Connection { + + private final Transport.Connection connection; + + private WrappedConnection(Transport.Connection connection) { + this.connection = connection; + } + + @Override + public DiscoveryNode getNode() { + return connection.getNode(); + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws IOException, TransportException { + TransportAddress address = connection.getNode().getAddress(); + SendRequestBehavior behavior = sendBehaviors.getOrDefault(address, defaultSendRequest); + if (behavior == null) { + connection.sendRequest(requestId, action, request, options); + } else { + behavior.sendRequest(connection, requestId, action, request, options); + } + } + + @Override + public boolean sendPing() { + return connection.sendPing(); + } + + @Override + public void addCloseListener(ActionListener listener) { + connection.addCloseListener(listener); + } + + + @Override + public boolean isClosed() { + return connection.isClosed(); + } + + @Override + public Version getVersion() { + return connection.getVersion(); + } + + @Override + public Object getCacheKey() { + return connection.getCacheKey(); + } + + @Override + public void close() { + connection.close(); + } + } + + @FunctionalInterface + public interface OpenConnectionBehavior { + Connection openConnection(Transport transport, DiscoveryNode discoveryNode, ConnectionProfile profile); + } + + @FunctionalInterface + public interface SendRequestBehavior { + void sendRequest(Connection connection, long requestId, String action, TransportRequest request, + TransportRequestOptions options) throws IOException; + + default void clearCallback() { + } + } +}