From ab44f5fd5d8d3fd9119332af2659dbd0a0fb087c Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 26 Mar 2019 09:32:53 -0600 Subject: [PATCH] Add InboundHandler for inbound message handling (#40430) This commit adds an InboundHandler to handle inbound message processing. With this commit, this code is moved out of the TcpTransport. Additionally, finer grained unit tests are added to ensure that the inbound processing works as expected --- .../transport/netty4/Netty4TransportIT.java | 131 -------- .../transport/nio/NioTransportIT.java | 132 -------- .../transport/InboundHandler.java | 286 ++++++++++++++++++ .../elasticsearch/transport/TcpTransport.java | 257 ++-------------- .../transport/InboundHandlerTests.java | 209 +++++++++++++ .../transport/TcpTransportTests.java | 157 ---------- .../AbstractSimpleTransportTestCase.java | 42 +-- 7 files changed, 526 insertions(+), 688 deletions(-) delete mode 100644 modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java delete mode 100644 plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java create mode 100644 server/src/main/java/org/elasticsearch/transport/InboundHandler.java create mode 100644 server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java deleted file mode 100644 index cf9791ce85a..00000000000 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportIT.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.netty4; - -import org.elasticsearch.ESNetty4IntegTestCase; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.health.ClusterHealthStatus; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.plugins.NetworkPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.InboundMessage; -import org.elasticsearch.transport.TcpChannel; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportSettings; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.is; - -@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numDataNodes = 1) -public class Netty4TransportIT extends ESNetty4IntegTestCase { - // static so we can use it in anonymous classes - private static String channelProfileName = null; - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(NetworkModule.TRANSPORT_TYPE_KEY, "exception-throwing").build(); - } - - @Override - protected Collection> nodePlugins() { - List> list = new ArrayList<>(); - list.add(ExceptionThrowingNetty4Transport.TestPlugin.class); - list.addAll(super.nodePlugins()); - return Collections.unmodifiableCollection(list); - } - - public void testThatConnectionFailsAsIntended() throws Exception { - Client transportClient = internalCluster().transportClient(); - ClusterHealthResponse clusterIndexHealths = transportClient.admin().cluster().prepareHealth().get(); - assertThat(clusterIndexHealths.getStatus(), is(ClusterHealthStatus.GREEN)); - try { - transportClient.filterWithHeader(Collections.singletonMap("ERROR", "MY MESSAGE")).admin().cluster().prepareHealth().get(); - fail("Expected exception, but didn't happen"); - } catch (ElasticsearchException e) { - assertThat(e.getMessage(), containsString("MY MESSAGE")); - assertThat(channelProfileName, is(TransportSettings.DEFAULT_PROFILE)); - } - } - - public static final class ExceptionThrowingNetty4Transport extends Netty4Transport { - - public static class TestPlugin extends Plugin implements NetworkPlugin { - - @Override - public Map> getTransports(Settings settings, ThreadPool threadPool, - PageCacheRecycler pageCacheRecycler, - CircuitBreakerService circuitBreakerService, - NamedWriteableRegistry namedWriteableRegistry, - NetworkService networkService) { - return Collections.singletonMap("exception-throwing", - () -> new ExceptionThrowingNetty4Transport(settings, threadPool, networkService, pageCacheRecycler, - namedWriteableRegistry, circuitBreakerService)); - } - } - - public ExceptionThrowingNetty4Transport( - Settings settings, - ThreadPool threadPool, - NetworkService networkService, - PageCacheRecycler recycler, - NamedWriteableRegistry namedWriteableRegistry, - CircuitBreakerService circuitBreakerService) { - super(settings, Version.CURRENT, threadPool, networkService, recycler, namedWriteableRegistry, circuitBreakerService); - } - - @Override - protected void handleRequest(TcpChannel channel, InboundMessage.Request request, int messageLengthBytes) throws IOException { - super.handleRequest(channel, request, messageLengthBytes); - channelProfileName = TransportSettings.DEFAULT_PROFILE; - } - - @Override - protected void validateRequest(StreamInput buffer, long requestId, String action) - throws IOException { - super.validateRequest(buffer, requestId, action); - String error = threadPool.getThreadContext().getHeader("ERROR"); - if (error != null) { - throw new ElasticsearchException(error); - } - } - - } - -} diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java b/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java deleted file mode 100644 index d02be2cff9e..00000000000 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/transport/nio/NioTransportIT.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.nio; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.NioIntegTestCase; -import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.health.ClusterHealthStatus; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.plugins.NetworkPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.ESIntegTestCase.Scope; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.InboundMessage; -import org.elasticsearch.transport.TcpChannel; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportSettings; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.is; - -@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numDataNodes = 1) -public class NioTransportIT extends NioIntegTestCase { - - // static so we can use it in anonymous classes - private static String channelProfileName = null; - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(NetworkModule.TRANSPORT_TYPE_KEY, "exception-throwing").build(); - } - - @Override - protected Collection> nodePlugins() { - List> list = new ArrayList<>(); - list.add(ExceptionThrowingNioTransport.TestPlugin.class); - list.addAll(super.nodePlugins()); - return Collections.unmodifiableCollection(list); - } - - public void testThatConnectionFailsAsIntended() throws Exception { - Client transportClient = internalCluster().transportClient(); - ClusterHealthResponse clusterIndexHealths = transportClient.admin().cluster().prepareHealth().get(); - assertThat(clusterIndexHealths.getStatus(), is(ClusterHealthStatus.GREEN)); - try { - transportClient.filterWithHeader(Collections.singletonMap("ERROR", "MY MESSAGE")).admin().cluster().prepareHealth().get(); - fail("Expected exception, but didn't happen"); - } catch (ElasticsearchException e) { - assertThat(e.getMessage(), containsString("MY MESSAGE")); - assertThat(channelProfileName, is(TransportSettings.DEFAULT_PROFILE)); - } - } - - public static final class ExceptionThrowingNioTransport extends NioTransport { - - private static final Logger logger = LogManager.getLogger(ExceptionThrowingNioTransport.class); - - public static class TestPlugin extends Plugin implements NetworkPlugin { - - @Override - public Map> getTransports(Settings settings, ThreadPool threadPool, - PageCacheRecycler pageCacheRecycler, - CircuitBreakerService circuitBreakerService, - NamedWriteableRegistry namedWriteableRegistry, - NetworkService networkService) { - return Collections.singletonMap("exception-throwing", - () -> new ExceptionThrowingNioTransport(settings, threadPool, networkService, pageCacheRecycler, - namedWriteableRegistry, circuitBreakerService)); - } - } - - ExceptionThrowingNioTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, - PageCacheRecycler pageCacheRecycler, NamedWriteableRegistry namedWriteableRegistry, - CircuitBreakerService circuitBreakerService) { - super(settings, Version.CURRENT, threadPool, networkService, pageCacheRecycler, namedWriteableRegistry, circuitBreakerService, - new NioGroupFactory(settings, logger)); - } - - @Override - protected void handleRequest(TcpChannel channel, InboundMessage.Request request, int messageLengthBytes) throws IOException { - super.handleRequest(channel, request, messageLengthBytes); - channelProfileName = TransportSettings.DEFAULT_PROFILE; - } - - @Override - protected void validateRequest(StreamInput buffer, long requestId, String action) - throws IOException { - super.validateRequest(buffer, requestId, action); - String error = threadPool.getThreadContext().getHeader("ERROR"); - if (error != null) { - throw new ElasticsearchException(error); - } - } - - } -} diff --git a/server/src/main/java/org/elasticsearch/transport/InboundHandler.java b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java new file mode 100644 index 00000000000..c50825d00a0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/InboundHandler.java @@ -0,0 +1,286 @@ +/* + * 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.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.Version; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.MapBuilder; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.metrics.MeanMetric; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +public class InboundHandler { + + private static final Logger logger = LogManager.getLogger(InboundHandler.class); + + private final MeanMetric readBytesMetric = new MeanMetric(); + private final ThreadPool threadPool; + private final OutboundHandler outboundHandler; + private final CircuitBreakerService circuitBreakerService; + private final InboundMessage.Reader reader; + private final TransportLogger transportLogger; + private final TransportHandshaker handshaker; + private final TransportKeepAlive keepAlive; + + private final Transport.ResponseHandlers responseHandlers = new Transport.ResponseHandlers(); + private volatile Map> requestHandlers = Collections.emptyMap(); + private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; + + InboundHandler(ThreadPool threadPool, OutboundHandler outboundHandler, InboundMessage.Reader reader, + CircuitBreakerService circuitBreakerService, TransportLogger transportLogger, TransportHandshaker handshaker, + TransportKeepAlive keepAlive) { + this.threadPool = threadPool; + this.outboundHandler = outboundHandler; + this.circuitBreakerService = circuitBreakerService; + this.reader = reader; + this.transportLogger = transportLogger; + this.handshaker = handshaker; + this.keepAlive = keepAlive; + } + + synchronized void registerRequestHandler(RequestHandlerRegistry reg) { + if (requestHandlers.containsKey(reg.getAction())) { + throw new IllegalArgumentException("transport handlers for action " + reg.getAction() + " is already registered"); + } + requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap(); + } + + final RequestHandlerRegistry getRequestHandler(String action) { + return requestHandlers.get(action); + } + + final Transport.ResponseHandlers getResponseHandlers() { + return responseHandlers; + } + + MeanMetric getReadBytes() { + return readBytesMetric; + } + + void setMessageListener(TransportMessageListener listener) { + if (messageListener == TransportMessageListener.NOOP_LISTENER) { + messageListener = listener; + } else { + throw new IllegalStateException("Cannot set message listener twice"); + } + } + + void inboundMessage(TcpChannel channel, BytesReference message) throws Exception { + channel.getChannelStats().markAccessed(threadPool.relativeTimeInMillis()); + transportLogger.logInboundMessage(channel, message); + readBytesMetric.inc(message.length() + TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE); + // Message length of 0 is a ping + if (message.length() != 0) { + messageReceived(message, channel); + } else { + keepAlive.receiveKeepAlive(channel); + } + } + + private void messageReceived(BytesReference reference, TcpChannel channel) throws IOException { + InetSocketAddress remoteAddress = channel.getRemoteAddress(); + + ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext existing = threadContext.stashContext(); + InboundMessage message = reader.deserialize(reference)) { + // Place the context with the headers from the message + message.getStoredContext().restore(); + threadContext.putTransient("_remote_address", remoteAddress); + if (message.isRequest()) { + handleRequest(channel, (InboundMessage.Request) message, reference.length()); + } else { + final TransportResponseHandler handler; + long requestId = message.getRequestId(); + if (message.isHandshake()) { + handler = handshaker.removeHandlerForHandshake(requestId); + } else { + TransportResponseHandler theHandler = + responseHandlers.onResponseReceived(requestId, messageListener); + if (theHandler == null && message.isError()) { + handler = handshaker.removeHandlerForHandshake(requestId); + } else { + handler = theHandler; + } + } + // ignore if its null, the service logs it + if (handler != null) { + if (message.isError()) { + handlerResponseError(message.getStreamInput(), handler); + } else { + handleResponse(remoteAddress, message.getStreamInput(), handler); + } + // Check the entire message has been read + final int nextByte = message.getStreamInput().read(); + // calling read() is useful to make sure the message is fully read, even if there is an EOS marker + if (nextByte != -1) { + throw new IllegalStateException("Message not fully read (response) for requestId [" + requestId + "], handler [" + + handler + "], error [" + message.isError() + "]; resetting"); + } + } + } + } + } + + private void handleRequest(TcpChannel channel, InboundMessage.Request message, int messageLengthBytes) { + final Set features = message.getFeatures(); + final String action = message.getActionName(); + final long requestId = message.getRequestId(); + final StreamInput stream = message.getStreamInput(); + final Version version = message.getVersion(); + messageListener.onRequestReceived(requestId, action); + TransportChannel transportChannel = null; + try { + if (message.isHandshake()) { + handshaker.handleHandshake(version, features, channel, requestId, stream); + } else { + final RequestHandlerRegistry reg = getRequestHandler(action); + if (reg == null) { + throw new ActionNotFoundTransportException(action); + } + CircuitBreaker breaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); + if (reg.canTripCircuitBreaker()) { + breaker.addEstimateBytesAndMaybeBreak(messageLengthBytes, ""); + } else { + breaker.addWithoutBreaking(messageLengthBytes); + } + transportChannel = new TcpTransportChannel(outboundHandler, channel, action, requestId, version, features, + circuitBreakerService, messageLengthBytes, message.isCompress()); + final TransportRequest request = reg.newRequest(stream); + request.remoteAddress(new TransportAddress(channel.getRemoteAddress())); + // in case we throw an exception, i.e. when the limit is hit, we don't want to verify + final int nextByte = stream.read(); + // calling read() is useful to make sure the message is fully read, even if there some kind of EOS marker + if (nextByte != -1) { + throw new IllegalStateException("Message not fully read (request) for requestId [" + requestId + "], action [" + action + + "], available [" + stream.available() + "]; resetting"); + } + threadPool.executor(reg.getExecutor()).execute(new RequestHandler(reg, request, transportChannel)); + } + } catch (Exception e) { + // the circuit breaker tripped + if (transportChannel == null) { + transportChannel = new TcpTransportChannel(outboundHandler, channel, action, requestId, version, features, + circuitBreakerService, 0, message.isCompress()); + } + try { + transportChannel.sendResponse(e); + } catch (IOException inner) { + inner.addSuppressed(e); + logger.warn(() -> new ParameterizedMessage("Failed to send error message back to client for action [{}]", action), inner); + } + } + } + + private void handleResponse(InetSocketAddress remoteAddress, final StreamInput stream, + final TransportResponseHandler handler) { + final T response; + try { + response = handler.read(stream); + response.remoteAddress(new TransportAddress(remoteAddress)); + } catch (Exception e) { + handleException(handler, new TransportSerializationException( + "Failed to deserialize response from handler [" + handler.getClass().getName() + "]", e)); + return; + } + threadPool.executor(handler.executor()).execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + handleException(handler, new ResponseHandlerFailureTransportException(e)); + } + + @Override + protected void doRun() { + handler.handleResponse(response); + } + }); + } + + private void handlerResponseError(StreamInput stream, final TransportResponseHandler handler) { + Exception error; + try { + error = stream.readException(); + } catch (Exception e) { + error = new TransportSerializationException("Failed to deserialize exception response from stream", e); + } + handleException(handler, error); + } + + private void handleException(final TransportResponseHandler handler, Throwable error) { + if (!(error instanceof RemoteTransportException)) { + error = new RemoteTransportException(error.getMessage(), error); + } + final RemoteTransportException rtx = (RemoteTransportException) error; + threadPool.executor(handler.executor()).execute(() -> { + try { + handler.handleException(rtx); + } catch (Exception e) { + logger.error(() -> new ParameterizedMessage("failed to handle exception response [{}]", handler), e); + } + }); + } + + private static class RequestHandler extends AbstractRunnable { + private final RequestHandlerRegistry reg; + private final TransportRequest request; + private final TransportChannel transportChannel; + + RequestHandler(RequestHandlerRegistry reg, TransportRequest request, TransportChannel transportChannel) { + this.reg = reg; + this.request = request; + this.transportChannel = transportChannel; + } + + @SuppressWarnings({"unchecked"}) + @Override + protected void doRun() throws Exception { + reg.processMessageReceived(request, transportChannel); + } + + @Override + public boolean isForceExecution() { + return reg.isForceExecution(); + } + + @Override + public void onFailure(Exception e) { + try { + transportChannel.sendResponse(e); + } catch (Exception inner) { + inner.addSuppressed(e); + logger.warn(() -> new ParameterizedMessage( + "Failed to send error message back to client for action [{}]", reg.getAction()), inner); + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index 8125d5bcb12..eb61af8d2a3 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -32,9 +32,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; @@ -52,10 +50,8 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.node.Node; @@ -107,15 +103,11 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private static final BytesReference EMPTY_BYTES_REFERENCE = new BytesArray(new byte[0]); protected final Settings settings; - private final CircuitBreakerService circuitBreakerService; protected final ThreadPool threadPool; - protected final BigArrays bigArrays; protected final PageCacheRecycler pageCacheRecycler; protected final NetworkService networkService; protected final Set profileSettings; - private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; - private final ConcurrentMap profileBoundAddresses = newConcurrentMap(); private final Map> serverChannels = newConcurrentMap(); private final Set acceptedChannels = ConcurrentCollections.newConcurrentSet(); @@ -125,14 +117,10 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final ReadWriteLock closeLock = new ReentrantReadWriteLock(); private volatile BoundTransportAddress boundAddress; - private final MeanMetric readBytesMetric = new MeanMetric(); - private volatile Map> requestHandlers = Collections.emptyMap(); - private final ResponseHandlers responseHandlers = new ResponseHandlers(); - private final TransportLogger transportLogger; private final TransportHandshaker handshaker; private final TransportKeepAlive keepAlive; - private final InboundMessage.Reader reader; private final OutboundHandler outboundHandler; + private final InboundHandler inboundHandler; public TcpTransport(Settings settings, Version version, ThreadPool threadPool, PageCacheRecycler pageCacheRecycler, CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry, @@ -140,11 +128,9 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements this.settings = settings; this.profileSettings = getProfileSettings(settings); this.threadPool = threadPool; - this.bigArrays = new BigArrays(pageCacheRecycler, circuitBreakerService, CircuitBreaker.IN_FLIGHT_REQUESTS); this.pageCacheRecycler = pageCacheRecycler; - this.circuitBreakerService = circuitBreakerService; this.networkService = networkService; - this.transportLogger = new TransportLogger(); + TransportLogger transportLogger = new TransportLogger(); String nodeName = Node.NODE_NAME_SETTING.get(settings); final Settings defaultFeatures = TransportSettings.DEFAULT_FEATURES_SETTING.get(settings); String[] features; @@ -159,16 +145,19 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements // use a sorted set to present the features in a consistent order features = new TreeSet<>(defaultFeatures.names()).toArray(new String[defaultFeatures.names().size()]); } - this.outboundHandler = new OutboundHandler(nodeName, version, features, threadPool, bigArrays, transportLogger); + BigArrays bigArrays = new BigArrays(pageCacheRecycler, circuitBreakerService, CircuitBreaker.IN_FLIGHT_REQUESTS); + this.outboundHandler = new OutboundHandler(nodeName, version, features, threadPool, bigArrays, transportLogger); this.handshaker = new TransportHandshaker(version, threadPool, (node, channel, requestId, v) -> outboundHandler.sendRequest(node, channel, requestId, TransportHandshaker.HANDSHAKE_ACTION_NAME, new TransportHandshaker.HandshakeRequest(version), TransportRequestOptions.EMPTY, v, false, true), (v, features1, channel, response, requestId) -> outboundHandler.sendResponse(v, features1, channel, requestId, TransportHandshaker.HANDSHAKE_ACTION_NAME, response, false, true)); + InboundMessage.Reader reader = new InboundMessage.Reader(version, namedWriteableRegistry, threadPool.getThreadContext()); this.keepAlive = new TransportKeepAlive(threadPool, this.outboundHandler::sendBytes); - this.reader = new InboundMessage.Reader(version, namedWriteableRegistry, threadPool.getThreadContext()); + this.inboundHandler = new InboundHandler(threadPool, outboundHandler, reader, circuitBreakerService, transportLogger, handshaker, + keepAlive); } @Override @@ -177,26 +166,13 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements @Override public synchronized void setMessageListener(TransportMessageListener listener) { - if (messageListener == TransportMessageListener.NOOP_LISTENER) { - messageListener = listener; - outboundHandler.setMessageListener(listener); - } else { - throw new IllegalStateException("Cannot set message listener twice"); - } - } - - @Override - public CircuitBreaker getInFlightRequestBreaker() { - // We always obtain a fresh breaker to reflect changes to the breaker configuration. - return circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); + outboundHandler.setMessageListener(listener); + inboundHandler.setMessageListener(listener); } @Override public synchronized void registerRequestHandler(RequestHandlerRegistry reg) { - if (requestHandlers.containsKey(reg.getAction())) { - throw new IllegalArgumentException("transport handlers for action " + reg.getAction() + " is already registered"); - } - requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap(); + inboundHandler.registerRequestHandler(reg); } public final class NodeChannels extends CloseableConnection { @@ -665,14 +641,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements */ public void inboundMessage(TcpChannel channel, BytesReference message) { try { - channel.getChannelStats().markAccessed(threadPool.relativeTimeInMillis()); - transportLogger.logInboundMessage(channel, message); - // Message length of 0 is a ping - if (message.length() != 0) { - messageReceived(message, channel); - } else { - keepAlive.receiveKeepAlive(channel); - } + inboundHandler.inboundMessage(channel, message); } catch (Exception e) { onException(channel, e); } @@ -820,200 +789,9 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements } } - /** - * This method handles the message receive part for both request and responses - */ - public final void messageReceived(BytesReference reference, TcpChannel channel) throws IOException { - readBytesMetric.inc(reference.length() + TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE); - InetSocketAddress remoteAddress = channel.getRemoteAddress(); - - ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext existing = threadContext.stashContext(); - InboundMessage message = reader.deserialize(reference)) { - // Place the context with the headers from the message - message.getStoredContext().restore(); - threadContext.putTransient("_remote_address", remoteAddress); - if (message.isRequest()) { - handleRequest(channel, (InboundMessage.Request) message, reference.length()); - } else { - final TransportResponseHandler handler; - long requestId = message.getRequestId(); - if (message.isHandshake()) { - handler = handshaker.removeHandlerForHandshake(requestId); - } else { - TransportResponseHandler theHandler = - responseHandlers.onResponseReceived(requestId, messageListener); - if (theHandler == null && message.isError()) { - handler = handshaker.removeHandlerForHandshake(requestId); - } else { - handler = theHandler; - } - } - // ignore if its null, the service logs it - if (handler != null) { - if (message.isError()) { - handlerResponseError(message.getStreamInput(), handler); - } else { - handleResponse(remoteAddress, message.getStreamInput(), handler); - } - // Check the entire message has been read - final int nextByte = message.getStreamInput().read(); - // calling read() is useful to make sure the message is fully read, even if there is an EOS marker - if (nextByte != -1) { - throw new IllegalStateException("Message not fully read (response) for requestId [" + requestId + "], handler [" - + handler + "], error [" + message.isError() + "]; resetting"); - } - } - } - } - } - - private void handleResponse(InetSocketAddress remoteAddress, final StreamInput stream, - final TransportResponseHandler handler) { - final T response; - try { - response = handler.read(stream); - response.remoteAddress(new TransportAddress(remoteAddress)); - } catch (Exception e) { - handleException(handler, new TransportSerializationException( - "Failed to deserialize response from handler [" + handler.getClass().getName() + "]", e)); - return; - } - threadPool.executor(handler.executor()).execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - handleException(handler, new ResponseHandlerFailureTransportException(e)); - } - - @Override - protected void doRun() throws Exception { - handler.handleResponse(response); - } - }); - - } - - /** - * Executed for a received response error - */ - private void handlerResponseError(StreamInput stream, final TransportResponseHandler handler) { - Exception error; - try { - error = stream.readException(); - } catch (Exception e) { - error = new TransportSerializationException("Failed to deserialize exception response from stream", e); - } - handleException(handler, error); - } - - private void handleException(final TransportResponseHandler handler, Throwable error) { - if (!(error instanceof RemoteTransportException)) { - error = new RemoteTransportException(error.getMessage(), error); - } - final RemoteTransportException rtx = (RemoteTransportException) error; - threadPool.executor(handler.executor()).execute(() -> { - try { - handler.handleException(rtx); - } catch (Exception e) { - logger.error(() -> new ParameterizedMessage("failed to handle exception response [{}]", handler), e); - } - }); - } - - protected void handleRequest(TcpChannel channel, InboundMessage.Request message, int messageLengthBytes) throws IOException { - final Set features = message.getFeatures(); - final String profileName = channel.getProfile(); - final String action = message.getActionName(); - final long requestId = message.getRequestId(); - final StreamInput stream = message.getStreamInput(); - final Version version = message.getVersion(); - messageListener.onRequestReceived(requestId, action); - TransportChannel transportChannel = null; - try { - if (message.isHandshake()) { - handshaker.handleHandshake(version, features, channel, requestId, stream); - } else { - final RequestHandlerRegistry reg = getRequestHandler(action); - if (reg == null) { - throw new ActionNotFoundTransportException(action); - } - if (reg.canTripCircuitBreaker()) { - getInFlightRequestBreaker().addEstimateBytesAndMaybeBreak(messageLengthBytes, ""); - } else { - getInFlightRequestBreaker().addWithoutBreaking(messageLengthBytes); - } - transportChannel = new TcpTransportChannel(outboundHandler, channel, action, requestId, version, features, - circuitBreakerService, messageLengthBytes, message.isCompress()); - final TransportRequest request = reg.newRequest(stream); - request.remoteAddress(new TransportAddress(channel.getRemoteAddress())); - // in case we throw an exception, i.e. when the limit is hit, we don't want to verify - validateRequest(stream, requestId, action); - threadPool.executor(reg.getExecutor()).execute(new RequestHandler(reg, request, transportChannel)); - } - } catch (Exception e) { - // the circuit breaker tripped - if (transportChannel == null) { - transportChannel = new TcpTransportChannel(outboundHandler, channel, action, requestId, version, features, - circuitBreakerService, 0, message.isCompress()); - } - try { - transportChannel.sendResponse(e); - } catch (IOException inner) { - inner.addSuppressed(e); - logger.warn(() -> new ParameterizedMessage("Failed to send error message back to client for action [{}]", action), inner); - } - } - } - - // This template method is needed to inject custom error checking logic in tests. - protected void validateRequest(StreamInput stream, long requestId, String action) throws IOException { - final int nextByte = stream.read(); - // calling read() is useful to make sure the message is fully read, even if there some kind of EOS marker - if (nextByte != -1) { - throw new IllegalStateException("Message not fully read (request) for requestId [" + requestId + "], action [" + action - + "], available [" + stream.available() + "]; resetting"); - } - } - - class RequestHandler extends AbstractRunnable { - private final RequestHandlerRegistry reg; - private final TransportRequest request; - private final TransportChannel transportChannel; - - RequestHandler(RequestHandlerRegistry reg, TransportRequest request, TransportChannel transportChannel) { - this.reg = reg; - this.request = request; - this.transportChannel = transportChannel; - } - - @SuppressWarnings({"unchecked"}) - @Override - protected void doRun() throws Exception { - reg.processMessageReceived(request, transportChannel); - } - - @Override - public boolean isForceExecution() { - return reg.isForceExecution(); - } - - @Override - public void onFailure(Exception e) { - if (lifecycleState() == Lifecycle.State.STARTED) { - // we can only send a response transport is started.... - try { - transportChannel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.warn(() -> new ParameterizedMessage( - "Failed to send error message back to client for action [{}]", reg.getAction()), inner); - } - } - } - } - public void executeHandshake(DiscoveryNode node, TcpChannel channel, ConnectionProfile profile, ActionListener listener) { - handshaker.sendHandshake(responseHandlers.newRequestId(), node, channel, profile.getHandshakeTimeout(), listener); + long requestId = inboundHandler.getResponseHandlers().newRequestId(); + handshaker.sendHandshake(requestId, node, channel, profile.getHandshakeTimeout(), listener); } final TransportKeepAlive getKeepAlive() { @@ -1037,7 +815,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements * * @throws IllegalStateException if the transport is not started / open */ - protected final void ensureOpen() { + private void ensureOpen() { if (lifecycle.started() == false) { throw new IllegalStateException("transport has been stopped"); } @@ -1046,7 +824,8 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements @Override public final TransportStats getStats() { MeanMetric transmittedBytes = outboundHandler.getTransmittedBytes(); - return new TransportStats(acceptedChannels.size(), readBytesMetric.count(), readBytesMetric.sum(), transmittedBytes.count(), + MeanMetric readBytes = inboundHandler.getReadBytes(); + return new TransportStats(acceptedChannels.size(), readBytes.count(), readBytes.sum(), transmittedBytes.count(), transmittedBytes.sum()); } @@ -1107,12 +886,12 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements @Override public final ResponseHandlers getResponseHandlers() { - return responseHandlers; + return inboundHandler.getResponseHandlers(); } @Override public final RequestHandlerRegistry getRequestHandler(String action) { - return requestHandlers.get(action); + return inboundHandler.getRequestHandler(action); } private final class ChannelsConnectedListener implements ActionListener { diff --git a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java new file mode 100644 index 00000000000..9c5d56315f6 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java @@ -0,0 +1,209 @@ +/* + * 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.ElasticsearchException; +import org.elasticsearch.Version; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class InboundHandlerTests extends ESTestCase { + + private final TestThreadPool threadPool = new TestThreadPool(getClass().getName()); + private final Version version = Version.CURRENT; + + private TaskManager taskManager; + private InboundHandler handler; + private FakeTcpChannel channel; + + @Before + public void setUp() throws Exception { + super.setUp(); + TransportLogger transportLogger = new TransportLogger(); + taskManager = new TaskManager(Settings.EMPTY, threadPool, Collections.emptySet()); + channel = new FakeTcpChannel(randomBoolean(), buildNewFakeTransportAddress().address(), buildNewFakeTransportAddress().address()); + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); + InboundMessage.Reader reader = new InboundMessage.Reader(version, namedWriteableRegistry, threadPool.getThreadContext()); + TransportHandshaker handshaker = new TransportHandshaker(version, threadPool, (n, c, r, v) -> { + }, (v, f, c, r, r_id) -> { + }); + TransportKeepAlive keepAlive = new TransportKeepAlive(threadPool, TcpChannel::sendMessage); + OutboundHandler outboundHandler = new OutboundHandler("node", version, new String[0], threadPool, BigArrays.NON_RECYCLING_INSTANCE, + transportLogger); + handler = new InboundHandler(threadPool, outboundHandler, reader, new NoneCircuitBreakerService(), transportLogger, handshaker, + keepAlive); + } + + @After + public void tearDown() throws Exception { + ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); + super.tearDown(); + } + + public void testPing() throws Exception { + AtomicReference channelCaptor = new AtomicReference<>(); + RequestHandlerRegistry registry = new RequestHandlerRegistry<>("test-request", TestRequest::new, taskManager, + (request, channel, task) -> channelCaptor.set(channel), ThreadPool.Names.SAME, false, true); + handler.registerRequestHandler(registry); + + handler.inboundMessage(channel, BytesArray.EMPTY); + assertEquals(1, handler.getReadBytes().count()); + assertEquals(6, handler.getReadBytes().sum()); + if (channel.isServerChannel()) { + BytesReference ping = channel.getMessageCaptor().get(); + assertEquals('E', ping.get(0)); + assertEquals(6, ping.length()); + } + } + + public void testRequestAndResponse() throws Exception { + String action = "test-request"; + boolean isCompressed = randomBoolean(); + boolean isError = randomBoolean(); + AtomicReference requestCaptor = new AtomicReference<>(); + AtomicReference responseCaptor = new AtomicReference<>(); + AtomicReference exceptionCaptor = new AtomicReference<>(); + AtomicReference channelCaptor = new AtomicReference<>(); + + long requestId = handler.getResponseHandlers().add(new Transport.ResponseContext<>(new TransportResponseHandler() { + @Override + public void handleResponse(TestResponse response) { + responseCaptor.set(response); + } + + @Override + public void handleException(TransportException exp) { + exceptionCaptor.set(exp); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public TestResponse read(StreamInput in) throws IOException { + return new TestResponse(in); + } + }, null, action)); + RequestHandlerRegistry registry = new RequestHandlerRegistry<>(action, TestRequest::new, taskManager, + (request, channel, task) -> { + channelCaptor.set(channel); + requestCaptor.set(request); + }, ThreadPool.Names.SAME, false, true); + handler.registerRequestHandler(registry); + String requestValue = randomAlphaOfLength(10); + OutboundMessage.Request request = new OutboundMessage.Request(threadPool.getThreadContext(), new String[0], + new TestRequest(requestValue), version, action, requestId, false, isCompressed); + + BytesReference bytes = request.serialize(new BytesStreamOutput()); + handler.inboundMessage(channel, bytes.slice(6, bytes.length() - 6)); + + TransportChannel transportChannel = channelCaptor.get(); + assertEquals(Version.CURRENT, transportChannel.getVersion()); + assertEquals("transport", transportChannel.getChannelType()); + assertEquals(requestValue, requestCaptor.get().value); + + String responseValue = randomAlphaOfLength(10); + if (isError) { + transportChannel.sendResponse(new ElasticsearchException("boom")); + } else { + transportChannel.sendResponse(new TestResponse(responseValue)); + } + BytesReference serializedResponse = channel.getMessageCaptor().get(); + handler.inboundMessage(channel, serializedResponse.slice(6, serializedResponse.length() - 6)); + + if (isError) { + assertTrue(exceptionCaptor.get() instanceof RemoteTransportException); + assertTrue(exceptionCaptor.get().getCause() instanceof ElasticsearchException); + assertEquals("boom", exceptionCaptor.get().getCause().getMessage()); + } else { + assertEquals(responseValue, responseCaptor.get().value); + } + } + + private static class TestRequest extends TransportRequest { + + String value; + + private TestRequest(String value) { + this.value = value; + } + + private TestRequest(StreamInput in) throws IOException { + super(in); + this.value = in.readString(); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(value); + } + } + + private static class TestResponse extends TransportResponse { + + String value; + + private TestResponse(String value) { + this.value = value; + } + + private TestResponse(StreamInput in) throws IOException { + super(in); + this.value = in.readString(); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(value); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java index 9d038032443..4519513db28 100644 --- a/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java +++ b/server/src/test/java/org/elasticsearch/transport/TcpTransportTests.java @@ -19,37 +19,15 @@ package org.elasticsearch.transport; -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressorFactory; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.network.CloseableChannel; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.io.StreamCorruptedException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.mockito.Mockito.mock; /** Unit tests for {@link TcpTransport} */ public class TcpTransportTests extends ESTestCase { @@ -157,141 +135,6 @@ public class TcpTransportTests extends ESTestCase { assertEquals(102, addresses[2].getPort()); } - @SuppressForbidden(reason = "Allow accessing localhost") - public void testCompressRequestAndResponse() throws IOException { - final boolean compressed = randomBoolean(); - Req request = new Req(randomRealisticUnicodeOfLengthBetween(10, 100)); - ThreadPool threadPool = new TestThreadPool(TcpTransportTests.class.getName()); - AtomicReference messageCaptor = new AtomicReference<>(); - try { - TcpTransport transport = new TcpTransport(Settings.EMPTY, Version.CURRENT, threadPool, - PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService(), null, null) { - - @Override - protected FakeServerChannel bind(String name, InetSocketAddress address) throws IOException { - return null; - } - - @Override - protected FakeTcpChannel initiateChannel(DiscoveryNode node) throws IOException { - return new FakeTcpChannel(false); - } - - @Override - protected void stopInternal() { - } - - @Override - public Releasable openConnection(DiscoveryNode node, ConnectionProfile profile, ActionListener listener) { - if (compressed) { - assertTrue(profile.getCompressionEnabled()); - } - int numConnections = profile.getNumConnections(); - ArrayList fakeChannels = new ArrayList<>(numConnections); - for (int i = 0; i < numConnections; ++i) { - fakeChannels.add(new FakeTcpChannel(false, messageCaptor)); - } - listener.onResponse(new NodeChannels(node, fakeChannels, profile, Version.CURRENT)); - return () -> CloseableChannel.closeChannels(fakeChannels, false); - } - }; - - DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT); - ConnectionProfile.Builder profileBuilder = new ConnectionProfile.Builder(TestProfiles.LIGHT_PROFILE); - if (compressed) { - profileBuilder.setCompressionEnabled(true); - } else { - profileBuilder.setCompressionEnabled(false); - } - PlainActionFuture future = PlainActionFuture.newFuture(); - transport.openConnection(node, profileBuilder.build(), future); - Transport.Connection connection = future.actionGet(); - connection.sendRequest(42, "foobar", request, TransportRequestOptions.EMPTY); - transport.registerRequestHandler(new RequestHandlerRegistry<>("foobar", Req::new, mock(TaskManager.class), - (request1, channel, task) -> channel.sendResponse(TransportResponse.Empty.INSTANCE), ThreadPool.Names.SAME, - true, true)); - - BytesReference reference = messageCaptor.get(); - assertNotNull(reference); - - AtomicReference responseCaptor = new AtomicReference<>(); - InetSocketAddress address = new InetSocketAddress(InetAddress.getLocalHost(), 0); - FakeTcpChannel responseChannel = new FakeTcpChannel(true, address, address, "profile", responseCaptor); - transport.messageReceived(reference.slice(6, reference.length() - 6), responseChannel); - - - StreamInput streamIn = responseCaptor.get().streamInput(); - streamIn.skip(TcpHeader.MARKER_BYTES_SIZE); - @SuppressWarnings("unused") - int len = streamIn.readInt(); - long requestId = streamIn.readLong(); - assertEquals(42, requestId); - byte status = streamIn.readByte(); - Version version = Version.fromId(streamIn.readInt()); - assertEquals(Version.CURRENT, version); - assertEquals(compressed, TransportStatus.isCompress(status)); - assertFalse(TransportStatus.isRequest(status)); - if (compressed) { - final int bytesConsumed = TcpHeader.HEADER_SIZE; - streamIn = CompressorFactory.compressor(reference.slice(bytesConsumed, reference.length() - bytesConsumed)) - .streamInput(streamIn); - } - threadPool.getThreadContext().readHeaders(streamIn); - TransportResponse.Empty.INSTANCE.readFrom(streamIn); - - } finally { - ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); - } - } - - private static final class FakeServerChannel implements TcpServerChannel { - - @Override - public void close() { - } - - @Override - public String getProfile() { - return null; - } - - @Override - public void addCloseListener(ActionListener listener) { - } - - @Override - public boolean isOpen() { - return false; - } - - @Override - public InetSocketAddress getLocalAddress() { - return null; - } - } - - private static final class Req extends TransportRequest { - public String value; - - private Req(String value) { - this.value = value; - } - - private Req(StreamInput in) throws IOException { - value = in.readString(); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - value = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(value); - } - } - public void testDecodeWithIncompleteHeader() throws IOException { BytesStreamOutput streamOutput = new BytesStreamOutput(1 << 14); streamOutput.write('E'); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index cb03742c297..0b2a4984fa2 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -34,12 +34,10 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.network.CloseableChannel; -import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -47,11 +45,9 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.mocksocket.MockServerSocket; import org.elasticsearch.node.Node; import org.elasticsearch.tasks.Task; @@ -63,7 +59,6 @@ import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.StubbableTransport; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.nio.MockNioTransport; import org.junit.After; import org.junit.Before; @@ -2001,34 +1996,23 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase { public void testTcpHandshake() { assumeTrue("only tcp transport has a handshake method", serviceA.getOriginalTransport() instanceof TcpTransport); - TcpTransport originalTransport = (TcpTransport) serviceA.getOriginalTransport(); - NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); - - MockNioTransport transport = new MockNioTransport(Settings.EMPTY, Version.CURRENT, threadPool, - new NetworkService(Collections.emptyList()), PageCacheRecycler.NON_RECYCLING_INSTANCE, namedWriteableRegistry, - new NoneCircuitBreakerService()) { - @Override - protected void handleRequest(TcpChannel channel, InboundMessage.Request request, int messageLengthBytes) - throws IOException { - // we flip the isHandshake bit back and act like the handler is not found - byte status = (byte) (request.status & ~(1 << 3)); - Version version = request.getVersion(); - InboundMessage.Request nonHandshakeRequest = new InboundMessage.Request(request.threadContext, version, - status, request.getRequestId(), request.getActionName(), request.getFeatures(), request.getStreamInput()); - super.handleRequest(channel, nonHandshakeRequest, messageLengthBytes); - } - }; - - try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, transport, Version.CURRENT, threadPool, - null, Collections.emptySet())) { + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { + service.addMessageListener(new TransportMessageListener() { + @Override + public void onRequestReceived(long requestId, String action) { + if (TransportHandshaker.HANDSHAKE_ACTION_NAME.equals(action)) { + throw new ActionNotFoundTransportException(action); + } + } + }); service.start(); service.acceptIncomingRequests(); // this acts like a node that doesn't have support for handshakes DiscoveryNode node = - new DiscoveryNode("TS_TPC", "TS_TPC", transport.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); + new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); ConnectTransportException exception = expectThrows(ConnectTransportException.class, () -> serviceA.connectToNode(node)); - assertTrue(exception.getCause() instanceof IllegalStateException); - assertEquals("handshake failed", exception.getCause().getMessage()); + assertTrue(exception.getCause() instanceof TransportException); + assertEquals("handshake failed because connection reset", exception.getCause().getMessage()); } ConnectionProfile connectionProfile = ConnectionProfile.buildDefaultConnectionProfile(Settings.EMPTY); @@ -2036,7 +2020,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase { DiscoveryNode node = new DiscoveryNode("TS_TPC", "TS_TPC", service.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); PlainActionFuture future = PlainActionFuture.newFuture(); - originalTransport.openConnection(node, connectionProfile, future); + serviceA.getOriginalTransport().openConnection(node, connectionProfile, future); try (Transport.Connection connection = future.actionGet()) { assertEquals(connection.getVersion(), Version.CURRENT); }