|
@@ -26,24 +26,16 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
|
|
import org.elasticsearch.ElasticsearchException;
|
|
|
import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
-import org.elasticsearch.action.NotifyOnceListener;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.common.Booleans;
|
|
|
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.bytes.CompositeBytesReference;
|
|
|
import org.elasticsearch.common.collect.MapBuilder;
|
|
|
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
|
|
import org.elasticsearch.common.component.Lifecycle;
|
|
|
-import org.elasticsearch.common.compress.Compressor;
|
|
|
-import org.elasticsearch.common.compress.CompressorFactory;
|
|
|
-import org.elasticsearch.common.compress.NotCompressedException;
|
|
|
-import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
|
|
-import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
|
|
|
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
|
|
-import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
|
|
|
import org.elasticsearch.common.io.stream.StreamInput;
|
|
|
import org.elasticsearch.common.lease.Releasable;
|
|
|
import org.elasticsearch.common.metrics.MeanMetric;
|
|
@@ -64,17 +56,14 @@ 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.core.internal.io.IOUtils;
|
|
|
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
|
|
import org.elasticsearch.monitor.jvm.JvmInfo;
|
|
|
import org.elasticsearch.node.Node;
|
|
|
import org.elasticsearch.rest.RestStatus;
|
|
|
import org.elasticsearch.threadpool.ThreadPool;
|
|
|
|
|
|
-import java.io.Closeable;
|
|
|
import java.io.IOException;
|
|
|
import java.io.StreamCorruptedException;
|
|
|
-import java.io.UncheckedIOException;
|
|
|
import java.net.BindException;
|
|
|
import java.net.InetAddress;
|
|
|
import java.net.InetSocketAddress;
|
|
@@ -136,8 +125,6 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
private final Map<String, List<TcpServerChannel>> serverChannels = newConcurrentMap();
|
|
|
private final Set<TcpChannel> acceptedChannels = ConcurrentCollections.newConcurrentSet();
|
|
|
|
|
|
- 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
|
|
|
private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
|
|
@@ -145,15 +132,16 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
private final String transportName;
|
|
|
|
|
|
private final MeanMetric readBytesMetric = new MeanMetric();
|
|
|
- private final MeanMetric transmittedBytesMetric = new MeanMetric();
|
|
|
private volatile Map<String, RequestHandlerRegistry<? extends TransportRequest>> 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 String nodeName;
|
|
|
|
|
|
- public TcpTransport(String transportName, Settings settings, Version version, ThreadPool threadPool,
|
|
|
+ public TcpTransport(String transportName, Settings settings, Version version, ThreadPool threadPool,
|
|
|
PageCacheRecycler pageCacheRecycler, CircuitBreakerService circuitBreakerService,
|
|
|
NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService) {
|
|
|
this.settings = settings;
|
|
@@ -163,17 +151,18 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
this.bigArrays = new BigArrays(pageCacheRecycler, circuitBreakerService, CircuitBreaker.IN_FLIGHT_REQUESTS);
|
|
|
this.pageCacheRecycler = pageCacheRecycler;
|
|
|
this.circuitBreakerService = circuitBreakerService;
|
|
|
- this.namedWriteableRegistry = namedWriteableRegistry;
|
|
|
this.networkService = networkService;
|
|
|
this.transportName = transportName;
|
|
|
this.transportLogger = new TransportLogger();
|
|
|
+ this.outboundHandler = new OutboundHandler(threadPool, bigArrays, transportLogger);
|
|
|
this.handshaker = new TransportHandshaker(version, threadPool,
|
|
|
(node, channel, requestId, v) -> sendRequestToChannel(node, channel, requestId,
|
|
|
TransportHandshaker.HANDSHAKE_ACTION_NAME, new TransportHandshaker.HandshakeRequest(version),
|
|
|
- TransportRequestOptions.EMPTY, v, false, TransportStatus.setHandshake((byte) 0)),
|
|
|
+ TransportRequestOptions.EMPTY, v, false, true),
|
|
|
(v, features, channel, response, requestId) -> sendResponse(v, features, channel, response, requestId,
|
|
|
- TransportHandshaker.HANDSHAKE_ACTION_NAME, false, TransportStatus.setHandshake((byte) 0)));
|
|
|
- this.keepAlive = new TransportKeepAlive(threadPool, this::internalSendMessage);
|
|
|
+ TransportHandshaker.HANDSHAKE_ACTION_NAME, false, true));
|
|
|
+ this.keepAlive = new TransportKeepAlive(threadPool, this.outboundHandler::sendBytes);
|
|
|
+ this.reader = new InboundMessage.Reader(version, namedWriteableRegistry, threadPool.getThreadContext());
|
|
|
this.nodeName = Node.NODE_NAME_SETTING.get(settings);
|
|
|
|
|
|
final Settings defaultFeatures = TransportSettings.DEFAULT_FEATURES_SETTING.get(settings);
|
|
@@ -280,7 +269,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
throw new NodeNotConnectedException(node, "connection already closed");
|
|
|
}
|
|
|
TcpChannel channel = channel(options.type());
|
|
|
- sendRequestToChannel(this.node, channel, requestId, action, request, options, getVersion(), compress, (byte) 0);
|
|
|
+ sendRequestToChannel(this.node, channel, requestId, action, request, options, getVersion(), compress);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -573,7 +562,8 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
for (Map.Entry<String, List<TcpServerChannel>> entry : serverChannels.entrySet()) {
|
|
|
String profile = entry.getKey();
|
|
|
List<TcpServerChannel> channels = entry.getValue();
|
|
|
- ActionListener<Void> closeFailLogger = ActionListener.wrap(c -> {},
|
|
|
+ ActionListener<Void> closeFailLogger = ActionListener.wrap(c -> {
|
|
|
+ },
|
|
|
e -> logger.warn(() -> new ParameterizedMessage("Error closing serverChannel for profile [{}]", profile), e));
|
|
|
channels.forEach(c -> c.addCloseListener(closeFailLogger));
|
|
|
CloseableChannel.closeChannels(channels, true);
|
|
@@ -628,26 +618,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
// in case we are able to return data, serialize the exception content and sent it back to the client
|
|
|
if (channel.isOpen()) {
|
|
|
BytesArray message = new BytesArray(e.getMessage().getBytes(StandardCharsets.UTF_8));
|
|
|
- ActionListener<Void> listener = new ActionListener<Void>() {
|
|
|
- @Override
|
|
|
- public void onResponse(Void aVoid) {
|
|
|
- CloseableChannel.closeChannel(channel);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(Exception e) {
|
|
|
- logger.debug("failed to send message to httpOnTransport channel", e);
|
|
|
- CloseableChannel.closeChannel(channel);
|
|
|
- }
|
|
|
- };
|
|
|
- // We do not call internalSendMessage because we are not sending a message that is an
|
|
|
- // elasticsearch binary message. We are just serializing an exception here. Not formatting it
|
|
|
- // as an elasticsearch transport message.
|
|
|
- try {
|
|
|
- channel.sendMessage(message, new SendListener(channel, message.length(), listener));
|
|
|
- } catch (Exception ex) {
|
|
|
- listener.onFailure(ex);
|
|
|
- }
|
|
|
+ outboundHandler.sendBytes(channel, message, ActionListener.wrap(() -> CloseableChannel.closeChannel(channel)));
|
|
|
}
|
|
|
} else {
|
|
|
logger.warn(() -> new ParameterizedMessage("exception caught on transport layer [{}], closing connection", channel), e);
|
|
@@ -691,65 +662,21 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
*/
|
|
|
protected abstract void stopInternal();
|
|
|
|
|
|
- private boolean canCompress(TransportRequest request) {
|
|
|
- return request instanceof BytesTransportRequest == false;
|
|
|
- }
|
|
|
-
|
|
|
private void sendRequestToChannel(final DiscoveryNode node, final TcpChannel channel, final long requestId, final String action,
|
|
|
final TransportRequest request, TransportRequestOptions options, Version channelVersion,
|
|
|
- boolean compressRequest, byte status) throws IOException, TransportException {
|
|
|
-
|
|
|
- // only compress if asked and the request is not bytes. Otherwise only
|
|
|
- // the header part is compressed, and the "body" can't be extracted as compressed
|
|
|
- final boolean compressMessage = compressRequest && canCompress(request);
|
|
|
-
|
|
|
- status = TransportStatus.setRequest(status);
|
|
|
- ReleasableBytesStreamOutput bStream = new ReleasableBytesStreamOutput(bigArrays);
|
|
|
- final CompressibleBytesOutputStream stream = new CompressibleBytesOutputStream(bStream, compressMessage);
|
|
|
- boolean addedReleaseListener = false;
|
|
|
- try {
|
|
|
- if (compressMessage) {
|
|
|
- status = TransportStatus.setCompress(status);
|
|
|
- }
|
|
|
-
|
|
|
- // we pick the smallest of the 2, to support both backward and forward compatibility
|
|
|
- // note, this is the only place we need to do this, since from here on, we use the serialized version
|
|
|
- // as the version to use also when the node receiving this request will send the response with
|
|
|
- Version version = Version.min(this.version, channelVersion);
|
|
|
-
|
|
|
- stream.setVersion(version);
|
|
|
- threadPool.getThreadContext().writeTo(stream);
|
|
|
- if (version.onOrAfter(Version.V_6_3_0)) {
|
|
|
- stream.writeStringArray(features);
|
|
|
- }
|
|
|
- stream.writeString(action);
|
|
|
- BytesReference message = buildMessage(requestId, status, node.getVersion(), request, stream);
|
|
|
- final TransportRequestOptions finalOptions = options;
|
|
|
- // this might be called in a different thread
|
|
|
- ReleaseListener releaseListener = new ReleaseListener(stream,
|
|
|
- () -> messageListener.onRequestSent(node, requestId, action, request, finalOptions));
|
|
|
- internalSendMessage(channel, message, releaseListener);
|
|
|
- addedReleaseListener = true;
|
|
|
- } finally {
|
|
|
- if (!addedReleaseListener) {
|
|
|
- IOUtils.close(stream);
|
|
|
- }
|
|
|
- }
|
|
|
+ boolean compressRequest) throws IOException, TransportException {
|
|
|
+ sendRequestToChannel(node, channel, requestId, action, request, options, channelVersion, compressRequest, false);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * sends a message to the given channel, using the given callbacks.
|
|
|
- */
|
|
|
- private void internalSendMessage(TcpChannel channel, BytesReference message, ActionListener<Void> listener) {
|
|
|
- channel.getChannelStats().markAccessed(threadPool.relativeTimeInMillis());
|
|
|
- transportLogger.logOutboundMessage(channel, message);
|
|
|
- try {
|
|
|
- channel.sendMessage(message, new SendListener(channel, message.length(), listener));
|
|
|
- } catch (Exception ex) {
|
|
|
- // call listener to ensure that any resources are released
|
|
|
- listener.onFailure(ex);
|
|
|
- onException(channel, ex);
|
|
|
- }
|
|
|
+ private void sendRequestToChannel(final DiscoveryNode node, final TcpChannel channel, final long requestId, final String action,
|
|
|
+ final TransportRequest request, TransportRequestOptions options, Version channelVersion,
|
|
|
+ boolean compressRequest, boolean isHandshake) throws IOException, TransportException {
|
|
|
+ Version version = Version.min(this.version, channelVersion);
|
|
|
+ OutboundMessage.Request message = new OutboundMessage.Request(threadPool.getThreadContext(), features, request, version, action,
|
|
|
+ requestId, isHandshake, compressRequest);
|
|
|
+ ActionListener<Void> listener = ActionListener.wrap(() ->
|
|
|
+ messageListener.onRequestSent(node, requestId, action, request, options));
|
|
|
+ outboundHandler.sendMessage(channel, message, listener);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -769,23 +696,13 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
final Exception error,
|
|
|
final long requestId,
|
|
|
final String action) throws IOException {
|
|
|
- try (BytesStreamOutput stream = new BytesStreamOutput()) {
|
|
|
- stream.setVersion(nodeVersion);
|
|
|
- stream.setFeatures(features);
|
|
|
- RemoteTransportException tx = new RemoteTransportException(
|
|
|
- nodeName, new TransportAddress(channel.getLocalAddress()), action, error);
|
|
|
- threadPool.getThreadContext().writeTo(stream);
|
|
|
- stream.writeException(tx);
|
|
|
- byte status = 0;
|
|
|
- status = TransportStatus.setResponse(status);
|
|
|
- status = TransportStatus.setError(status);
|
|
|
- final BytesReference bytes = stream.bytes();
|
|
|
- final BytesReference header = buildHeader(requestId, status, nodeVersion, bytes.length());
|
|
|
- CompositeBytesReference message = new CompositeBytesReference(header, bytes);
|
|
|
- ReleaseListener releaseListener = new ReleaseListener(null,
|
|
|
- () -> messageListener.onResponseSent(requestId, action, error));
|
|
|
- internalSendMessage(channel, message, releaseListener);
|
|
|
- }
|
|
|
+ Version version = Version.min(this.version, nodeVersion);
|
|
|
+ TransportAddress address = new TransportAddress(channel.getLocalAddress());
|
|
|
+ RemoteTransportException tx = new RemoteTransportException(nodeName, address, action, error);
|
|
|
+ OutboundMessage.Response message = new OutboundMessage.Response(threadPool.getThreadContext(), features, tx, version, requestId,
|
|
|
+ false, false);
|
|
|
+ ActionListener<Void> listener = ActionListener.wrap(() -> messageListener.onResponseSent(requestId, action, error));
|
|
|
+ outboundHandler.sendMessage(channel, message, listener);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -801,7 +718,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
final long requestId,
|
|
|
final String action,
|
|
|
final boolean compress) throws IOException {
|
|
|
- sendResponse(nodeVersion, features, channel, response, requestId, action, compress, (byte) 0);
|
|
|
+ sendResponse(nodeVersion, features, channel, response, requestId, action, compress, false);
|
|
|
}
|
|
|
|
|
|
private void sendResponse(
|
|
@@ -812,82 +729,18 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
final long requestId,
|
|
|
final String action,
|
|
|
boolean compress,
|
|
|
- byte status) throws IOException {
|
|
|
-
|
|
|
- status = TransportStatus.setResponse(status);
|
|
|
- ReleasableBytesStreamOutput bStream = new ReleasableBytesStreamOutput(bigArrays);
|
|
|
- CompressibleBytesOutputStream stream = new CompressibleBytesOutputStream(bStream, compress);
|
|
|
- boolean addedReleaseListener = false;
|
|
|
- try {
|
|
|
- if (compress) {
|
|
|
- status = TransportStatus.setCompress(status);
|
|
|
- }
|
|
|
- threadPool.getThreadContext().writeTo(stream);
|
|
|
- stream.setVersion(nodeVersion);
|
|
|
- stream.setFeatures(features);
|
|
|
- BytesReference message = buildMessage(requestId, status, nodeVersion, response, stream);
|
|
|
-
|
|
|
- // this might be called in a different thread
|
|
|
- ReleaseListener releaseListener = new ReleaseListener(stream,
|
|
|
- () -> messageListener.onResponseSent(requestId, action, response));
|
|
|
- internalSendMessage(channel, message, releaseListener);
|
|
|
- addedReleaseListener = true;
|
|
|
- } finally {
|
|
|
- if (!addedReleaseListener) {
|
|
|
- IOUtils.close(stream);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Writes the Tcp message header into a bytes reference.
|
|
|
- *
|
|
|
- * @param requestId the request ID
|
|
|
- * @param status the request status
|
|
|
- * @param protocolVersion the protocol version used to serialize the data in the message
|
|
|
- * @param length the payload length in bytes
|
|
|
- * @see TcpHeader
|
|
|
- */
|
|
|
- private BytesReference buildHeader(long requestId, byte status, Version protocolVersion, int length) throws IOException {
|
|
|
- try (BytesStreamOutput headerOutput = new BytesStreamOutput(TcpHeader.HEADER_SIZE)) {
|
|
|
- headerOutput.setVersion(protocolVersion);
|
|
|
- TcpHeader.writeHeader(headerOutput, requestId, status, protocolVersion, length);
|
|
|
- final BytesReference bytes = headerOutput.bytes();
|
|
|
- assert bytes.length() == TcpHeader.HEADER_SIZE : "header size mismatch expected: " + TcpHeader.HEADER_SIZE + " but was: "
|
|
|
- + bytes.length();
|
|
|
- return bytes;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Serializes the given message into a bytes representation
|
|
|
- */
|
|
|
- private BytesReference buildMessage(long requestId, byte status, Version nodeVersion, TransportMessage message,
|
|
|
- CompressibleBytesOutputStream stream) throws IOException {
|
|
|
- final BytesReference zeroCopyBuffer;
|
|
|
- if (message instanceof BytesTransportRequest) { // what a shitty optimization - we should use a direct send method instead
|
|
|
- BytesTransportRequest bRequest = (BytesTransportRequest) message;
|
|
|
- assert nodeVersion.equals(bRequest.version());
|
|
|
- bRequest.writeThin(stream);
|
|
|
- zeroCopyBuffer = bRequest.bytes;
|
|
|
- } else {
|
|
|
- message.writeTo(stream);
|
|
|
- zeroCopyBuffer = BytesArray.EMPTY;
|
|
|
- }
|
|
|
- // we have to call materializeBytes() here before accessing the bytes. A CompressibleBytesOutputStream
|
|
|
- // might be implementing compression. And materializeBytes() ensures that some marker bytes (EOS marker)
|
|
|
- // are written. Otherwise we barf on the decompressing end when we read past EOF on purpose in the
|
|
|
- // #validateRequest method. this might be a problem in deflate after all but it's important to write
|
|
|
- // the marker bytes.
|
|
|
- final BytesReference messageBody = stream.materializeBytes();
|
|
|
- final BytesReference header = buildHeader(requestId, status, stream.getVersion(), messageBody.length() + zeroCopyBuffer.length());
|
|
|
- return new CompositeBytesReference(header, messageBody, zeroCopyBuffer);
|
|
|
+ boolean isHandshake) throws IOException {
|
|
|
+ Version version = Version.min(this.version, nodeVersion);
|
|
|
+ OutboundMessage.Response message = new OutboundMessage.Response(threadPool.getThreadContext(), features, response, version,
|
|
|
+ requestId, isHandshake, compress);
|
|
|
+ ActionListener<Void> listener = ActionListener.wrap(() -> messageListener.onResponseSent(requestId, action, response));
|
|
|
+ outboundHandler.sendMessage(channel, message, listener);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Handles inbound message that has been decoded.
|
|
|
*
|
|
|
- * @param channel the channel the message if fomr
|
|
|
+ * @param channel the channel the message is from
|
|
|
* @param message the message
|
|
|
*/
|
|
|
public void inboundMessage(TcpChannel channel, BytesReference message) {
|
|
@@ -1055,53 +908,26 @@ 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 {
|
|
|
- String profileName = channel.getProfile();
|
|
|
+ readBytesMetric.inc(reference.length() + TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE);
|
|
|
InetSocketAddress remoteAddress = channel.getRemoteAddress();
|
|
|
- int messageLengthBytes = reference.length();
|
|
|
- final int totalMessageSize = messageLengthBytes + TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE;
|
|
|
- readBytesMetric.inc(totalMessageSize);
|
|
|
- // we have additional bytes to read, outside of the header
|
|
|
- boolean hasMessageBytesToRead = (totalMessageSize - TcpHeader.HEADER_SIZE) > 0;
|
|
|
- StreamInput streamIn = reference.streamInput();
|
|
|
- boolean success = false;
|
|
|
- try (ThreadContext.StoredContext tCtx = threadPool.getThreadContext().stashContext()) {
|
|
|
- long requestId = streamIn.readLong();
|
|
|
- byte status = streamIn.readByte();
|
|
|
- Version version = Version.fromId(streamIn.readInt());
|
|
|
- if (TransportStatus.isCompress(status) && hasMessageBytesToRead && streamIn.available() > 0) {
|
|
|
- Compressor compressor;
|
|
|
- try {
|
|
|
- final int bytesConsumed = TcpHeader.REQUEST_ID_SIZE + TcpHeader.STATUS_SIZE + TcpHeader.VERSION_ID_SIZE;
|
|
|
- compressor = CompressorFactory.compressor(reference.slice(bytesConsumed, reference.length() - bytesConsumed));
|
|
|
- } catch (NotCompressedException ex) {
|
|
|
- int maxToRead = Math.min(reference.length(), 10);
|
|
|
- StringBuilder sb = new StringBuilder("stream marked as compressed, but no compressor found, first [").append(maxToRead)
|
|
|
- .append("] content bytes out of [").append(reference.length())
|
|
|
- .append("] readable bytes with message size [").append(messageLengthBytes).append("] ").append("] are [");
|
|
|
- for (int i = 0; i < maxToRead; i++) {
|
|
|
- sb.append(reference.get(i)).append(",");
|
|
|
- }
|
|
|
- sb.append("]");
|
|
|
- throw new IllegalStateException(sb.toString());
|
|
|
- }
|
|
|
- streamIn = compressor.streamInput(streamIn);
|
|
|
- }
|
|
|
- final boolean isHandshake = TransportStatus.isHandshake(status);
|
|
|
- ensureVersionCompatibility(version, this.version, isHandshake);
|
|
|
- streamIn = new NamedWriteableAwareStreamInput(streamIn, namedWriteableRegistry);
|
|
|
- streamIn.setVersion(version);
|
|
|
- threadPool.getThreadContext().readHeaders(streamIn);
|
|
|
- threadPool.getThreadContext().putTransient("_remote_address", remoteAddress);
|
|
|
- if (TransportStatus.isRequest(status)) {
|
|
|
- handleRequest(channel, profileName, streamIn, requestId, messageLengthBytes, version, remoteAddress, status);
|
|
|
+
|
|
|
+ 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.RequestMessage) message, reference.length());
|
|
|
} else {
|
|
|
final TransportResponseHandler<?> handler;
|
|
|
- if (isHandshake) {
|
|
|
+ long requestId = message.getRequestId();
|
|
|
+ if (message.isHandshake()) {
|
|
|
handler = handshaker.removeHandlerForHandshake(requestId);
|
|
|
} else {
|
|
|
TransportResponseHandler<? extends TransportResponse> theHandler =
|
|
|
responseHandlers.onResponseReceived(requestId, messageListener);
|
|
|
- if (theHandler == null && TransportStatus.isError(status)) {
|
|
|
+ if (theHandler == null && message.isError()) {
|
|
|
handler = handshaker.removeHandlerForHandshake(requestId);
|
|
|
} else {
|
|
|
handler = theHandler;
|
|
@@ -1109,40 +935,20 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
}
|
|
|
// ignore if its null, the service logs it
|
|
|
if (handler != null) {
|
|
|
- if (TransportStatus.isError(status)) {
|
|
|
- handlerResponseError(streamIn, handler);
|
|
|
+ if (message.isError()) {
|
|
|
+ handlerResponseError(message.getStreamInput(), handler);
|
|
|
} else {
|
|
|
- handleResponse(remoteAddress, streamIn, handler);
|
|
|
+ handleResponse(remoteAddress, message.getStreamInput(), handler);
|
|
|
}
|
|
|
// Check the entire message has been read
|
|
|
- final int nextByte = streamIn.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 [" + TransportStatus.isError(status) + "]; resetting");
|
|
|
+ + handler + "], error [" + message.isError() + "]; resetting");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- success = true;
|
|
|
- } finally {
|
|
|
- if (success) {
|
|
|
- IOUtils.close(streamIn);
|
|
|
- } else {
|
|
|
- IOUtils.closeWhileHandlingException(streamIn);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- static void ensureVersionCompatibility(Version version, Version currentVersion, boolean isHandshake) {
|
|
|
- // for handshakes we are compatible with N-2 since otherwise we can't figure out our initial version
|
|
|
- // since we are compatible with N-1 and N+1 so we always send our minCompatVersion as the initial version in the
|
|
|
- // handshake. This looks odd but it's required to establish the connection correctly we check for real compatibility
|
|
|
- // once the connection is established
|
|
|
- final Version compatibilityVersion = isHandshake ? currentVersion.minimumCompatibilityVersion() : currentVersion;
|
|
|
- if (version.isCompatible(compatibilityVersion) == false) {
|
|
|
- final Version minCompatibilityVersion = isHandshake ? compatibilityVersion : compatibilityVersion.minimumCompatibilityVersion();
|
|
|
- String msg = "Received " + (isHandshake ? "handshake " : "") + "message from unsupported version: [";
|
|
|
- throw new IllegalStateException(msg + version + "] minimal compatible version is: [" + minCompatibilityVersion + "]");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1198,20 +1004,17 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
});
|
|
|
}
|
|
|
|
|
|
- protected String handleRequest(TcpChannel channel, String profileName, final StreamInput stream, long requestId,
|
|
|
- int messageLengthBytes, Version version, InetSocketAddress remoteAddress, byte status)
|
|
|
- throws IOException {
|
|
|
- final Set<String> features;
|
|
|
- if (version.onOrAfter(Version.V_6_3_0)) {
|
|
|
- features = Collections.unmodifiableSet(new TreeSet<>(Arrays.asList(stream.readStringArray())));
|
|
|
- } else {
|
|
|
- features = Collections.emptySet();
|
|
|
- }
|
|
|
- final String action = stream.readString();
|
|
|
+ protected void handleRequest(TcpChannel channel, InboundMessage.RequestMessage message, int messageLengthBytes) throws IOException {
|
|
|
+ final Set<String> 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 (TransportStatus.isHandshake(status)) {
|
|
|
+ if (message.isHandshake()) {
|
|
|
handshaker.handleHandshake(version, features, channel, requestId, stream);
|
|
|
} else {
|
|
|
final RequestHandlerRegistry reg = getRequestHandler(action);
|
|
@@ -1224,9 +1027,9 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
getInFlightRequestBreaker().addWithoutBreaking(messageLengthBytes);
|
|
|
}
|
|
|
transportChannel = new TcpTransportChannel(this, channel, transportName, action, requestId, version, features, profileName,
|
|
|
- messageLengthBytes, TransportStatus.isCompress(status));
|
|
|
+ messageLengthBytes, message.isCompress());
|
|
|
final TransportRequest request = reg.newRequest(stream);
|
|
|
- request.remoteAddress(new TransportAddress(remoteAddress));
|
|
|
+ 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));
|
|
@@ -1235,7 +1038,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
// the circuit breaker tripped
|
|
|
if (transportChannel == null) {
|
|
|
transportChannel = new TcpTransportChannel(this, channel, transportName, action, requestId, version, features,
|
|
|
- profileName, 0, TransportStatus.isCompress(status));
|
|
|
+ profileName, 0, message.isCompress());
|
|
|
}
|
|
|
try {
|
|
|
transportChannel.sendResponse(e);
|
|
@@ -1244,7 +1047,6 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
logger.warn(() -> new ParameterizedMessage("Failed to send error message back to client for action [{}]", action), inner);
|
|
|
}
|
|
|
}
|
|
|
- return action;
|
|
|
}
|
|
|
|
|
|
// This template method is needed to inject custom error checking logic in tests.
|
|
@@ -1321,70 +1123,11 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * This listener increments the transmitted bytes metric on success.
|
|
|
- */
|
|
|
- private class SendListener extends NotifyOnceListener<Void> {
|
|
|
-
|
|
|
- private final TcpChannel channel;
|
|
|
- private final long messageSize;
|
|
|
- private final ActionListener<Void> delegateListener;
|
|
|
-
|
|
|
- private SendListener(TcpChannel channel, long messageSize, ActionListener<Void> delegateListener) {
|
|
|
- this.channel = channel;
|
|
|
- this.messageSize = messageSize;
|
|
|
- this.delegateListener = delegateListener;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void innerOnResponse(Void v) {
|
|
|
- transmittedBytesMetric.inc(messageSize);
|
|
|
- delegateListener.onResponse(v);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void innerOnFailure(Exception e) {
|
|
|
- logger.warn(() -> new ParameterizedMessage("send message failed [channel: {}]", channel), e);
|
|
|
- delegateListener.onFailure(e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private class ReleaseListener implements ActionListener<Void> {
|
|
|
-
|
|
|
- private final Closeable optionalCloseable;
|
|
|
- private final Runnable transportAdaptorCallback;
|
|
|
-
|
|
|
- private ReleaseListener(Closeable optionalCloseable, Runnable transportAdaptorCallback) {
|
|
|
- this.optionalCloseable = optionalCloseable;
|
|
|
- this.transportAdaptorCallback = transportAdaptorCallback;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onResponse(Void aVoid) {
|
|
|
- closeAndCallback(null);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void onFailure(Exception e) {
|
|
|
- closeAndCallback(e);
|
|
|
- }
|
|
|
-
|
|
|
- private void closeAndCallback(final Exception e) {
|
|
|
- try {
|
|
|
- IOUtils.close(optionalCloseable, transportAdaptorCallback::run);
|
|
|
- } catch (final IOException inner) {
|
|
|
- if (e != null) {
|
|
|
- inner.addSuppressed(e);
|
|
|
- }
|
|
|
- throw new UncheckedIOException(inner);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public final TransportStats getStats() {
|
|
|
- return new TransportStats(acceptedChannels.size(), readBytesMetric.count(), readBytesMetric.sum(), transmittedBytesMetric.count(),
|
|
|
- transmittedBytesMetric.sum());
|
|
|
+ MeanMetric transmittedBytes = outboundHandler.getTransmittedBytes();
|
|
|
+ return new TransportStats(acceptedChannels.size(), readBytesMetric.count(), readBytesMetric.sum(), transmittedBytes.count(),
|
|
|
+ transmittedBytes.sum());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1559,7 +1302,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|
|
public void onTimeout() {
|
|
|
if (countDown.fastForward()) {
|
|
|
CloseableChannel.closeChannels(channels, false);
|
|
|
- listener.onFailure(new ConnectTransportException(node, "connect_timeout[" + connectionProfile.getConnectTimeout() + "]"));
|
|
|
+ listener.onFailure(new ConnectTransportException(node, "connect_timeout[" + connectionProfile.getConnectTimeout() + "]"));
|
|
|
}
|
|
|
}
|
|
|
}
|