diff --git a/core/src/main/java/org/elasticsearch/ElasticsearchException.java b/core/src/main/java/org/elasticsearch/ElasticsearchException.java
index ad5a2e79cd1..aa92fc176cf 100644
--- a/core/src/main/java/org/elasticsearch/ElasticsearchException.java
+++ b/core/src/main/java/org/elasticsearch/ElasticsearchException.java
@@ -30,6 +30,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.transport.TcpTransport;
import java.io.IOException;
import java.util.ArrayList;
@@ -496,7 +497,7 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
org.elasticsearch.index.shard.IndexShardStartedException::new, 23),
SEARCH_CONTEXT_MISSING_EXCEPTION(org.elasticsearch.search.SearchContextMissingException.class,
org.elasticsearch.search.SearchContextMissingException::new, 24),
- GENERAL_SCRIPT_EXCEPTION(org.elasticsearch.script.GeneralScriptException.class,
+ GENERAL_SCRIPT_EXCEPTION(org.elasticsearch.script.GeneralScriptException.class,
org.elasticsearch.script.GeneralScriptException::new, 25),
BATCH_OPERATION_EXCEPTION(org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException.class,
org.elasticsearch.index.shard.TranslogRecoveryPerformer.BatchOperationException::new, 26),
@@ -676,8 +677,8 @@ public class ElasticsearchException extends RuntimeException implements ToXConte
org.elasticsearch.indices.IndexAlreadyExistsException::new, 123),
SCRIPT_PARSE_EXCEPTION(org.elasticsearch.script.Script.ScriptParseException.class,
org.elasticsearch.script.Script.ScriptParseException::new, 124),
- HTTP_ON_TRANSPORT_EXCEPTION(org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException.class,
- org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException::new, 125),
+ HTTP_ON_TRANSPORT_EXCEPTION(TcpTransport.HttpOnTransportException.class,
+ TcpTransport.HttpOnTransportException::new, 125),
MAPPER_PARSING_EXCEPTION(org.elasticsearch.index.mapper.MapperParsingException.class,
org.elasticsearch.index.mapper.MapperParsingException::new, 126),
SEARCH_CONTEXT_EXCEPTION(org.elasticsearch.search.SearchContextException.class,
diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java
index 5786f6b1cfb..c9313fc08c4 100644
--- a/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java
+++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClient.java
@@ -50,8 +50,8 @@ import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.threadpool.ExecutorBuilder;
import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.TransportService;
-import org.elasticsearch.transport.netty.NettyTransport;
import java.io.Closeable;
import java.util.ArrayList;
@@ -107,7 +107,7 @@ public class TransportClient extends AbstractClient {
private PluginsService newPluginService(final Settings settings) {
final Settings.Builder settingsBuilder = Settings.builder()
- .put(NettyTransport.PING_SCHEDULE.getKey(), "5s") // enable by default the transport schedule ping interval
+ .put(TcpTransport.PING_SCHEDULE.getKey(), "5s") // enable by default the transport schedule ping interval
.put(InternalSettingsPreparer.prepareSettings(settings))
.put(NetworkService.NETWORK_SERVER.getKey(), false)
.put(CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE);
diff --git a/core/src/main/java/org/elasticsearch/common/io/Channels.java b/core/src/main/java/org/elasticsearch/common/io/Channels.java
index 2fa7ca1cdec..71a2d66dee3 100644
--- a/core/src/main/java/org/elasticsearch/common/io/Channels.java
+++ b/core/src/main/java/org/elasticsearch/common/io/Channels.java
@@ -20,7 +20,6 @@
package org.elasticsearch.common.io;
import org.elasticsearch.common.SuppressForbidden;
-import org.jboss.netty.buffer.ChannelBuffer;
import java.io.EOFException;
import java.io.IOException;
@@ -159,25 +158,6 @@ public final class Channels {
return bytesRead;
}
-
- /**
- * Copies bytes from source {@link org.jboss.netty.buffer.ChannelBuffer} to a {@link java.nio.channels.GatheringByteChannel}
- *
- * @param source ChannelBuffer to copy from
- * @param sourceIndex index in source to start copying from
- * @param length how many bytes to copy
- * @param channel target GatheringByteChannel
- */
- public static void writeToChannel(ChannelBuffer source, int sourceIndex, int length, GatheringByteChannel channel) throws IOException {
- while (length > 0) {
- int written = source.getBytes(sourceIndex, channel, length);
- sourceIndex += written;
- length -= written;
- }
- assert length == 0;
- }
-
-
/**
* Writes part of a byte array to a {@link java.nio.channels.WritableByteChannel}
*
diff --git a/core/src/main/java/org/elasticsearch/common/netty/KeepFrameDecoder.java b/core/src/main/java/org/elasticsearch/common/netty/KeepFrameDecoder.java
deleted file mode 100644
index d2b02b7c51e..00000000000
--- a/core/src/main/java/org/elasticsearch/common/netty/KeepFrameDecoder.java
+++ /dev/null
@@ -1,37 +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.common.netty;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.frame.FrameDecoder;
-
-/**
- * A marker to not remove frame decoder from the resulting jar so plugins can use it.
- */
-public class KeepFrameDecoder extends FrameDecoder {
-
- public static final KeepFrameDecoder decoder = new KeepFrameDecoder();
-
- @Override
- protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buffer) throws Exception {
- return null;
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/common/netty/ReleaseChannelFutureListener.java b/core/src/main/java/org/elasticsearch/common/netty/ReleaseChannelFutureListener.java
deleted file mode 100644
index 6f2979d13ca..00000000000
--- a/core/src/main/java/org/elasticsearch/common/netty/ReleaseChannelFutureListener.java
+++ /dev/null
@@ -1,42 +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.common.netty;
-
-import org.elasticsearch.common.lease.Releasable;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-
-/**
- * A channel listener that releases a {@link org.elasticsearch.common.lease.Releasable} when
- * the operation is complete.
- */
-public class ReleaseChannelFutureListener implements ChannelFutureListener {
-
- private final Releasable releasable;
-
- public ReleaseChannelFutureListener(Releasable releasable) {
- this.releasable = releasable;
- }
-
- @Override
- public void operationComplete(ChannelFuture future) throws Exception {
- releasable.close();
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
index 5cb37d14858..9469ac8b509 100644
--- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
+++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
@@ -89,6 +89,7 @@ import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.TransportSettings;
@@ -279,14 +280,14 @@ public final class ClusterSettings extends AbstractScopedSettings {
TransportSettings.PUBLISH_PORT,
TransportSettings.PORT,
NettyTransport.WORKER_COUNT,
- NettyTransport.CONNECTIONS_PER_NODE_RECOVERY,
- NettyTransport.CONNECTIONS_PER_NODE_BULK,
- NettyTransport.CONNECTIONS_PER_NODE_REG,
- NettyTransport.CONNECTIONS_PER_NODE_STATE,
- NettyTransport.CONNECTIONS_PER_NODE_PING,
- NettyTransport.PING_SCHEDULE,
- NettyTransport.TCP_BLOCKING_CLIENT,
- NettyTransport.TCP_CONNECT_TIMEOUT,
+ TcpTransport.CONNECTIONS_PER_NODE_RECOVERY,
+ TcpTransport.CONNECTIONS_PER_NODE_BULK,
+ TcpTransport.CONNECTIONS_PER_NODE_REG,
+ TcpTransport.CONNECTIONS_PER_NODE_STATE,
+ TcpTransport.CONNECTIONS_PER_NODE_PING,
+ TcpTransport.PING_SCHEDULE,
+ TcpTransport.TCP_BLOCKING_CLIENT,
+ TcpTransport.TCP_CONNECT_TIMEOUT,
NettyTransport.NETTY_MAX_CUMULATION_BUFFER_CAPACITY,
NettyTransport.NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS,
NettyTransport.NETTY_RECEIVE_PREDICTOR_SIZE,
@@ -294,12 +295,12 @@ public final class ClusterSettings extends AbstractScopedSettings {
NettyTransport.NETTY_RECEIVE_PREDICTOR_MAX,
NetworkService.NETWORK_SERVER,
NettyTransport.NETTY_BOSS_COUNT,
- NettyTransport.TCP_NO_DELAY,
- NettyTransport.TCP_KEEP_ALIVE,
- NettyTransport.TCP_REUSE_ADDRESS,
- NettyTransport.TCP_SEND_BUFFER_SIZE,
- NettyTransport.TCP_RECEIVE_BUFFER_SIZE,
- NettyTransport.TCP_BLOCKING_SERVER,
+ TcpTransport.TCP_NO_DELAY,
+ TcpTransport.TCP_KEEP_ALIVE,
+ TcpTransport.TCP_REUSE_ADDRESS,
+ TcpTransport.TCP_SEND_BUFFER_SIZE,
+ TcpTransport.TCP_RECEIVE_BUFFER_SIZE,
+ TcpTransport.TCP_BLOCKING_SERVER,
NetworkService.GLOBAL_NETWORK_HOST_SETTING,
NetworkService.GLOBAL_NETWORK_BINDHOST_SETTING,
NetworkService.GLOBAL_NETWORK_PUBLISHHOST_SETTING,
diff --git a/core/src/main/java/org/elasticsearch/http/netty/ESHttpResponseEncoder.java b/core/src/main/java/org/elasticsearch/http/netty/ESHttpResponseEncoder.java
index 36f2c3a138b..afa69a2fe02 100644
--- a/core/src/main/java/org/elasticsearch/http/netty/ESHttpResponseEncoder.java
+++ b/core/src/main/java/org/elasticsearch/http/netty/ESHttpResponseEncoder.java
@@ -19,7 +19,7 @@
package org.elasticsearch.http.netty;
-import org.elasticsearch.common.netty.NettyUtils;
+import org.elasticsearch.transport.netty.NettyUtils;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.buffer.CompositeChannelBuffer;
diff --git a/core/src/main/java/org/elasticsearch/http/netty/NettyHttpChannel.java b/core/src/main/java/org/elasticsearch/http/netty/NettyHttpChannel.java
index 3413a746963..c4253df2860 100644
--- a/core/src/main/java/org/elasticsearch/http/netty/NettyHttpChannel.java
+++ b/core/src/main/java/org/elasticsearch/http/netty/NettyHttpChannel.java
@@ -24,8 +24,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.lease.Releasable;
-import org.elasticsearch.common.netty.NettyUtils;
-import org.elasticsearch.common.netty.ReleaseChannelFutureListener;
+import org.elasticsearch.transport.netty.NettyUtils;
import org.elasticsearch.http.netty.cors.CorsHandler;
import org.elasticsearch.http.netty.pipelining.OrderedDownstreamChannelEvent;
import org.elasticsearch.http.netty.pipelining.OrderedUpstreamMessageEvent;
@@ -128,7 +127,7 @@ public final class NettyHttpChannel extends AbstractRestChannel {
}
if (content instanceof Releasable) {
- future.addListener(new ReleaseChannelFutureListener((Releasable) content));
+ future.addListener((x) -> ((Releasable)content).close());
addedReleaseListener = true;
}
diff --git a/core/src/main/java/org/elasticsearch/http/netty/NettyHttpRequest.java b/core/src/main/java/org/elasticsearch/http/netty/NettyHttpRequest.java
index d26841ead97..d62252bc0ce 100644
--- a/core/src/main/java/org/elasticsearch/http/netty/NettyHttpRequest.java
+++ b/core/src/main/java/org/elasticsearch/http/netty/NettyHttpRequest.java
@@ -21,7 +21,7 @@ package org.elasticsearch.http.netty;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.netty.NettyUtils;
+import org.elasticsearch.transport.netty.NettyUtils;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.support.RestUtils;
import org.jboss.netty.channel.Channel;
diff --git a/core/src/main/java/org/elasticsearch/http/netty/NettyHttpServerTransport.java b/core/src/main/java/org/elasticsearch/http/netty/NettyHttpServerTransport.java
index 22852b7c0ff..77c140ce7ce 100644
--- a/core/src/main/java/org/elasticsearch/http/netty/NettyHttpServerTransport.java
+++ b/core/src/main/java/org/elasticsearch/http/netty/NettyHttpServerTransport.java
@@ -24,8 +24,8 @@ import com.carrotsearch.hppc.IntSet;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.netty.NettyUtils;
-import org.elasticsearch.common.netty.OpenChannelsHandler;
+import org.elasticsearch.transport.netty.NettyUtils;
+import org.elasticsearch.transport.netty.OpenChannelsHandler;
import org.elasticsearch.common.network.NetworkAddress;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.Setting;
diff --git a/core/src/main/java/org/elasticsearch/transport/TcpHeader.java b/core/src/main/java/org/elasticsearch/transport/TcpHeader.java
new file mode 100644
index 00000000000..bbc54c0eb62
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/transport/TcpHeader.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.transport;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.common.io.stream.StreamOutput;
+
+import java.io.IOException;
+
+public class TcpHeader {
+ public static final int MARKER_BYTES_SIZE = 2 * 1;
+
+ public static final int MESSAGE_LENGTH_SIZE = 4;
+
+ public static final int REQUEST_ID_SIZE = 8;
+
+ public static final int STATUS_SIZE = 1;
+
+ public static final int VERSION_ID_SIZE = 4;
+
+ public static final int HEADER_SIZE = MARKER_BYTES_SIZE + MESSAGE_LENGTH_SIZE + REQUEST_ID_SIZE + STATUS_SIZE + VERSION_ID_SIZE;
+
+ public static void writeHeader(StreamOutput output, long requestId, byte status, Version version, int messageSize) throws IOException {
+ output.writeByte((byte)'E');
+ output.writeByte((byte)'S');
+ // write the size, the size indicates the remaining message size, not including the size int
+ output.writeInt(messageSize - TcpHeader.MARKER_BYTES_SIZE - TcpHeader.MESSAGE_LENGTH_SIZE);
+ output.writeLong(requestId);
+ output.writeByte(status);
+ output.writeInt(version.id);
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java
new file mode 100644
index 00000000000..f9b5eceb4b8
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -0,0 +1,1347 @@
+/*
+ * 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 com.carrotsearch.hppc.IntHashSet;
+import com.carrotsearch.hppc.IntSet;
+import org.apache.lucene.util.IOUtils;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ExceptionsHelper;
+import org.elasticsearch.Version;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+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.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.ReleasableBytesStream;
+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.io.stream.StreamOutput;
+import org.elasticsearch.common.lease.Releasable;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.metrics.CounterMetric;
+import org.elasticsearch.common.network.NetworkAddress;
+import org.elasticsearch.common.network.NetworkService;
+import org.elasticsearch.common.network.NetworkUtils;
+import org.elasticsearch.common.settings.Setting;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.BoundTransportAddress;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.elasticsearch.common.transport.PortsRange;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.concurrent.AbstractLifecycleRunnable;
+import org.elasticsearch.common.util.concurrent.AbstractRunnable;
+import org.elasticsearch.common.util.concurrent.KeyedLock;
+import org.elasticsearch.common.util.concurrent.ThreadContext;
+import org.elasticsearch.indices.breaker.CircuitBreakerService;
+import org.elasticsearch.monitor.jvm.JvmInfo;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.support.TransportStatus;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.StreamCorruptedException;
+import java.net.BindException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.nio.channels.CancelledKeyException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static java.util.Collections.unmodifiableMap;
+import static org.elasticsearch.common.settings.Setting.boolSetting;
+import static org.elasticsearch.common.settings.Setting.intSetting;
+import static org.elasticsearch.common.settings.Setting.timeSetting;
+import static org.elasticsearch.common.transport.NetworkExceptionHelper.isCloseConnectionException;
+import static org.elasticsearch.common.transport.NetworkExceptionHelper.isConnectException;
+import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
+
+/**
+ */
+public abstract class TcpTransport extends AbstractLifecycleComponent implements Transport {
+
+ public static final String HTTP_SERVER_WORKER_THREAD_NAME_PREFIX = "http_server_worker";
+ public static final String HTTP_SERVER_BOSS_THREAD_NAME_PREFIX = "http_server_boss";
+ public static final String TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX = "transport_client_worker";
+ public static final String TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX = "transport_client_boss";
+
+ // the scheduled internal ping interval setting, defaults to disabled (-1)
+ public static final Setting PING_SCHEDULE =
+ timeSetting("transport.ping_schedule", TimeValue.timeValueSeconds(-1), Setting.Property.NodeScope);
+ public static final Setting CONNECTIONS_PER_NODE_RECOVERY =
+ intSetting("transport.connections_per_node.recovery", 2, 1, Setting.Property.NodeScope);
+ public static final Setting CONNECTIONS_PER_NODE_BULK =
+ intSetting("transport.connections_per_node.bulk", 3, 1, Setting.Property.NodeScope);
+ public static final Setting CONNECTIONS_PER_NODE_REG =
+ intSetting("transport.connections_per_node.reg", 6, 1, Setting.Property.NodeScope);
+ public static final Setting CONNECTIONS_PER_NODE_STATE =
+ intSetting("transport.connections_per_node.state", 1, 1, Setting.Property.NodeScope);
+ public static final Setting CONNECTIONS_PER_NODE_PING =
+ intSetting("transport.connections_per_node.ping", 1, 1, Setting.Property.NodeScope);
+ public static final Setting TCP_CONNECT_TIMEOUT =
+ timeSetting("transport.tcp.connect_timeout", NetworkService.TcpSettings.TCP_CONNECT_TIMEOUT, Setting.Property.NodeScope);
+ public static final Setting TCP_NO_DELAY =
+ boolSetting("transport.tcp_no_delay", NetworkService.TcpSettings.TCP_NO_DELAY, Setting.Property.NodeScope);
+ public static final Setting TCP_KEEP_ALIVE =
+ boolSetting("transport.tcp.keep_alive", NetworkService.TcpSettings.TCP_KEEP_ALIVE, Setting.Property.NodeScope);
+ public static final Setting TCP_REUSE_ADDRESS =
+ boolSetting("transport.tcp.reuse_address", NetworkService.TcpSettings.TCP_REUSE_ADDRESS, Setting.Property.NodeScope);
+ public static final Setting TCP_BLOCKING_CLIENT =
+ boolSetting("transport.tcp.blocking_client", NetworkService.TcpSettings.TCP_BLOCKING_CLIENT, Setting.Property.NodeScope);
+ public static final Setting TCP_BLOCKING_SERVER =
+ boolSetting("transport.tcp.blocking_server", NetworkService.TcpSettings.TCP_BLOCKING_SERVER, Setting.Property.NodeScope);
+ public static final Setting TCP_SEND_BUFFER_SIZE =
+ Setting.byteSizeSetting("transport.tcp.send_buffer_size", NetworkService.TcpSettings.TCP_SEND_BUFFER_SIZE,
+ Setting.Property.NodeScope);
+ public static final Setting TCP_RECEIVE_BUFFER_SIZE =
+ Setting.byteSizeSetting("transport.tcp.receive_buffer_size", NetworkService.TcpSettings.TCP_RECEIVE_BUFFER_SIZE,
+ Setting.Property.NodeScope);
+
+ private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.9);
+ private static final int PING_DATA_SIZE = -1;
+
+ protected final int connectionsPerNodeRecovery;
+ protected final int connectionsPerNodeBulk;
+ protected final int connectionsPerNodeReg;
+ protected final int connectionsPerNodeState;
+ protected final int connectionsPerNodePing;
+ protected final TimeValue connectTimeout;
+ protected final boolean blockingClient;
+ private final CircuitBreakerService circuitBreakerService;
+ // package visibility for tests
+ protected final ScheduledPing scheduledPing;
+ private final TimeValue pingSchedule;
+ protected final ThreadPool threadPool;
+ private final BigArrays bigArrays;
+ protected final NetworkService networkService;
+
+ protected volatile TransportServiceAdapter transportServiceAdapter;
+ // node id to actual channel
+ protected final ConcurrentMap connectedNodes = newConcurrentMap();
+ protected final Map> serverChannels = newConcurrentMap();
+ protected final ConcurrentMap profileBoundAddresses = newConcurrentMap();
+
+ protected final KeyedLock connectionLock = new KeyedLock<>();
+ private final NamedWriteableRegistry namedWriteableRegistry;
+
+ // this lock is here to make sure we close this transport and disconnect all the client nodes
+ // connections while no connect operations is going on... (this might help with 100% CPU when stopping the transport?)
+ protected final ReadWriteLock globalLock = new ReentrantReadWriteLock();
+ protected final boolean compress;
+ protected volatile BoundTransportAddress boundAddress;
+ private final String transportName;
+
+ public TcpTransport(String transportName, Settings settings, ThreadPool threadPool, BigArrays bigArrays,
+ CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry,
+ NetworkService networkService) {
+ super(settings);
+ this.threadPool = threadPool;
+ this.bigArrays = bigArrays;
+ this.circuitBreakerService = circuitBreakerService;
+ this.scheduledPing = new ScheduledPing();
+ this.pingSchedule = PING_SCHEDULE.get(settings);
+ this.namedWriteableRegistry = namedWriteableRegistry;
+ this.compress = Transport.TRANSPORT_TCP_COMPRESS.get(settings);
+ this.networkService = networkService;
+ this.transportName = transportName;
+
+ this.connectionsPerNodeRecovery = CONNECTIONS_PER_NODE_RECOVERY.get(settings);
+ this.connectionsPerNodeBulk = CONNECTIONS_PER_NODE_BULK.get(settings);
+ this.connectionsPerNodeReg = CONNECTIONS_PER_NODE_REG.get(settings);
+ this.connectionsPerNodeState = CONNECTIONS_PER_NODE_STATE.get(settings);
+ this.connectionsPerNodePing = CONNECTIONS_PER_NODE_PING.get(settings);
+ this.connectTimeout = TCP_CONNECT_TIMEOUT.get(settings);
+ this.blockingClient = TCP_BLOCKING_CLIENT.get(settings);
+ }
+
+ @Override
+ protected void doStart() {
+ if (pingSchedule.millis() > 0) {
+ threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, scheduledPing);
+ }
+ }
+
+ @Override
+ public CircuitBreaker getInFlightRequestBreaker() {
+ // We always obtain a fresh breaker to reflect changes to the breaker configuration.
+ return circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS);
+ }
+
+ @Override
+ public void transportServiceAdapter(TransportServiceAdapter service) {
+ this.transportServiceAdapter = service;
+ }
+
+ public Settings settings() {
+ return this.settings;
+ }
+
+ public boolean isCompressed() {
+ return compress;
+ }
+
+ public class ScheduledPing extends AbstractLifecycleRunnable {
+
+ /**
+ * The magic number (must be lower than 0) for a ping message. This is handled
+ * specifically in {@link TcpTransport#validateMessageHeader}.
+ */
+ private final BytesReference pingHeader;
+ final CounterMetric successfulPings = new CounterMetric();
+ final CounterMetric failedPings = new CounterMetric();
+
+ public ScheduledPing() {
+ super(lifecycle, logger);
+ try (BytesStreamOutput out = new BytesStreamOutput()) {
+ out.writeByte((byte) 'E');
+ out.writeByte((byte) 'S');
+ out.writeInt(PING_DATA_SIZE);
+ pingHeader = out.bytes();
+ } catch (IOException e) {
+ throw new IllegalStateException(e.getMessage(), e); // won't happen
+ }
+ }
+
+ @Override
+ protected void doRunInLifecycle() throws Exception {
+ for (Map.Entry entry : connectedNodes.entrySet()) {
+ DiscoveryNode node = entry.getKey();
+ NodeChannels channels = entry.getValue();
+ for (Channel channel : channels.allChannels) {
+ try {
+ sendMessage(channel, pingHeader, successfulPings::inc, false);
+ } catch (Throwable t) {
+ if (isOpen(channel)) {
+ logger.debug("[{}] failed to send ping transport message", t, node);
+ failedPings.inc();
+ } else {
+ logger.trace("[{}] failed to send ping transport message (channel closed)", t, node);
+ }
+ }
+ }
+ }
+ }
+
+ public long getSuccessfulPings() {
+ return successfulPings.count();
+ }
+
+ public long getFailedPings() {
+ return failedPings.count();
+ }
+
+ @Override
+ protected void onAfterInLifecycle() {
+ threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, this);
+ }
+
+ @Override
+ public void onFailure(Throwable t) {
+ if (lifecycle.stoppedOrClosed()) {
+ logger.trace("failed to send ping transport message", t);
+ } else {
+ logger.warn("failed to send ping transport message", t);
+ }
+ }
+ }
+
+ public class NodeChannels implements Closeable {
+
+ public List allChannels = Collections.emptyList();
+ public Channel[] recovery;
+ public final AtomicInteger recoveryCounter = new AtomicInteger();
+ public Channel[] bulk;
+ public final AtomicInteger bulkCounter = new AtomicInteger();
+ public Channel[] reg;
+ public final AtomicInteger regCounter = new AtomicInteger();
+ public Channel[] state;
+ public final AtomicInteger stateCounter = new AtomicInteger();
+ public Channel[] ping;
+ public final AtomicInteger pingCounter = new AtomicInteger();
+
+ public NodeChannels(Channel[] recovery, Channel[] bulk, Channel[] reg, Channel[] state, Channel[] ping) {
+ this.recovery = recovery;
+ this.bulk = bulk;
+ this.reg = reg;
+ this.state = state;
+ this.ping = ping;
+ }
+
+ public void start() {
+ List newAllChannels = new ArrayList<>();
+ newAllChannels.addAll(Arrays.asList(recovery));
+ newAllChannels.addAll(Arrays.asList(bulk));
+ newAllChannels.addAll(Arrays.asList(reg));
+ newAllChannels.addAll(Arrays.asList(state));
+ newAllChannels.addAll(Arrays.asList(ping));
+ this.allChannels = Collections.unmodifiableList(newAllChannels);
+ }
+
+ public boolean hasChannel(Channel channel) {
+ for (Channel channel1 : allChannels) {
+ if (channel.equals(channel1)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public Channel channel(TransportRequestOptions.Type type) {
+ if (type == TransportRequestOptions.Type.REG) {
+ return reg[Math.floorMod(regCounter.incrementAndGet(), reg.length)];
+ } else if (type == TransportRequestOptions.Type.STATE) {
+ return state[Math.floorMod(stateCounter.incrementAndGet(), state.length)];
+ } else if (type == TransportRequestOptions.Type.PING) {
+ return ping[Math.floorMod(pingCounter.incrementAndGet(), ping.length)];
+ } else if (type == TransportRequestOptions.Type.BULK) {
+ return bulk[Math.floorMod(bulkCounter.incrementAndGet(), bulk.length)];
+ } else if (type == TransportRequestOptions.Type.RECOVERY) {
+ return recovery[Math.floorMod(recoveryCounter.incrementAndGet(), recovery.length)];
+ } else {
+ throw new IllegalArgumentException("no type channel for [" + type + "]");
+ }
+ }
+
+ public synchronized void close() {
+ closeChannels(allChannels);
+ }
+ }
+
+ @Override
+ public boolean nodeConnected(DiscoveryNode node) {
+ return connectedNodes.containsKey(node);
+ }
+
+ @Override
+ public void connectToNodeLight(DiscoveryNode node) throws ConnectTransportException {
+ connectToNode(node, true);
+ }
+
+ @Override
+ public void connectToNode(DiscoveryNode node) {
+ connectToNode(node, false);
+ }
+
+ public void connectToNode(DiscoveryNode node, boolean light) {
+ if (!lifecycle.started()) {
+ throw new IllegalStateException("can't add nodes to a stopped transport");
+ }
+ if (node == null) {
+ throw new ConnectTransportException(null, "can't connect to a null node");
+ }
+ globalLock.readLock().lock();
+ try {
+
+ try (Releasable ignored = connectionLock.acquire(node.getId())) {
+ if (!lifecycle.started()) {
+ throw new IllegalStateException("can't add nodes to a stopped transport");
+ }
+ NodeChannels nodeChannels = connectedNodes.get(node);
+ if (nodeChannels != null) {
+ return;
+ }
+ try {
+ if (light) {
+ nodeChannels = connectToChannelsLight(node);
+ } else {
+ try {
+ nodeChannels = connectToChannels(node);
+ } catch (Throwable e) {
+ logger.trace("failed to connect to [{}], cleaning dangling connections", e, node);
+ throw e;
+ }
+ }
+ // we acquire a connection lock, so no way there is an existing connection
+ nodeChannels.start();
+ connectedNodes.put(node, nodeChannels);
+ if (logger.isDebugEnabled()) {
+ logger.debug("connected to node [{}]", node);
+ }
+ transportServiceAdapter.raiseNodeConnected(node);
+ } catch (ConnectTransportException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new ConnectTransportException(node, "general node connection failure", e);
+ }
+ }
+ } finally {
+ globalLock.readLock().unlock();
+ }
+ }
+ /**
+ * Disconnects from a node, only if the relevant channel is found to be part of the node channels.
+ */
+ protected boolean disconnectFromNode(DiscoveryNode node, Channel channel, String reason) {
+ // this might be called multiple times from all the node channels, so do a lightweight
+ // check outside of the lock
+ NodeChannels nodeChannels = connectedNodes.get(node);
+ if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
+ try (Releasable ignored = connectionLock.acquire(node.getId())) {
+ nodeChannels = connectedNodes.get(node);
+ // check again within the connection lock, if its still applicable to remove it
+ if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
+ connectedNodes.remove(node);
+ try {
+ logger.debug("disconnecting from [{}], {}", node, reason);
+ nodeChannels.close();
+ } finally {
+ logger.trace("disconnected from [{}], {}", node, reason);
+ transportServiceAdapter.raiseNodeDisconnected(node);
+ }
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Disconnects from a node if a channel is found as part of that nodes channels.
+ */
+ protected final void disconnectFromNodeChannel(final Channel channel, final Throwable failure) {
+ threadPool.generic().execute(() -> {
+ try {
+ closeChannels(Collections.singletonList(channel));
+ } finally {
+ for (DiscoveryNode node : connectedNodes.keySet()) {
+ if (disconnectFromNode(node, channel, ExceptionsHelper.detailedMessage(failure))) {
+ // if we managed to find this channel and disconnect from it, then break, no need to check on
+ // the rest of the nodes
+ break;
+ }
+ }
+ }
+ });
+ }
+
+ protected Channel nodeChannel(DiscoveryNode node, TransportRequestOptions options) throws ConnectTransportException {
+ NodeChannels nodeChannels = connectedNodes.get(node);
+ if (nodeChannels == null) {
+ throw new NodeNotConnectedException(node, "Node not connected");
+ }
+ return nodeChannels.channel(options.type());
+ }
+
+ @Override
+ public void disconnectFromNode(DiscoveryNode node) {
+ try (Releasable ignored = connectionLock.acquire(node.getId())) {
+ NodeChannels nodeChannels = connectedNodes.remove(node);
+ if (nodeChannels != null) {
+ try {
+ logger.debug("disconnecting from [{}] due to explicit disconnect call", node);
+ nodeChannels.close();
+ } finally {
+ logger.trace("disconnected from [{}] due to explicit disconnect call", node);
+ transportServiceAdapter.raiseNodeDisconnected(node);
+ }
+ }
+ }
+ }
+
+ protected Version getCurrentVersion() {
+ // this is just for tests to mock stuff like the nodes version - tests can override this internally
+ return Version.CURRENT;
+ }
+
+ @Override
+ public boolean addressSupported(Class extends TransportAddress> address) {
+ return InetSocketTransportAddress.class.equals(address);
+ }
+
+ @Override
+ public BoundTransportAddress boundAddress() {
+ return this.boundAddress;
+ }
+
+ @Override
+ public Map profileBoundAddresses() {
+ return unmodifiableMap(new HashMap<>(profileBoundAddresses));
+ }
+
+ protected Map buildProfileSettings() {
+ // extract default profile first and create standard bootstrap
+ Map profiles = TransportSettings.TRANSPORT_PROFILES_SETTING.get(settings()).getAsGroups(true);
+ if (!profiles.containsKey(TransportSettings.DEFAULT_PROFILE)) {
+ profiles = new HashMap<>(profiles);
+ profiles.put(TransportSettings.DEFAULT_PROFILE, Settings.EMPTY);
+ }
+ Settings defaultSettings = profiles.get(TransportSettings.DEFAULT_PROFILE);
+ Map result = new HashMap<>();
+ // loop through all profiles and start them up, special handling for default one
+ for (Map.Entry entry : profiles.entrySet()) {
+ Settings profileSettings = entry.getValue();
+ String name = entry.getKey();
+
+ if (!Strings.hasLength(name)) {
+ logger.info("transport profile configured without a name. skipping profile with settings [{}]",
+ profileSettings.toDelimitedString(','));
+ continue;
+ } else if (TransportSettings.DEFAULT_PROFILE.equals(name)) {
+ profileSettings = Settings.builder()
+ .put(profileSettings)
+ .put("port", profileSettings.get("port", TransportSettings.PORT.get(this.settings)))
+ .build();
+ } else if (profileSettings.get("port") == null) {
+ // if profile does not have a port, skip it
+ logger.info("No port configured for profile [{}], not binding", name);
+ continue;
+ }
+ Settings mergedSettings = Settings.builder()
+ .put(defaultSettings)
+ .put(profileSettings)
+ .build();
+ result.put(name, mergedSettings);
+ }
+ return result;
+ }
+
+ @Override
+ public List getLocalAddresses() {
+ List local = new ArrayList<>();
+ local.add("127.0.0.1");
+ // check if v6 is supported, if so, v4 will also work via mapped addresses.
+ if (NetworkUtils.SUPPORTS_V6) {
+ local.add("[::1]"); // may get ports appended!
+ }
+ return local;
+ }
+
+ protected void bindServer(final String name, final Settings settings) {
+ // Bind and start to accept incoming connections.
+ InetAddress hostAddresses[];
+ String bindHosts[] = settings.getAsArray("bind_host", null);
+ try {
+ hostAddresses = networkService.resolveBindHostAddresses(bindHosts);
+ } catch (IOException e) {
+ throw new BindTransportException("Failed to resolve host " + Arrays.toString(bindHosts) + "", e);
+ }
+ if (logger.isDebugEnabled()) {
+ String[] addresses = new String[hostAddresses.length];
+ for (int i = 0; i < hostAddresses.length; i++) {
+ addresses[i] = NetworkAddress.format(hostAddresses[i]);
+ }
+ logger.debug("binding server bootstrap to: {}", (Object)addresses);
+ }
+
+ assert hostAddresses.length > 0;
+
+ List boundAddresses = new ArrayList<>();
+ for (InetAddress hostAddress : hostAddresses) {
+ boundAddresses.add(bindToPort(name, hostAddress, settings.get("port")));
+ }
+
+ final BoundTransportAddress boundTransportAddress = createBoundTransportAddress(name, settings, boundAddresses);
+
+ if (TransportSettings.DEFAULT_PROFILE.equals(name)) {
+ this.boundAddress = boundTransportAddress;
+ } else {
+ profileBoundAddresses.put(name, boundTransportAddress);
+ }
+ }
+
+ protected InetSocketAddress bindToPort(final String name, final InetAddress hostAddress, String port) {
+ PortsRange portsRange = new PortsRange(port);
+ final AtomicReference lastException = new AtomicReference<>();
+ final AtomicReference boundSocket = new AtomicReference<>();
+ boolean success = portsRange.iterate(portNumber -> {
+ try {
+ Channel channel = bind(name, new InetSocketAddress(hostAddress, portNumber));
+ synchronized (serverChannels) {
+ List list = serverChannels.get(name);
+ if (list == null) {
+ list = new ArrayList<>();
+ serverChannels.put(name, list);
+ }
+ list.add(channel);
+ boundSocket.set(getLocalAddress(channel));
+ }
+ } catch (Exception e) {
+ lastException.set(e);
+ return false;
+ }
+ return true;
+ });
+ if (!success) {
+ throw new BindTransportException("Failed to bind to [" + port + "]", lastException.get());
+ }
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("Bound profile [{}] to address {{}}", name, NetworkAddress.format(boundSocket.get()));
+ }
+
+ return boundSocket.get();
+ }
+
+ private BoundTransportAddress createBoundTransportAddress(String name, Settings profileSettings,
+ List boundAddresses) {
+ String[] boundAddressesHostStrings = new String[boundAddresses.size()];
+ TransportAddress[] transportBoundAddresses = new TransportAddress[boundAddresses.size()];
+ for (int i = 0; i < boundAddresses.size(); i++) {
+ InetSocketAddress boundAddress = boundAddresses.get(i);
+ boundAddressesHostStrings[i] = boundAddress.getHostString();
+ transportBoundAddresses[i] = new InetSocketTransportAddress(boundAddress);
+ }
+
+ final String[] publishHosts;
+ if (TransportSettings.DEFAULT_PROFILE.equals(name)) {
+ publishHosts = TransportSettings.PUBLISH_HOST.get(settings).toArray(Strings.EMPTY_ARRAY);
+ } else {
+ publishHosts = profileSettings.getAsArray("publish_host", boundAddressesHostStrings);
+ }
+
+ final InetAddress publishInetAddress;
+ try {
+ publishInetAddress = networkService.resolvePublishHostAddresses(publishHosts);
+ } catch (Exception e) {
+ throw new BindTransportException("Failed to resolve publish address", e);
+ }
+
+ final int publishPort = resolvePublishPort(name, settings, profileSettings, boundAddresses, publishInetAddress);
+ final TransportAddress publishAddress = new InetSocketTransportAddress(new InetSocketAddress(publishInetAddress, publishPort));
+ return new BoundTransportAddress(transportBoundAddresses, publishAddress);
+ }
+
+ // package private for tests
+ public static int resolvePublishPort(String profileName, Settings settings, Settings profileSettings,
+ List boundAddresses, InetAddress publishInetAddress) {
+ int publishPort;
+ if (TransportSettings.DEFAULT_PROFILE.equals(profileName)) {
+ publishPort = TransportSettings.PUBLISH_PORT.get(settings);
+ } else {
+ publishPort = profileSettings.getAsInt("publish_port", -1);
+ }
+
+ // if port not explicitly provided, search for port of address in boundAddresses that matches publishInetAddress
+ if (publishPort < 0) {
+ for (InetSocketAddress boundAddress : boundAddresses) {
+ InetAddress boundInetAddress = boundAddress.getAddress();
+ if (boundInetAddress.isAnyLocalAddress() || boundInetAddress.equals(publishInetAddress)) {
+ publishPort = boundAddress.getPort();
+ break;
+ }
+ }
+ }
+
+ // if no matching boundAddress found, check if there is a unique port for all bound addresses
+ if (publishPort < 0) {
+ final IntSet ports = new IntHashSet();
+ for (InetSocketAddress boundAddress : boundAddresses) {
+ ports.add(boundAddress.getPort());
+ }
+ if (ports.size() == 1) {
+ publishPort = ports.iterator().next().value;
+ }
+ }
+
+ if (publishPort < 0) {
+ String profileExplanation = TransportSettings.DEFAULT_PROFILE.equals(profileName) ? "" : " for profile " + profileName;
+ throw new BindTransportException("Failed to auto-resolve publish port" + profileExplanation + ", multiple bound addresses " +
+ boundAddresses + " with distinct ports and none of them matched the publish address (" + publishInetAddress + "). " +
+ "Please specify a unique port by setting " + TransportSettings.PORT.getKey() + " or " +
+ TransportSettings.PUBLISH_PORT.getKey());
+ }
+ return publishPort;
+ }
+
+ @Override
+ public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws Exception {
+ return parse(address, settings.get("transport.profiles.default.port", TransportSettings.PORT.get(settings)), perAddressLimit);
+ }
+
+ // this code is a take on guava's HostAndPort, like a HostAndPortRange
+
+ // pattern for validating ipv6 bracket addresses.
+ // not perfect, but PortsRange should take care of any port range validation, not a regex
+ private static final Pattern BRACKET_PATTERN = Pattern.compile("^\\[(.*:.*)\\](?::([\\d\\-]*))?$");
+
+ /** parse a hostname+port range spec into its equivalent addresses */
+ static TransportAddress[] parse(String hostPortString, String defaultPortRange, int perAddressLimit) throws UnknownHostException {
+ Objects.requireNonNull(hostPortString);
+ String host;
+ String portString = null;
+
+ if (hostPortString.startsWith("[")) {
+ // Parse a bracketed host, typically an IPv6 literal.
+ Matcher matcher = BRACKET_PATTERN.matcher(hostPortString);
+ if (!matcher.matches()) {
+ throw new IllegalArgumentException("Invalid bracketed host/port range: " + hostPortString);
+ }
+ host = matcher.group(1);
+ portString = matcher.group(2); // could be null
+ } else {
+ int colonPos = hostPortString.indexOf(':');
+ if (colonPos >= 0 && hostPortString.indexOf(':', colonPos + 1) == -1) {
+ // Exactly 1 colon. Split into host:port.
+ host = hostPortString.substring(0, colonPos);
+ portString = hostPortString.substring(colonPos + 1);
+ } else {
+ // 0 or 2+ colons. Bare hostname or IPv6 literal.
+ host = hostPortString;
+ // 2+ colons and not bracketed: exception
+ if (colonPos >= 0) {
+ throw new IllegalArgumentException("IPv6 addresses must be bracketed: " + hostPortString);
+ }
+ }
+ }
+
+ // if port isn't specified, fill with the default
+ if (portString == null || portString.isEmpty()) {
+ portString = defaultPortRange;
+ }
+
+ // generate address for each port in the range
+ Set addresses = new HashSet<>(Arrays.asList(InetAddress.getAllByName(host)));
+ List transportAddresses = new ArrayList<>();
+ int[] ports = new PortsRange(portString).ports();
+ int limit = Math.min(ports.length, perAddressLimit);
+ for (int i = 0; i < limit; i++) {
+ for (InetAddress address : addresses) {
+ transportAddresses.add(new InetSocketTransportAddress(address, ports[i]));
+ }
+ }
+ return transportAddresses.toArray(new TransportAddress[transportAddresses.size()]);
+ }
+
+ @Override
+ protected final void doClose() {
+ }
+
+ @Override
+ protected final void doStop() {
+ final CountDownLatch latch = new CountDownLatch(1);
+ // make sure we run it on another thread than a possible IO handler thread
+ threadPool.generic().execute(() -> {
+ globalLock.writeLock().lock();
+ try {
+ for (Iterator it = connectedNodes.values().iterator(); it.hasNext(); ) {
+ NodeChannels nodeChannels = it.next();
+ it.remove();
+ nodeChannels.close();
+ }
+
+ for (Map.Entry> entry : serverChannels.entrySet()) {
+ try {
+ closeChannels(entry.getValue());
+ } catch (Throwable t) {
+ logger.debug("Error closing serverChannel for profile [{}]", t, entry.getKey());
+ }
+ }
+ try {
+ stopInternal();
+ } finally {
+ for (Iterator it = connectedNodes.values().iterator(); it.hasNext(); ) {
+ NodeChannels nodeChannels = it.next();
+ it.remove();
+ nodeChannels.close();
+ }
+ }
+
+ } finally {
+ globalLock.writeLock().unlock();
+ latch.countDown();
+ }
+ });
+
+ try {
+ latch.await(30, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ Thread.interrupted();
+ // ignore
+ }
+ }
+
+ protected void onException(Channel channel, Throwable e) {
+ if (!lifecycle.started()) {
+ // ignore
+ return;
+ }
+ if (isCloseConnectionException(e)) {
+ logger.trace("close connection exception caught on transport layer [{}], disconnecting from relevant node", e,
+ channel);
+ // close the channel, which will cause a node to be disconnected if relevant
+ disconnectFromNodeChannel(channel, e);
+ } else if (isConnectException(e)) {
+ logger.trace("connect exception caught on transport layer [{}]", e, channel);
+ // close the channel as safe measure, which will cause a node to be disconnected if relevant
+ disconnectFromNodeChannel(channel, e);
+ } else if (e instanceof BindException) {
+ logger.trace("bind exception caught on transport layer [{}]", e, channel);
+ // close the channel as safe measure, which will cause a node to be disconnected if relevant
+ disconnectFromNodeChannel(channel, e);
+ } else if (e instanceof CancelledKeyException) {
+ logger.trace("cancelled key exception caught on transport layer [{}], disconnecting from relevant node", e,
+ channel);
+ // close the channel as safe measure, which will cause a node to be disconnected if relevant
+ disconnectFromNodeChannel(channel, e);
+ } else if (e instanceof TcpTransport.HttpOnTransportException) {
+ // in case we are able to return data, serialize the exception content and sent it back to the client
+ if (isOpen(channel)) {
+ sendMessage(channel, new BytesArray(e.getMessage().getBytes(StandardCharsets.UTF_8)), () -> {}, true);
+ }
+ } else {
+ logger.warn("exception caught on transport layer [{}], closing connection", e, channel);
+ // close the channel, which will cause a node to be disconnected if relevant
+ disconnectFromNodeChannel(channel, e);
+ }
+ }
+
+ /**
+ * Returns the channels local address
+ */
+ protected abstract InetSocketAddress getLocalAddress(Channel channel);
+
+ /**
+ * Binds to the given {@link InetSocketAddress}
+ * @param name the profile name
+ * @param address the address to bind to
+ */
+ protected abstract Channel bind(String name, InetSocketAddress address);
+
+ /**
+ * Closes all channels in this list
+ */
+ protected abstract void closeChannels(List channel);
+
+ /**
+ * Connects to the given node in a light way. This means we are not creating multiple connections like we do
+ * for production connections. This connection is for pings or handshakes
+ */
+ protected abstract NodeChannels connectToChannelsLight(DiscoveryNode node);
+
+
+ protected abstract void sendMessage(Channel channel, BytesReference reference, Runnable sendListener, boolean close);
+
+ /**
+ * Connects to the node in a heavy way.
+ *
+ * @see #connectToChannelsLight(DiscoveryNode)
+ */
+ protected abstract NodeChannels connectToChannels(DiscoveryNode node);
+
+ /**
+ * Called to tear down internal resources
+ */
+ protected void stopInternal() {}
+
+ public boolean canCompress(TransportRequest request) {
+ return compress;
+ }
+
+ @Override
+ public void sendRequest(final DiscoveryNode node, final long requestId, final String action, final TransportRequest request,
+ TransportRequestOptions options) throws IOException, TransportException {
+
+ Channel targetChannel = nodeChannel(node, options);
+
+ if (compress) {
+ options = TransportRequestOptions.builder(options).withCompress(true).build();
+ }
+ byte status = 0;
+ status = TransportStatus.setRequest(status);
+ ReleasableBytesStreamOutput bStream = new ReleasableBytesStreamOutput(bigArrays);
+ boolean addedReleaseListener = false;
+ try {
+ bStream.skip(TcpHeader.HEADER_SIZE);
+ StreamOutput stream = bStream;
+ // only compress if asked, and, the request is not bytes, since then only
+ // the header part is compressed, and the "body" can't be extracted as compressed
+ if (options.compress() && canCompress(request)) {
+ status = TransportStatus.setCompress(status);
+ stream = CompressorFactory.COMPRESSOR.streamOutput(stream);
+ }
+
+ // 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.smallest(getCurrentVersion(), node.getVersion());
+
+ stream.setVersion(version);
+ threadPool.getThreadContext().writeTo(stream);
+ stream.writeString(action);
+
+ Message writeable = prepareSend(node.getVersion(), request, stream, bStream);
+ try (StreamOutput headerOutput = writeable.getHeaderOutput()) {
+ TcpHeader.writeHeader(headerOutput, requestId, status, version,
+ writeable.size());
+ }
+ final TransportRequestOptions finalOptions = options;
+ Runnable onRequestSent = () -> {
+ try {
+ Releasables.close(bStream.bytes());
+ } finally {
+ transportServiceAdapter.onRequestSent(node, requestId, action, request, finalOptions);
+ }
+ };
+ writeable.send(targetChannel, onRequestSent);
+ addedReleaseListener = true;
+
+ } finally {
+ if (!addedReleaseListener) {
+ Releasables.close(bStream.bytes());
+ }
+ }
+ }
+
+ /**
+ * Sends back an error response to the caller via the given channel
+ * @param nodeVersion the caller node version
+ * @param channel the channel to send the response to
+ * @param error the error to return
+ * @param requestId the request ID this response replies to
+ * @param action the action this response replies to
+ */
+ public void sendErrorResponse(Version nodeVersion, Channel channel, final Throwable error, final long requestId,
+ final String action) throws IOException {
+ BytesStreamOutput stream = new BytesStreamOutput();
+ stream.setVersion(nodeVersion);
+ stream.skip(TcpHeader.HEADER_SIZE);
+ RemoteTransportException tx = new RemoteTransportException(
+ nodeName(), new InetSocketTransportAddress(getLocalAddress(channel)), action, error);
+ stream.writeThrowable(tx);
+ byte status = 0;
+ status = TransportStatus.setResponse(status);
+ status = TransportStatus.setError(status);
+
+ final BytesReference bytes = stream.bytes();
+ Message writeable = prepareSend(nodeVersion, bytes);
+ try (StreamOutput headerOutput = writeable.getHeaderOutput()) {
+ TcpHeader.writeHeader(headerOutput, requestId, status, nodeVersion,
+ writeable.size());
+ }
+ Runnable onRequestSent = () -> {
+ transportServiceAdapter.onResponseSent(requestId, action, error);
+ };
+ writeable.send(channel, onRequestSent);
+ }
+
+ /**
+ * Sends the response to the given channel. This method should be used to send {@link TransportResponse} objects back to the caller.
+ *
+ * @see #sendErrorResponse(Version, Object, Throwable, long, String) for sending back errors to the caller
+ */
+ public void sendResponse(Version nodeVersion, Channel channel, final TransportResponse response, final long requestId,
+ final String action, TransportResponseOptions options) throws IOException {
+ if (compress) {
+ options = TransportResponseOptions.builder(options).withCompress(true).build();
+ }
+
+ byte status = 0;
+ status = TransportStatus.setResponse(status); // TODO share some code with sendRequest
+ ReleasableBytesStreamOutput bStream = new ReleasableBytesStreamOutput(bigArrays);
+ boolean addedReleaseListener = false;
+ try {
+ bStream.skip(TcpHeader.HEADER_SIZE);
+ StreamOutput stream = bStream;
+ if (options.compress()) {
+ status = TransportStatus.setCompress(status);
+ stream = CompressorFactory.COMPRESSOR.streamOutput(stream);
+ }
+ stream.setVersion(nodeVersion);
+ Message writeable = prepareSend(nodeVersion, response, stream, bStream);
+ try (StreamOutput headerOutput = writeable.getHeaderOutput()) {
+ TcpHeader.writeHeader(headerOutput, requestId, status, nodeVersion,
+ writeable.size());
+ }
+ final TransportResponseOptions finalOptions = options;
+ Runnable onRequestSent = () -> {
+ try {
+ Releasables.close(bStream.bytes());
+ } finally {
+ transportServiceAdapter.onResponseSent(requestId, action, response, finalOptions);
+ }
+ };
+ writeable.send(channel, onRequestSent);
+ addedReleaseListener = true;
+
+ } finally {
+ if (!addedReleaseListener) {
+ Releasables.close(bStream.bytes());
+ }
+ }
+ }
+
+ /**
+ * Serializes the given message into a bytes representation and forwards to {@link #prepareSend(Version, TransportMessage,
+ * StreamOutput, ReleasableBytesStream)}
+ */
+ protected Message prepareSend(Version nodeVersion, TransportMessage message, StreamOutput stream,
+ ReleasableBytesStream writtenBytes) throws IOException {
+ message.writeTo(stream);
+ stream.close();
+ return prepareSend(nodeVersion, writtenBytes.bytes());
+ }
+
+ /**
+ * prepares a implementation specific message to send across the network
+ */
+ protected abstract Message prepareSend(Version nodeVersion, BytesReference bytesReference) throws IOException;
+
+ /**
+ * Allows implementations to transform TransportMessages into implementation specific messages
+ */
+ protected interface Message {
+ /**
+ * Creates an output to write the message header to.
+ */
+ StreamOutput getHeaderOutput();
+
+ /**
+ * Returns the size of the message in bytes
+ */
+ int size();
+
+ /**
+ * sends the message to the channel
+ * @param channel the channe to send the message to
+ * @param onRequestSent a callback executed once the message has been fully send
+ */
+ void send(Channel channel, Runnable onRequestSent);
+ }
+
+
+ /**
+ * Validates the first N bytes of the message header and returns true
if the message is
+ * a ping message and has no payload ie. isn't a real user level message.
+ *
+ * @throws IllegalStateException if the message is too short, less than the header or less that the header plus the message size
+ * @throws HttpOnTransportException if the message has no valid header and appears to be a HTTP message
+ * @throws IllegalArgumentException if the message is greater that the maximum allowed frame size. This is dependent on the available
+ * memory.
+ */
+ public static boolean validateMessageHeader(BytesReference buffer) throws IOException {
+ final int sizeHeaderLength = TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE;
+ if (buffer.length() < sizeHeaderLength) {
+ throw new IllegalStateException("message size must be >= to the header size");
+ }
+ int offset = 0;
+ if (buffer.get(offset) != 'E' || buffer.get(offset + 1) != 'S') {
+ // special handling for what is probably HTTP
+ if (bufferStartsWith(buffer, offset, "GET ") ||
+ bufferStartsWith(buffer, offset, "POST ") ||
+ bufferStartsWith(buffer, offset, "PUT ") ||
+ bufferStartsWith(buffer, offset, "HEAD ") ||
+ bufferStartsWith(buffer, offset, "DELETE ") ||
+ bufferStartsWith(buffer, offset, "OPTIONS ") ||
+ bufferStartsWith(buffer, offset, "PATCH ") ||
+ bufferStartsWith(buffer, offset, "TRACE ")) {
+
+ throw new HttpOnTransportException("This is not a HTTP port");
+ }
+
+ // we have 6 readable bytes, show 4 (should be enough)
+ throw new StreamCorruptedException("invalid internal transport message format, got ("
+ + Integer.toHexString(buffer.get(offset) & 0xFF) + ","
+ + Integer.toHexString(buffer.get(offset + 1) & 0xFF) + ","
+ + Integer.toHexString(buffer.get(offset + 2) & 0xFF) + ","
+ + Integer.toHexString(buffer.get(offset + 3) & 0xFF) + ")");
+ }
+
+ final int dataLen;
+ try (StreamInput input = buffer.streamInput()) {
+ input.skip(TcpHeader.MARKER_BYTES_SIZE);
+ dataLen = input.readInt();
+ if (dataLen == PING_DATA_SIZE) {
+ // discard the messages we read and continue, this is achieved by skipping the bytes
+ // and returning null
+ return false;
+ }
+ }
+ if (dataLen <= 0) {
+ throw new StreamCorruptedException("invalid data length: " + dataLen);
+ }
+ // safety against too large frames being sent
+ if (dataLen > NINETY_PER_HEAP_SIZE) {
+ throw new IllegalArgumentException("transport content length received [" + new ByteSizeValue(dataLen) + "] exceeded ["
+ + new ByteSizeValue(NINETY_PER_HEAP_SIZE) + "]");
+ }
+
+ if (buffer.length() < dataLen + sizeHeaderLength) {
+ throw new IllegalStateException("buffer must be >= to the message size but wasn't");
+ }
+ return true;
+ }
+
+ private static boolean bufferStartsWith(BytesReference buffer, int offset, String method) {
+ char[] chars = method.toCharArray();
+ for (int i = 0; i < chars.length; i++) {
+ if (buffer.get(offset+ i) != chars[i]) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ /**
+ * A helper exception to mark an incoming connection as potentially being HTTP
+ * so an appropriate error code can be returned
+ */
+ public static class HttpOnTransportException extends ElasticsearchException {
+
+ public HttpOnTransportException(String msg) {
+ super(msg);
+ }
+
+ @Override
+ public RestStatus status() {
+ return RestStatus.BAD_REQUEST;
+ }
+
+ public HttpOnTransportException(StreamInput in) throws IOException{
+ super(in);
+ }
+ }
+
+ protected abstract boolean isOpen(Channel channel);
+
+ /**
+ * This method handles the message receive part for both request and responses
+ */
+ public final void messageReceived(BytesReference reference, Channel channel, String profileName,
+ InetSocketAddress remoteAddress, int messageLengthBytes) throws IOException {
+ final int totalMessageSize = messageLengthBytes + TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE;
+ transportServiceAdapter.received(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);
+ }
+ if (version.onOrAfter(Version.CURRENT.minimumCompatibilityVersion()) == false || version.major != Version.CURRENT.major) {
+ throw new IllegalStateException("Received message from unsupported version: [" + version
+ + "] minimal compatible version is: [" +Version.CURRENT.minimumCompatibilityVersion() + "]");
+ }
+ streamIn = new NamedWriteableAwareStreamInput(streamIn, namedWriteableRegistry);
+ streamIn.setVersion(version);
+ if (TransportStatus.isRequest(status)) {
+ threadPool.getThreadContext().readHeaders(streamIn);
+ handleRequest(channel, profileName, streamIn, requestId, messageLengthBytes, version, remoteAddress);
+ } else {
+ final TransportResponseHandler> handler = transportServiceAdapter.onResponseReceived(requestId);
+ // ignore if its null, the adapter logs it
+ if (handler != null) {
+ if (TransportStatus.isError(status)) {
+ handlerResponseError(streamIn, handler);
+ } else {
+ handleResponse(remoteAddress, streamIn, handler);
+ }
+ // Check the entire message has been read
+ final int nextByte = streamIn.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");
+ }
+ }
+ }
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(streamIn);
+ } else {
+ IOUtils.closeWhileHandlingException(streamIn);
+ }
+ }
+ }
+
+ private void handleResponse(InetSocketAddress remoteAddress, final StreamInput stream, final TransportResponseHandler handler) {
+ final TransportResponse response = handler.newInstance();
+ response.remoteAddress(new InetSocketTransportAddress(remoteAddress));
+ try {
+ response.readFrom(stream);
+ } catch (Throwable e) {
+ handleException(handler, new TransportSerializationException(
+ "Failed to deserialize response of type [" + response.getClass().getName() + "]", e));
+ return;
+ }
+ threadPool.executor(handler.executor()).execute(new AbstractRunnable() {
+ @Override
+ public void onFailure(Throwable t) {
+ handleException(handler, new ResponseHandlerFailureTransportException(t));
+ }
+
+ @Override
+ protected void doRun() throws Exception {
+ handler.handleResponse(response);
+ }});
+
+ }
+
+ /**
+ * Executed for a received response error
+ */
+ private void handlerResponseError(StreamInput stream, final TransportResponseHandler handler) {
+ Throwable error;
+ try {
+ error = stream.readThrowable();
+ } catch (Throwable 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 (Throwable e) {
+ logger.error("failed to handle exception response [{}]", e, handler);
+ }
+ });
+ }
+
+ protected String handleRequest(Channel channel, String profileName, final StreamInput stream, long requestId,
+ int messageLengthBytes, Version version, InetSocketAddress remoteAddress) throws IOException {
+ final String action = stream.readString();
+ transportServiceAdapter.onRequestReceived(requestId, action);
+ TransportChannel transportChannel = null;
+ try {
+ final RequestHandlerRegistry reg = transportServiceAdapter.getRequestHandler(action);
+ if (reg == null) {
+ throw new ActionNotFoundTransportException(action);
+ }
+ if (reg.canTripCircuitBreaker()) {
+ getInFlightRequestBreaker().addEstimateBytesAndMaybeBreak(messageLengthBytes, "");
+ } else {
+ getInFlightRequestBreaker().addWithoutBreaking(messageLengthBytes);
+ }
+ transportChannel = new TcpTransportChannel<>(this, channel, transportName, action, requestId, version, profileName,
+ messageLengthBytes);
+ final TransportRequest request = reg.newRequest();
+ request.remoteAddress(new InetSocketTransportAddress(remoteAddress));
+ request.readFrom(stream);
+ // 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 (Throwable e) {
+ // the circuit breaker tripped
+ if (transportChannel == null) {
+ transportChannel = new TcpTransportChannel<>(this, channel, transportName, action, requestId, version, profileName, 0);
+ }
+ try {
+ transportChannel.sendResponse(e);
+ } catch (IOException e1) {
+ logger.warn("Failed to send error message back to client for action [{}]", e, action);
+ logger.warn("Actual Exception", e1);
+ }
+ }
+ return action;
+ }
+
+ // 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;
+
+ public 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(Throwable e) {
+ if (lifecycleState() == Lifecycle.State.STARTED) {
+ // we can only send a response transport is started....
+ try {
+ transportChannel.sendResponse(e);
+ } catch (Throwable e1) {
+ logger.warn("Failed to send error message back to client for action [{}]", e1, reg.getAction());
+ logger.warn("Actual Exception", e);
+ }
+ }
+ }
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java b/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java
new file mode 100644
index 00000000000..74bdad48746
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/transport/TcpTransportChannel.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.transport;
+
+import org.elasticsearch.Version;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ *
+ */
+public final class TcpTransportChannel implements TransportChannel {
+ private final TcpTransport transport;
+ protected final Version version;
+ protected final String action;
+ protected final long requestId;
+ private final String profileName;
+ private final long reservedBytes;
+ private final AtomicBoolean released = new AtomicBoolean();
+ private final String channelType;
+ private final Channel channel;
+
+ public TcpTransportChannel(TcpTransport transport, Channel channel, String channelType, String action,
+ long requestId, Version version, String profileName, long reservedBytes) {
+ this.version = version;
+ this.channel = channel;
+ this.transport = transport;
+ this.action = action;
+ this.requestId = requestId;
+ this.profileName = profileName;
+ this.reservedBytes = reservedBytes;
+ this.channelType = channelType;
+ }
+
+ @Override
+ public final String getProfileName() {
+ return profileName;
+ }
+
+ @Override
+ public final String action() {
+ return this.action;
+ }
+
+ @Override
+ public final void sendResponse(TransportResponse response) throws IOException {
+ sendResponse(response, TransportResponseOptions.EMPTY);
+ }
+
+ @Override
+ public final void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException {
+ release();
+ transport.sendResponse(version, channel, response, requestId, action, options);
+
+ }
+
+ @Override
+ public void sendResponse(Throwable error) throws IOException {
+ release();
+ transport.sendErrorResponse(version, channel, error, requestId, action);
+ }
+
+ private void release() {
+ // attempt to release once atomically
+ if (released.compareAndSet(false, true) == false) {
+ throw new IllegalStateException("reserved bytes are already released");
+ }
+ transport.getInFlightRequestBreaker().addWithoutBreaking(-reservedBytes);
+ }
+
+ @Override
+ public final long getRequestId() {
+ return requestId;
+ }
+
+ @Override
+ public final String getChannelType() {
+ return channelType;
+ }
+
+ public Channel getChannel() {
+ return channel;
+ }
+
+}
+
diff --git a/core/src/main/java/org/elasticsearch/transport/Transport.java b/core/src/main/java/org/elasticsearch/transport/Transport.java
index 76793d5598f..754d2105e83 100644
--- a/core/src/main/java/org/elasticsearch/transport/Transport.java
+++ b/core/src/main/java/org/elasticsearch/transport/Transport.java
@@ -20,6 +20,8 @@
package org.elasticsearch.transport;
import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.breaker.CircuitBreaker;
+import org.elasticsearch.common.breaker.NoopCircuitBreaker;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
@@ -94,4 +96,9 @@ public interface Transport extends LifecycleComponent {
long serverOpen();
List getLocalAddresses();
+
+ default CircuitBreaker getInFlightRequestBreaker() {
+ return new NoopCircuitBreaker("in-flight-noop");
+ }
+
}
diff --git a/core/src/main/java/org/elasticsearch/transport/Transports.java b/core/src/main/java/org/elasticsearch/transport/Transports.java
index 68d828fc72f..1186c821309 100644
--- a/core/src/main/java/org/elasticsearch/transport/Transports.java
+++ b/core/src/main/java/org/elasticsearch/transport/Transports.java
@@ -20,7 +20,6 @@
package org.elasticsearch.transport;
import org.elasticsearch.transport.local.LocalTransport;
-import org.elasticsearch.transport.netty.NettyTransport;
import java.util.Arrays;
@@ -39,10 +38,10 @@ public enum Transports {
final String threadName = t.getName();
for (String s : Arrays.asList(
LocalTransport.LOCAL_TRANSPORT_THREAD_NAME_PREFIX,
- NettyTransport.HTTP_SERVER_BOSS_THREAD_NAME_PREFIX,
- NettyTransport.HTTP_SERVER_WORKER_THREAD_NAME_PREFIX,
- NettyTransport.TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX,
- NettyTransport.TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX,
+ TcpTransport.HTTP_SERVER_BOSS_THREAD_NAME_PREFIX,
+ TcpTransport.HTTP_SERVER_WORKER_THREAD_NAME_PREFIX,
+ TcpTransport.TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX,
+ TcpTransport.TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX,
TEST_MOCK_TRANSPORT_THREAD_PREFIX)) {
if (threadName.contains(s)) {
return true;
diff --git a/core/src/main/java/org/elasticsearch/common/netty/ChannelBufferBytesReference.java b/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferBytesReference.java
similarity index 85%
rename from core/src/main/java/org/elasticsearch/common/netty/ChannelBufferBytesReference.java
rename to core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferBytesReference.java
index 42cfe3c611f..60dbdd21bd1 100644
--- a/core/src/main/java/org/elasticsearch/common/netty/ChannelBufferBytesReference.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferBytesReference.java
@@ -16,13 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.elasticsearch.common.netty;
+package org.elasticsearch.transport.netty;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.transport.netty.ChannelBufferStreamInputFactory;
import org.jboss.netty.buffer.ChannelBuffer;
import java.io.IOException;
@@ -34,9 +33,12 @@ import java.nio.charset.StandardCharsets;
final class ChannelBufferBytesReference implements BytesReference {
private final ChannelBuffer buffer;
+ private final int size;
- ChannelBufferBytesReference(ChannelBuffer buffer) {
+ ChannelBufferBytesReference(ChannelBuffer buffer, int size) {
this.buffer = buffer;
+ this.size = size;
+ assert size <= buffer.readableBytes() : "size[" + size +"] > " + buffer.readableBytes();
}
@Override
@@ -46,25 +48,24 @@ final class ChannelBufferBytesReference implements BytesReference {
@Override
public int length() {
- return buffer.readableBytes();
+ return size;
}
@Override
public BytesReference slice(int from, int length) {
- return new ChannelBufferBytesReference(buffer.slice(buffer.readerIndex() + from, length));
+ return new ChannelBufferBytesReference(buffer.slice(buffer.readerIndex() + from, length), length);
}
@Override
public StreamInput streamInput() {
- return ChannelBufferStreamInputFactory.create(buffer.duplicate());
+ return new ChannelBufferStreamInput(buffer.duplicate(), size);
}
@Override
public void writeTo(OutputStream os) throws IOException {
- buffer.getBytes(buffer.readerIndex(), os, length());
+ buffer.getBytes(buffer.readerIndex(), os, size);
}
- @Override
public byte[] toBytes() {
return copyBytesArray().toBytes();
}
@@ -72,7 +73,7 @@ final class ChannelBufferBytesReference implements BytesReference {
@Override
public BytesArray toBytesArray() {
if (buffer.hasArray()) {
- return new BytesArray(buffer.array(), buffer.arrayOffset() + buffer.readerIndex(), buffer.readableBytes());
+ return new BytesArray(buffer.array(), buffer.arrayOffset() + buffer.readerIndex(), size);
}
return copyBytesArray();
}
@@ -111,7 +112,7 @@ final class ChannelBufferBytesReference implements BytesReference {
@Override
public BytesRef toBytesRef() {
if (buffer.hasArray()) {
- return new BytesRef(buffer.array(), buffer.arrayOffset() + buffer.readerIndex(), buffer.readableBytes());
+ return new BytesRef(buffer.array(), buffer.arrayOffset() + buffer.readerIndex(), size);
}
byte[] copy = new byte[buffer.readableBytes()];
buffer.getBytes(buffer.readerIndex(), copy);
@@ -120,7 +121,7 @@ final class ChannelBufferBytesReference implements BytesReference {
@Override
public BytesRef copyBytesRef() {
- byte[] copy = new byte[buffer.readableBytes()];
+ byte[] copy = new byte[size];
buffer.getBytes(buffer.readerIndex(), copy);
return new BytesRef(copy);
}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInput.java b/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInput.java
index 9e2b43d43db..3b95ddd74c7 100644
--- a/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInput.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInput.java
@@ -22,16 +22,14 @@ package org.elasticsearch.transport.netty;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.netty.NettyUtils;
import org.jboss.netty.buffer.ChannelBuffer;
-import java.io.EOFException;
import java.io.IOException;
/**
* A Netty {@link org.jboss.netty.buffer.ChannelBuffer} based {@link org.elasticsearch.common.io.stream.StreamInput}.
*/
-public class ChannelBufferStreamInput extends StreamInput {
+class ChannelBufferStreamInput extends StreamInput {
private final ChannelBuffer buffer;
private final int startIndex;
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInputFactory.java b/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInputFactory.java
deleted file mode 100644
index 554f710ec4d..00000000000
--- a/core/src/main/java/org/elasticsearch/transport/netty/ChannelBufferStreamInputFactory.java
+++ /dev/null
@@ -1,36 +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.netty;
-
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.jboss.netty.buffer.ChannelBuffer;
-
-/**
- */
-public class ChannelBufferStreamInputFactory {
-
- public static StreamInput create(ChannelBuffer buffer) {
- return new ChannelBufferStreamInput(buffer, buffer.readableBytes());
- }
-
- public static StreamInput create(ChannelBuffer buffer, int size) {
- return new ChannelBufferStreamInput(buffer, size);
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/MessageChannelHandler.java b/core/src/main/java/org/elasticsearch/transport/netty/MessageChannelHandler.java
deleted file mode 100644
index 3274aa7b975..00000000000
--- a/core/src/main/java/org/elasticsearch/transport/netty/MessageChannelHandler.java
+++ /dev/null
@@ -1,437 +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.netty;
-
-import org.apache.lucene.util.IOUtils;
-import org.elasticsearch.Version;
-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.NamedWriteableAwareStreamInput;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.logging.ESLogger;
-import org.elasticsearch.common.netty.NettyUtils;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.util.concurrent.AbstractRunnable;
-import org.elasticsearch.common.util.concurrent.ThreadContext;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.ActionNotFoundTransportException;
-import org.elasticsearch.transport.RemoteTransportException;
-import org.elasticsearch.transport.RequestHandlerRegistry;
-import org.elasticsearch.transport.ResponseHandlerFailureTransportException;
-import org.elasticsearch.transport.TransportRequest;
-import org.elasticsearch.transport.TransportResponse;
-import org.elasticsearch.transport.TransportResponseHandler;
-import org.elasticsearch.transport.TransportSerializationException;
-import org.elasticsearch.transport.TransportServiceAdapter;
-import org.elasticsearch.transport.Transports;
-import org.elasticsearch.transport.support.TransportStatus;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.channel.WriteCompletionEvent;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-/**
- * A handler (must be the last one!) that does size based frame decoding and forwards the actual message
- * to the relevant action.
- */
-public class MessageChannelHandler extends SimpleChannelUpstreamHandler {
-
- protected final ESLogger logger;
- protected final ThreadPool threadPool;
- protected final TransportServiceAdapter transportServiceAdapter;
- protected final NettyTransport transport;
- protected final String profileName;
- private final ThreadContext threadContext;
-
- public MessageChannelHandler(NettyTransport transport, ESLogger logger, String profileName) {
- this.threadPool = transport.threadPool();
- this.threadContext = threadPool.getThreadContext();
- this.transportServiceAdapter = transport.transportServiceAdapter();
- this.transport = transport;
- this.logger = logger;
- this.profileName = profileName;
- }
-
- @Override
- public void writeComplete(ChannelHandlerContext ctx, WriteCompletionEvent e) throws Exception {
- transportServiceAdapter.sent(e.getWrittenAmount());
- super.writeComplete(ctx, e);
- }
-
- @Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
- Transports.assertTransportThread();
- Object m = e.getMessage();
- if (!(m instanceof ChannelBuffer)) {
- ctx.sendUpstream(e);
- return;
- }
- ChannelBuffer buffer = (ChannelBuffer) m;
- Marker marker = new Marker(buffer);
- int size = marker.messageSizeWithRemainingHeaders();
- transportServiceAdapter.received(marker.messageSizeWithAllHeaders());
-
- // we have additional bytes to read, outside of the header
- boolean hasMessageBytesToRead = marker.messageSize() != 0;
-
- // netty always copies a buffer, either in NioWorker in its read handler, where it copies to a fresh
- // buffer, or in the cumulation buffer, which is cleaned each time
- StreamInput streamIn = ChannelBufferStreamInputFactory.create(buffer, size);
- boolean success = false;
- try (ThreadContext.StoredContext tCtx = threadContext.stashContext()) {
- long requestId = streamIn.readLong();
- byte status = streamIn.readByte();
- Version version = Version.fromId(streamIn.readInt());
-
- if (TransportStatus.isCompress(status) && hasMessageBytesToRead && buffer.readable()) {
- Compressor compressor;
- try {
- compressor = CompressorFactory.compressor(NettyUtils.toBytesReference(buffer));
- } catch (NotCompressedException ex) {
- int maxToRead = Math.min(buffer.readableBytes(), 10);
- int offset = buffer.readerIndex();
- StringBuilder sb = new StringBuilder("stream marked as compressed, but no compressor found, first [").append(maxToRead)
- .append("] content bytes out of [").append(buffer.readableBytes())
- .append("] readable bytes with message size [").append(size).append("] ").append("] are [");
- for (int i = 0; i < maxToRead; i++) {
- sb.append(buffer.getByte(offset + i)).append(",");
- }
- sb.append("]");
- throw new IllegalStateException(sb.toString());
- }
- streamIn = compressor.streamInput(streamIn);
- }
- if (version.onOrAfter(Version.CURRENT.minimumCompatibilityVersion()) == false || version.major != Version.CURRENT.major) {
- throw new IllegalStateException("Received message from unsupported version: [" + version
- + "] minimal compatible version is: [" +Version.CURRENT.minimumCompatibilityVersion() + "]");
- }
- streamIn.setVersion(version);
- if (TransportStatus.isRequest(status)) {
- threadContext.readHeaders(streamIn);
- handleRequest(ctx.getChannel(), marker, streamIn, requestId, size, version);
- } else {
- TransportResponseHandler> handler = transportServiceAdapter.onResponseReceived(requestId);
- // ignore if its null, the adapter logs it
- if (handler != null) {
- if (TransportStatus.isError(status)) {
- handlerResponseError(streamIn, handler);
- } else {
- handleResponse(ctx.getChannel(), streamIn, handler);
- }
- marker.validateResponse(streamIn, requestId, handler, TransportStatus.isError(status));
- }
- }
- success = true;
- } finally {
- try {
- if (success) {
- IOUtils.close(streamIn);
- } else {
- IOUtils.closeWhileHandlingException(streamIn);
- }
- } finally {
- // Set the expected position of the buffer, no matter what happened
- buffer.readerIndex(marker.expectedReaderIndex());
- }
- }
- }
-
- protected void handleResponse(Channel channel, StreamInput buffer, final TransportResponseHandler handler) {
- buffer = new NamedWriteableAwareStreamInput(buffer, transport.namedWriteableRegistry);
- final TransportResponse response = handler.newInstance();
- response.remoteAddress(new InetSocketTransportAddress((InetSocketAddress) channel.getRemoteAddress()));
- response.remoteAddress();
- try {
- response.readFrom(buffer);
- } catch (Throwable e) {
- handleException(handler, new TransportSerializationException(
- "Failed to deserialize response of type [" + response.getClass().getName() + "]", e));
- return;
- }
- try {
- if (ThreadPool.Names.SAME.equals(handler.executor())) {
- //noinspection unchecked
- handler.handleResponse(response);
- } else {
- threadPool.executor(handler.executor()).execute(new ResponseHandler(handler, response));
- }
- } catch (Throwable e) {
- handleException(handler, new ResponseHandlerFailureTransportException(e));
- }
- }
-
- private void handlerResponseError(StreamInput buffer, final TransportResponseHandler handler) {
- Throwable error;
- try {
- error = buffer.readThrowable();
- } catch (Throwable 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;
- if (ThreadPool.Names.SAME.equals(handler.executor())) {
- try {
- handler.handleException(rtx);
- } catch (Throwable e) {
- logger.error("failed to handle exception response [{}]", e, handler);
- }
- } else {
- threadPool.executor(handler.executor()).execute(new Runnable() {
- @Override
- public void run() {
- try {
- handler.handleException(rtx);
- } catch (Throwable e) {
- logger.error("failed to handle exception response [{}]", e, handler);
- }
- }
- });
- }
- }
-
- protected String handleRequest(Channel channel, Marker marker, StreamInput buffer, long requestId, int messageLengthBytes,
- Version version) throws IOException {
- buffer = new NamedWriteableAwareStreamInput(buffer, transport.namedWriteableRegistry);
- final String action = buffer.readString();
- transportServiceAdapter.onRequestReceived(requestId, action);
- NettyTransportChannel transportChannel = null;
- try {
- final RequestHandlerRegistry reg = transportServiceAdapter.getRequestHandler(action);
- if (reg == null) {
- throw new ActionNotFoundTransportException(action);
- }
- if (reg.canTripCircuitBreaker()) {
- transport.inFlightRequestsBreaker().addEstimateBytesAndMaybeBreak(messageLengthBytes, "");
- } else {
- transport.inFlightRequestsBreaker().addWithoutBreaking(messageLengthBytes);
- }
- transportChannel = new NettyTransportChannel(transport, transportServiceAdapter, action, channel,
- requestId, version, profileName, messageLengthBytes);
- final TransportRequest request = reg.newRequest();
- request.remoteAddress(new InetSocketTransportAddress((InetSocketAddress) channel.getRemoteAddress()));
- request.readFrom(buffer);
- // in case we throw an exception, i.e. when the limit is hit, we don't want to verify
- validateRequest(marker, buffer, requestId, action);
- if (ThreadPool.Names.SAME.equals(reg.getExecutor())) {
- //noinspection unchecked
- reg.processMessageReceived(request, transportChannel);
- } else {
- threadPool.executor(reg.getExecutor()).execute(new RequestHandler(reg, request, transportChannel));
- }
- } catch (Throwable e) {
- // the circuit breaker tripped
- if (transportChannel == null) {
- transportChannel = new NettyTransportChannel(transport, transportServiceAdapter, action, channel,
- requestId, version, profileName, 0);
- }
- try {
- transportChannel.sendResponse(e);
- } catch (IOException e1) {
- logger.warn("Failed to send error message back to client for action [{}]", e, action);
- logger.warn("Actual Exception", e1);
- }
- }
- return action;
- }
-
- // This template method is needed to inject custom error checking logic in tests.
- protected void validateRequest(Marker marker, StreamInput buffer, long requestId, String action) throws IOException {
- marker.validateRequest(buffer, requestId, action);
- }
-
-
- @Override
- public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
- transport.exceptionCaught(ctx, e);
- }
-
- class ResponseHandler implements Runnable {
-
- private final TransportResponseHandler handler;
- private final TransportResponse response;
-
- public ResponseHandler(TransportResponseHandler handler, TransportResponse response) {
- this.handler = handler;
- this.response = response;
- }
-
- @SuppressWarnings({"unchecked"})
- @Override
- public void run() {
- try {
- handler.handleResponse(response);
- } catch (Throwable e) {
- handleException(handler, new ResponseHandlerFailureTransportException(e));
- }
- }
- }
-
- class RequestHandler extends AbstractRunnable {
- private final RequestHandlerRegistry reg;
- private final TransportRequest request;
- private final NettyTransportChannel transportChannel;
-
- public RequestHandler(RequestHandlerRegistry reg, TransportRequest request, NettyTransportChannel 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(Throwable e) {
- if (transport.lifecycleState() == Lifecycle.State.STARTED) {
- // we can only send a response transport is started....
- try {
- transportChannel.sendResponse(e);
- } catch (Throwable e1) {
- logger.warn("Failed to send error message back to client for action [{}]", e1, reg.getAction());
- logger.warn("Actual Exception", e);
- }
- }
- }
- }
-
- /**
- * Internal helper class to store characteristic offsets of a buffer during processing
- */
- protected static final class Marker {
- private final ChannelBuffer buffer;
- private final int remainingMessageSize;
- private final int expectedReaderIndex;
-
- public Marker(ChannelBuffer buffer) {
- this.buffer = buffer;
- // when this constructor is called, we have read already two parts of the message header: the marker bytes and the message
- // message length (see SizeHeaderFrameDecoder). Hence we have to rewind the index for MESSAGE_LENGTH_SIZE bytes to read the
- // remaining message length again.
- this.remainingMessageSize = buffer.getInt(buffer.readerIndex() - NettyHeader.MESSAGE_LENGTH_SIZE);
- this.expectedReaderIndex = buffer.readerIndex() + remainingMessageSize;
- }
-
- /**
- * @return the number of bytes that have yet to be read from the buffer
- */
- public int messageSizeWithRemainingHeaders() {
- return remainingMessageSize;
- }
-
- /**
- * @return the number in bytes for the message including all headers (even the ones that have been read from the buffer already)
- */
- public int messageSizeWithAllHeaders() {
- return remainingMessageSize + NettyHeader.MARKER_BYTES_SIZE + NettyHeader.MESSAGE_LENGTH_SIZE;
- }
-
- /**
- * @return the number of bytes for the message itself (excluding all headers).
- */
- public int messageSize() {
- return messageSizeWithAllHeaders() - NettyHeader.HEADER_SIZE;
- }
-
- /**
- * @return the expected index of the buffer's reader after the message has been consumed entirely.
- */
- public int expectedReaderIndex() {
- return expectedReaderIndex;
- }
-
- /**
- * Validates that a request has been fully read (not too few bytes but also not too many bytes).
- *
- * @param stream A stream that is associated with the buffer that is tracked by this marker.
- * @param requestId The current request id.
- * @param action The currently executed action.
- * @throws IOException Iff the stream could not be read.
- * @throws IllegalStateException Iff the request has not been fully read.
- */
- public 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
- + "], readerIndex [" + buffer.readerIndex() + "] vs expected [" + expectedReaderIndex + "]; resetting");
- }
- if (buffer.readerIndex() < expectedReaderIndex) {
- throw new IllegalStateException("Message is fully read (request), yet there are "
- + (expectedReaderIndex - buffer.readerIndex()) + " remaining bytes; resetting");
- }
- if (buffer.readerIndex() > expectedReaderIndex) {
- throw new IllegalStateException(
- "Message read past expected size (request) for requestId [" + requestId + "], action [" + action
- + "], readerIndex [" + buffer.readerIndex() + "] vs expected [" + expectedReaderIndex + "]; resetting");
- }
- }
-
- /**
- * Validates that a response has been fully read (not too few bytes but also not too many bytes).
- *
- * @param stream A stream that is associated with the buffer that is tracked by this marker.
- * @param requestId The corresponding request id for this response.
- * @param handler The current response handler.
- * @param error Whether validate an error response.
- * @throws IOException Iff the stream could not be read.
- * @throws IllegalStateException Iff the request has not been fully read.
- */
- public void validateResponse(StreamInput stream, long requestId,
- TransportResponseHandler> handler, boolean error) throws IOException {
- // Check the entire message has been read
- final int nextByte = stream.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 [" + error + "]; resetting");
- }
- if (buffer.readerIndex() < expectedReaderIndex) {
- throw new IllegalStateException("Message is fully read (response), yet there are "
- + (expectedReaderIndex - buffer.readerIndex()) + " remaining bytes; resetting");
- }
- if (buffer.readerIndex() > expectedReaderIndex) {
- throw new IllegalStateException("Message read past expected size (response) for requestId [" + requestId
- + "], handler [" + handler + "], error [" + error + "]; resetting");
- }
- }
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyHeader.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyHeader.java
deleted file mode 100644
index 8e4423fb447..00000000000
--- a/core/src/main/java/org/elasticsearch/transport/netty/NettyHeader.java
+++ /dev/null
@@ -1,76 +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.netty;
-
-import org.elasticsearch.Version;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-
-/**
- */
-public class NettyHeader {
- public static final int MARKER_BYTES_SIZE = 2 * 1;
-
- public static final int MESSAGE_LENGTH_SIZE = 4;
-
- public static final int REQUEST_ID_SIZE = 8;
-
- public static final int STATUS_SIZE = 1;
-
- public static final int VERSION_ID_SIZE = 4;
-
- public static final int HEADER_SIZE = MARKER_BYTES_SIZE + MESSAGE_LENGTH_SIZE + REQUEST_ID_SIZE + STATUS_SIZE + VERSION_ID_SIZE;
-
- /**
- * The magic number (must be lower than 0) for a ping message. This is handled
- * specifically in {@link org.elasticsearch.transport.netty.SizeHeaderFrameDecoder}.
- */
- public static final int PING_DATA_SIZE = -1;
- private final static ChannelBuffer pingHeader;
- static {
- pingHeader = ChannelBuffers.buffer(6);
- pingHeader.writeByte('E');
- pingHeader.writeByte('S');
- pingHeader.writeInt(PING_DATA_SIZE);
- }
-
- /**
- * A ping header is same as regular header, just with -1 for the size of the message.
- */
- public static ChannelBuffer pingHeader() {
- return pingHeader.duplicate();
- }
-
- public static void writeHeader(ChannelBuffer buffer, long requestId, byte status, Version version) {
- int index = buffer.readerIndex();
- buffer.setByte(index, 'E');
- index += 1;
- buffer.setByte(index, 'S');
- index += 1;
- // write the size, the size indicates the remaining message size, not including the size int
- buffer.setInt(index, buffer.readableBytes() - MARKER_BYTES_SIZE - MESSAGE_LENGTH_SIZE);
- index += MESSAGE_LENGTH_SIZE;
- buffer.setLong(index, requestId);
- index += REQUEST_ID_SIZE;
- buffer.setByte(index, status);
- index += STATUS_SIZE;
- buffer.setInt(index, version.id);
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLogger.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLogger.java
index 2a1fc3226a4..d8307f32244 100644
--- a/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLogger.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLogger.java
@@ -27,11 +27,11 @@ import org.jboss.netty.logging.AbstractInternalLogger;
*
*/
@SuppressLoggerChecks(reason = "safely delegates to logger")
-public class NettyInternalESLogger extends AbstractInternalLogger {
+final class NettyInternalESLogger extends AbstractInternalLogger {
private final ESLogger logger;
- public NettyInternalESLogger(ESLogger logger) {
+ NettyInternalESLogger(ESLogger logger) {
this.logger = logger;
}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLoggerFactory.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLoggerFactory.java
deleted file mode 100644
index 70c5e651c2c..00000000000
--- a/core/src/main/java/org/elasticsearch/transport/netty/NettyInternalESLoggerFactory.java
+++ /dev/null
@@ -1,35 +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.netty;
-
-import org.elasticsearch.common.logging.Loggers;
-import org.jboss.netty.logging.InternalLogger;
-import org.jboss.netty.logging.InternalLoggerFactory;
-
-/**
- *
- */
-public class NettyInternalESLoggerFactory extends InternalLoggerFactory {
-
- @Override
- public InternalLogger newInstance(String name) {
- return new NettyInternalESLogger(Loggers.getLogger(name));
- }
-}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyMessageChannelHandler.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyMessageChannelHandler.java
new file mode 100644
index 00000000000..0f2805459c9
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/transport/netty/NettyMessageChannelHandler.java
@@ -0,0 +1,86 @@
+/*
+ * 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.netty;
+
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.transport.TcpHeader;
+import org.elasticsearch.transport.TcpTransport;
+import org.elasticsearch.transport.TcpTransportChannel;
+import org.elasticsearch.transport.TransportServiceAdapter;
+import org.elasticsearch.transport.Transports;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.WriteCompletionEvent;
+
+import java.net.InetSocketAddress;
+
+/**
+ * A handler (must be the last one!) that does size based frame decoding and forwards the actual message
+ * to the relevant action.
+ */
+class NettyMessageChannelHandler extends SimpleChannelUpstreamHandler {
+
+ protected final TransportServiceAdapter transportServiceAdapter;
+ protected final NettyTransport transport;
+ protected final String profileName;
+
+ NettyMessageChannelHandler(NettyTransport transport, String profileName) {
+ this.transportServiceAdapter = transport.transportServiceAdapter();
+ this.transport = transport;
+ this.profileName = profileName;
+ }
+
+ @Override
+ public void writeComplete(ChannelHandlerContext ctx, WriteCompletionEvent e) throws Exception {
+ transportServiceAdapter.sent(e.getWrittenAmount());
+ super.writeComplete(ctx, e);
+ }
+
+ @Override
+ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
+ Transports.assertTransportThread();
+ Object m = e.getMessage();
+ if (!(m instanceof ChannelBuffer)) {
+ ctx.sendUpstream(e);
+ return;
+ }
+ final ChannelBuffer buffer = (ChannelBuffer) m;
+ final int remainingMessageSize = buffer.getInt(buffer.readerIndex() - TcpHeader.MESSAGE_LENGTH_SIZE);
+ final int expectedReaderIndex = buffer.readerIndex() + remainingMessageSize;
+ InetSocketAddress remoteAddress = (InetSocketAddress) ctx.getChannel().getRemoteAddress();
+ try {
+ // netty always copies a buffer, either in NioWorker in its read handler, where it copies to a fresh
+ // buffer, or in the cumulation buffer, which is cleaned each time so it could be bigger than the actual size
+ BytesReference reference = NettyUtils.toBytesReference(buffer, remainingMessageSize);
+ transport.messageReceived(reference, ctx.getChannel(), profileName, remoteAddress, remainingMessageSize);
+ } finally {
+ // Set the expected position of the buffer, no matter what happened
+ buffer.readerIndex(expectedReaderIndex);
+ }
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
+ transport.exceptionCaught(ctx, e);
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java
index be1305244bc..3c75f68eeb0 100644
--- a/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/NettyTransport.java
@@ -19,58 +19,36 @@
package org.elasticsearch.transport.netty;
-import com.carrotsearch.hppc.IntHashSet;
-import com.carrotsearch.hppc.IntSet;
-import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
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.BytesReference;
import org.elasticsearch.common.bytes.ReleasablePagedBytesReference;
-import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.compress.CompressorFactory;
import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.common.io.ReleasableBytesStream;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
-import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
-import org.elasticsearch.common.metrics.CounterMetric;
-import org.elasticsearch.common.netty.NettyUtils;
-import org.elasticsearch.common.netty.OpenChannelsHandler;
-import org.elasticsearch.common.netty.ReleaseChannelFutureListener;
-import org.elasticsearch.common.network.NetworkAddress;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.network.NetworkService.TcpSettings;
import org.elasticsearch.common.network.NetworkUtils;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.transport.PortsRange;
-import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.common.util.concurrent.AbstractLifecycleRunnable;
import org.elasticsearch.common.util.concurrent.EsExecutors;
-import org.elasticsearch.common.util.concurrent.KeyedLock;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.BindTransportException;
import org.elasticsearch.transport.BytesTransportRequest;
import org.elasticsearch.transport.ConnectTransportException;
-import org.elasticsearch.transport.NodeNotConnectedException;
-import org.elasticsearch.transport.Transport;
-import org.elasticsearch.transport.TransportException;
+import org.elasticsearch.transport.TcpTransport;
+import org.elasticsearch.transport.TransportMessage;
import org.elasticsearch.transport.TransportRequest;
-import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportServiceAdapter;
import org.elasticsearch.transport.TransportSettings;
-import org.elasticsearch.transport.support.TransportStatus;
import org.jboss.netty.bootstrap.ClientBootstrap;
import org.jboss.netty.bootstrap.ServerBootstrap;
import org.jboss.netty.buffer.ChannelBuffer;
@@ -94,42 +72,17 @@ import org.jboss.netty.channel.socket.oio.OioServerSocketChannelFactory;
import org.jboss.netty.util.HashedWheelTimer;
import java.io.IOException;
-import java.net.BindException;
-import java.net.InetAddress;
import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.net.UnknownHostException;
-import java.nio.channels.CancelledKeyException;
-import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import static java.util.Collections.unmodifiableMap;
-import static org.elasticsearch.common.settings.Setting.boolSetting;
import static org.elasticsearch.common.settings.Setting.byteSizeSetting;
import static org.elasticsearch.common.settings.Setting.intSetting;
-import static org.elasticsearch.common.settings.Setting.timeSetting;
-import static org.elasticsearch.common.transport.NetworkExceptionHelper.isCloseConnectionException;
-import static org.elasticsearch.common.transport.NetworkExceptionHelper.isConnectException;
import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory;
@@ -139,51 +92,16 @@ import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadF
* longer. Med is for the typical search / single doc index. And High for things like cluster state. Ping is reserved for
* sending out ping requests to other nodes.
*/
-public class NettyTransport extends AbstractLifecycleComponent implements Transport {
+public class NettyTransport extends TcpTransport {
static {
NettyUtils.setup();
}
- public static final String HTTP_SERVER_WORKER_THREAD_NAME_PREFIX = "http_server_worker";
- public static final String HTTP_SERVER_BOSS_THREAD_NAME_PREFIX = "http_server_boss";
- public static final String TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX = "transport_client_worker";
- public static final String TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX = "transport_client_boss";
-
public static final Setting WORKER_COUNT =
new Setting<>("transport.netty.worker_count",
(s) -> Integer.toString(EsExecutors.boundedNumberOfProcessors(s) * 2),
(s) -> Setting.parseInt(s, 1, "transport.netty.worker_count"), Property.NodeScope);
- public static final Setting CONNECTIONS_PER_NODE_RECOVERY =
- intSetting("transport.connections_per_node.recovery", 2, 1, Property.NodeScope);
- public static final Setting CONNECTIONS_PER_NODE_BULK =
- intSetting("transport.connections_per_node.bulk", 3, 1, Property.NodeScope);
- public static final Setting CONNECTIONS_PER_NODE_REG =
- intSetting("transport.connections_per_node.reg", 6, 1, Property.NodeScope);
- public static final Setting CONNECTIONS_PER_NODE_STATE =
- intSetting("transport.connections_per_node.state", 1, 1, Property.NodeScope);
- public static final Setting CONNECTIONS_PER_NODE_PING =
- intSetting("transport.connections_per_node.ping", 1, 1, Property.NodeScope);
- // the scheduled internal ping interval setting, defaults to disabled (-1)
- public static final Setting PING_SCHEDULE =
- timeSetting("transport.ping_schedule", TimeValue.timeValueSeconds(-1), Property.NodeScope);
- public static final Setting TCP_BLOCKING_CLIENT =
- boolSetting("transport.tcp.blocking_client", TcpSettings.TCP_BLOCKING_CLIENT, Property.NodeScope);
- public static final Setting TCP_CONNECT_TIMEOUT =
- timeSetting("transport.tcp.connect_timeout", TcpSettings.TCP_CONNECT_TIMEOUT, Property.NodeScope);
- public static final Setting TCP_NO_DELAY =
- boolSetting("transport.tcp_no_delay", TcpSettings.TCP_NO_DELAY, Property.NodeScope);
- public static final Setting TCP_KEEP_ALIVE =
- boolSetting("transport.tcp.keep_alive", TcpSettings.TCP_KEEP_ALIVE, Property.NodeScope);
- public static final Setting TCP_BLOCKING_SERVER =
- boolSetting("transport.tcp.blocking_server", TcpSettings.TCP_BLOCKING_SERVER, Property.NodeScope);
- public static final Setting TCP_REUSE_ADDRESS =
- boolSetting("transport.tcp.reuse_address", TcpSettings.TCP_REUSE_ADDRESS, Property.NodeScope);
-
- public static final Setting TCP_SEND_BUFFER_SIZE =
- Setting.byteSizeSetting("transport.tcp.send_buffer_size", TcpSettings.TCP_SEND_BUFFER_SIZE, Property.NodeScope);
- public static final Setting TCP_RECEIVE_BUFFER_SIZE =
- Setting.byteSizeSetting("transport.tcp.receive_buffer_size", TcpSettings.TCP_RECEIVE_BUFFER_SIZE, Property.NodeScope);
public static final Setting NETTY_MAX_CUMULATION_BUFFER_CAPACITY =
Setting.byteSizeSetting("transport.netty.max_cumulation_buffer_capacity", new ByteSizeValue(-1), Property.NodeScope);
@@ -192,16 +110,16 @@ public class NettyTransport extends AbstractLifecycleComponent implem
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
public static final Setting NETTY_RECEIVE_PREDICTOR_SIZE = Setting.byteSizeSetting(
- "transport.netty.receive_predictor_size",
- settings -> {
- long defaultReceiverPredictor = 512 * 1024;
- if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
- // we can guess a better default...
- long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / WORKER_COUNT.get(settings));
- defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
- }
- return new ByteSizeValue(defaultReceiverPredictor).toString();
- }, Property.NodeScope);
+ "transport.netty.receive_predictor_size",
+ settings -> {
+ long defaultReceiverPredictor = 512 * 1024;
+ if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
+ // we can guess a better default...
+ long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / WORKER_COUNT.get(settings));
+ defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
+ }
+ return new ByteSizeValue(defaultReceiverPredictor).toString();
+ }, Property.NodeScope);
public static final Setting NETTY_RECEIVE_PREDICTOR_MIN =
byteSizeSetting("transport.netty.receive_predictor_min", NETTY_RECEIVE_PREDICTOR_SIZE, Property.NodeScope);
public static final Setting NETTY_RECEIVE_PREDICTOR_MAX =
@@ -209,69 +127,25 @@ public class NettyTransport extends AbstractLifecycleComponent implem
public static final Setting NETTY_BOSS_COUNT =
intSetting("transport.netty.boss_count", 1, 1, Property.NodeScope);
- protected final NetworkService networkService;
- protected final boolean blockingClient;
- protected final TimeValue connectTimeout;
protected final ByteSizeValue maxCumulationBufferCapacity;
protected final int maxCompositeBufferComponents;
- protected final boolean compress;
protected final ReceiveBufferSizePredictorFactory receiveBufferSizePredictorFactory;
protected final int workerCount;
protected final ByteSizeValue receivePredictorMin;
protected final ByteSizeValue receivePredictorMax;
-
- protected final int connectionsPerNodeRecovery;
- protected final int connectionsPerNodeBulk;
- protected final int connectionsPerNodeReg;
- protected final int connectionsPerNodeState;
- protected final int connectionsPerNodePing;
-
- private final TimeValue pingSchedule;
-
- protected final BigArrays bigArrays;
- protected final ThreadPool threadPool;
// package private for testing
volatile OpenChannelsHandler serverOpenChannels;
protected volatile ClientBootstrap clientBootstrap;
- // node id to actual channel
- protected final ConcurrentMap connectedNodes = newConcurrentMap();
protected final Map serverBootstraps = newConcurrentMap();
- protected final Map> serverChannels = newConcurrentMap();
- protected final ConcurrentMap profileBoundAddresses = newConcurrentMap();
- protected volatile TransportServiceAdapter transportServiceAdapter;
- protected volatile BoundTransportAddress boundAddress;
- protected final KeyedLock connectionLock = new KeyedLock<>();
- protected final NamedWriteableRegistry namedWriteableRegistry;
- private final CircuitBreakerService circuitBreakerService;
-
- // this lock is here to make sure we close this transport and disconnect all the client nodes
- // connections while no connect operations is going on... (this might help with 100% CPU when stopping the transport?)
- private final ReadWriteLock globalLock = new ReentrantReadWriteLock();
-
- // package visibility for tests
- final ScheduledPing scheduledPing;
@Inject
public NettyTransport(Settings settings, ThreadPool threadPool, NetworkService networkService, BigArrays bigArrays,
NamedWriteableRegistry namedWriteableRegistry, CircuitBreakerService circuitBreakerService) {
- super(settings);
- this.threadPool = threadPool;
- this.networkService = networkService;
- this.bigArrays = bigArrays;
-
+ super("netty", settings, threadPool, bigArrays, circuitBreakerService, namedWriteableRegistry, networkService);
this.workerCount = WORKER_COUNT.get(settings);
- this.blockingClient = TCP_BLOCKING_CLIENT.get(settings);
- this.connectTimeout = TCP_CONNECT_TIMEOUT.get(settings);
this.maxCumulationBufferCapacity = NETTY_MAX_CUMULATION_BUFFER_CAPACITY.get(settings);
this.maxCompositeBufferComponents = NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS.get(settings);
- this.compress = Transport.TRANSPORT_TCP_COMPRESS.get(settings);
-
- this.connectionsPerNodeRecovery = CONNECTIONS_PER_NODE_RECOVERY.get(settings);
- this.connectionsPerNodeBulk = CONNECTIONS_PER_NODE_BULK.get(settings);
- this.connectionsPerNodeReg = CONNECTIONS_PER_NODE_REG.get(settings);
- this.connectionsPerNodeState = CONNECTIONS_PER_NODE_STATE.get(settings);
- this.connectionsPerNodePing = CONNECTIONS_PER_NODE_PING.get(settings);
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
this.receivePredictorMin = NETTY_RECEIVE_PREDICTOR_MIN.get(settings);
@@ -280,37 +154,14 @@ public class NettyTransport extends AbstractLifecycleComponent implem
receiveBufferSizePredictorFactory = new FixedReceiveBufferSizePredictorFactory((int) receivePredictorMax.bytes());
} else {
receiveBufferSizePredictorFactory = new AdaptiveReceiveBufferSizePredictorFactory((int) receivePredictorMin.bytes(),
- (int) receivePredictorMin.bytes(), (int) receivePredictorMax.bytes());
+ (int) receivePredictorMin.bytes(), (int) receivePredictorMax.bytes());
}
-
- this.scheduledPing = new ScheduledPing();
- this.pingSchedule = PING_SCHEDULE.get(settings);
- this.namedWriteableRegistry = namedWriteableRegistry;
- this.circuitBreakerService = circuitBreakerService;
- }
-
- public Settings settings() {
- return this.settings;
- }
-
- @Override
- public void transportServiceAdapter(TransportServiceAdapter service) {
- this.transportServiceAdapter = service;
}
TransportServiceAdapter transportServiceAdapter() {
return transportServiceAdapter;
}
- ThreadPool threadPool() {
- return threadPool;
- }
-
- CircuitBreaker inFlightRequestsBreaker() {
- // We always obtain a fresh breaker to reflect changes to the breaker configuration.
- return circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS);
- }
-
@Override
protected void doStart() {
boolean success = false;
@@ -319,51 +170,17 @@ public class NettyTransport extends AbstractLifecycleComponent implem
if (NetworkService.NETWORK_SERVER.get(settings)) {
final OpenChannelsHandler openChannels = new OpenChannelsHandler(logger);
this.serverOpenChannels = openChannels;
-
- // extract default profile first and create standard bootstrap
- Map profiles = TransportSettings.TRANSPORT_PROFILES_SETTING.get(settings()).getAsGroups(true);
- if (!profiles.containsKey(TransportSettings.DEFAULT_PROFILE)) {
- profiles = new HashMap<>(profiles);
- profiles.put(TransportSettings.DEFAULT_PROFILE, Settings.EMPTY);
- }
-
- Settings fallbackSettings = createFallbackSettings();
- Settings defaultSettings = profiles.get(TransportSettings.DEFAULT_PROFILE);
-
// loop through all profiles and start them up, special handling for default one
- for (Map.Entry entry : profiles.entrySet()) {
- Settings profileSettings = entry.getValue();
- String name = entry.getKey();
-
- if (!Strings.hasLength(name)) {
- logger.info("transport profile configured without a name. skipping profile with settings [{}]",
- profileSettings.toDelimitedString(','));
- continue;
- } else if (TransportSettings.DEFAULT_PROFILE.equals(name)) {
- profileSettings = Settings.builder()
- .put(profileSettings)
- .put("port", profileSettings.get("port", TransportSettings.PORT.get(this.settings)))
- .build();
- } else if (profileSettings.get("port") == null) {
- // if profile does not have a port, skip it
- logger.info("No port configured for profile [{}], not binding", name);
- continue;
- }
-
+ for (Map.Entry entry : buildProfileSettings().entrySet()) {
// merge fallback settings with default settings with profile settings so we have complete settings with default values
- Settings mergedSettings = Settings.builder()
- .put(fallbackSettings)
- .put(defaultSettings)
- .put(profileSettings)
- .build();
-
- createServerBootstrap(name, mergedSettings);
- bindServerBootstrap(name, mergedSettings);
+ final Settings settings = Settings.builder()
+ .put(createFallbackSettings())
+ .put(entry.getValue()).build();
+ createServerBootstrap(entry.getKey(), settings);
+ bindServer(entry.getKey(), settings);
}
}
- if (pingSchedule.millis() > 0) {
- threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, scheduledPing);
- }
+ super.doStart();
success = true;
} finally {
if (success == false) {
@@ -372,24 +189,18 @@ public class NettyTransport extends AbstractLifecycleComponent implem
}
}
- @Override
- public Map profileBoundAddresses() {
- return unmodifiableMap(new HashMap<>(profileBoundAddresses));
- }
-
private ClientBootstrap createClientBootstrap() {
-
if (blockingClient) {
clientBootstrap = new ClientBootstrap(new OioClientSocketChannelFactory(
- Executors.newCachedThreadPool(daemonThreadFactory(settings, TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX))));
+ Executors.newCachedThreadPool(daemonThreadFactory(settings, TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX))));
} else {
int bossCount = NETTY_BOSS_COUNT.get(settings);
clientBootstrap = new ClientBootstrap(
- new NioClientSocketChannelFactory(
- Executors.newCachedThreadPool(daemonThreadFactory(settings, TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX)),
- bossCount,
- new NioWorkerPool(Executors.newCachedThreadPool(
- daemonThreadFactory(settings, TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX)), workerCount),
+ new NioClientSocketChannelFactory(
+ Executors.newCachedThreadPool(daemonThreadFactory(settings, TRANSPORT_CLIENT_BOSS_THREAD_NAME_PREFIX)),
+ bossCount,
+ new NioWorkerPool(Executors.newCachedThreadPool(
+ daemonThreadFactory(settings, TRANSPORT_CLIENT_WORKER_THREAD_NAME_PREFIX)), workerCount),
new HashedWheelTimer(daemonThreadFactory(settings, "transport_client_timer"))));
}
clientBootstrap.setPipelineFactory(configureClientChannelPipelineFactory());
@@ -436,19 +247,19 @@ public class NettyTransport extends AbstractLifecycleComponent implem
fallbackSettingsBuilder.put("tcp_no_delay", fallbackTcpNoDelay);
boolean fallbackTcpKeepAlive = settings.getAsBoolean("transport.netty.tcp_keep_alive", TcpSettings.TCP_KEEP_ALIVE.get(settings));
- fallbackSettingsBuilder.put("tcp_keep_alive", fallbackTcpKeepAlive);
+ fallbackSettingsBuilder.put("tcp_keep_alive", fallbackTcpKeepAlive);
boolean fallbackReuseAddress = settings.getAsBoolean("transport.netty.reuse_address", TcpSettings.TCP_REUSE_ADDRESS.get(settings));
fallbackSettingsBuilder.put("reuse_address", fallbackReuseAddress);
ByteSizeValue fallbackTcpSendBufferSize = settings.getAsBytesSize("transport.netty.tcp_send_buffer_size",
- TCP_SEND_BUFFER_SIZE.get(settings));
+ TCP_SEND_BUFFER_SIZE.get(settings));
if (fallbackTcpSendBufferSize.bytes() >= 0) {
fallbackSettingsBuilder.put("tcp_send_buffer_size", fallbackTcpSendBufferSize);
}
ByteSizeValue fallbackTcpBufferSize = settings.getAsBytesSize("transport.netty.tcp_receive_buffer_size",
- TCP_RECEIVE_BUFFER_SIZE.get(settings));
+ TCP_RECEIVE_BUFFER_SIZE.get(settings));
if (fallbackTcpBufferSize.bytes() >= 0) {
fallbackSettingsBuilder.put("tcp_receive_buffer_size", fallbackTcpBufferSize);
}
@@ -456,146 +267,6 @@ public class NettyTransport extends AbstractLifecycleComponent implem
return fallbackSettingsBuilder.build();
}
- private void bindServerBootstrap(final String name, final Settings settings) {
- // Bind and start to accept incoming connections.
- InetAddress hostAddresses[];
- String bindHosts[] = settings.getAsArray("bind_host", null);
- try {
- hostAddresses = networkService.resolveBindHostAddresses(bindHosts);
- } catch (IOException e) {
- throw new BindTransportException("Failed to resolve host " + Arrays.toString(bindHosts) + "", e);
- }
- if (logger.isDebugEnabled()) {
- String[] addresses = new String[hostAddresses.length];
- for (int i = 0; i < hostAddresses.length; i++) {
- addresses[i] = NetworkAddress.format(hostAddresses[i]);
- }
- logger.debug("binding server bootstrap to: {}", (Object)addresses);
- }
-
- assert hostAddresses.length > 0;
-
- List boundAddresses = new ArrayList<>();
- for (InetAddress hostAddress : hostAddresses) {
- boundAddresses.add(bindToPort(name, hostAddress, settings.get("port")));
- }
-
- final BoundTransportAddress boundTransportAddress = createBoundTransportAddress(name, settings, boundAddresses);
-
- if (TransportSettings.DEFAULT_PROFILE.equals(name)) {
- this.boundAddress = boundTransportAddress;
- } else {
- profileBoundAddresses.put(name, boundTransportAddress);
- }
- }
-
- private InetSocketAddress bindToPort(final String name, final InetAddress hostAddress, String port) {
- PortsRange portsRange = new PortsRange(port);
- final AtomicReference lastException = new AtomicReference<>();
- final AtomicReference boundSocket = new AtomicReference<>();
- boolean success = portsRange.iterate(new PortsRange.PortCallback() {
- @Override
- public boolean onPortNumber(int portNumber) {
- try {
- Channel channel = serverBootstraps.get(name).bind(new InetSocketAddress(hostAddress, portNumber));
- synchronized (serverChannels) {
- List list = serverChannels.get(name);
- if (list == null) {
- list = new ArrayList<>();
- serverChannels.put(name, list);
- }
- list.add(channel);
- boundSocket.set((InetSocketAddress) channel.getLocalAddress());
- }
- } catch (Exception e) {
- lastException.set(e);
- return false;
- }
- return true;
- }
- });
- if (!success) {
- throw new BindTransportException("Failed to bind to [" + port + "]", lastException.get());
- }
-
- if (logger.isDebugEnabled()) {
- logger.debug("Bound profile [{}] to address {{}}", name, NetworkAddress.format(boundSocket.get()));
- }
-
- return boundSocket.get();
- }
-
- private BoundTransportAddress createBoundTransportAddress(String name, Settings profileSettings,
- List boundAddresses) {
- String[] boundAddressesHostStrings = new String[boundAddresses.size()];
- TransportAddress[] transportBoundAddresses = new TransportAddress[boundAddresses.size()];
- for (int i = 0; i < boundAddresses.size(); i++) {
- InetSocketAddress boundAddress = boundAddresses.get(i);
- boundAddressesHostStrings[i] = boundAddress.getHostString();
- transportBoundAddresses[i] = new InetSocketTransportAddress(boundAddress);
- }
-
- final String[] publishHosts;
- if (TransportSettings.DEFAULT_PROFILE.equals(name)) {
- publishHosts = TransportSettings.PUBLISH_HOST.get(settings).toArray(Strings.EMPTY_ARRAY);
- } else {
- publishHosts = profileSettings.getAsArray("publish_host", boundAddressesHostStrings);
- }
-
- final InetAddress publishInetAddress;
- try {
- publishInetAddress = networkService.resolvePublishHostAddresses(publishHosts);
- } catch (Exception e) {
- throw new BindTransportException("Failed to resolve publish address", e);
- }
-
- final int publishPort = resolvePublishPort(name, settings, profileSettings, boundAddresses, publishInetAddress);
- final TransportAddress publishAddress = new InetSocketTransportAddress(new InetSocketAddress(publishInetAddress, publishPort));
- return new BoundTransportAddress(transportBoundAddresses, publishAddress);
- }
-
- // package private for tests
- static int resolvePublishPort(String profileName, Settings settings, Settings profileSettings, List boundAddresses,
- InetAddress publishInetAddress) {
- int publishPort;
- if (TransportSettings.DEFAULT_PROFILE.equals(profileName)) {
- publishPort = TransportSettings.PUBLISH_PORT.get(settings);
- } else {
- publishPort = profileSettings.getAsInt("publish_port", -1);
- }
-
- // if port not explicitly provided, search for port of address in boundAddresses that matches publishInetAddress
- if (publishPort < 0) {
- for (InetSocketAddress boundAddress : boundAddresses) {
- InetAddress boundInetAddress = boundAddress.getAddress();
- if (boundInetAddress.isAnyLocalAddress() || boundInetAddress.equals(publishInetAddress)) {
- publishPort = boundAddress.getPort();
- break;
- }
- }
- }
-
- // if no matching boundAddress found, check if there is a unique port for all bound addresses
- if (publishPort < 0) {
- final IntSet ports = new IntHashSet();
- for (InetSocketAddress boundAddress : boundAddresses) {
- ports.add(boundAddress.getPort());
- }
- if (ports.size() == 1) {
- publishPort = ports.iterator().next().value;
- }
- }
-
- if (publishPort < 0) {
- String profileExplanation = TransportSettings.DEFAULT_PROFILE.equals(profileName) ? "" : " for profile " + profileName;
- throw new BindTransportException("Failed to auto-resolve publish port" + profileExplanation + ", multiple bound addresses " +
- boundAddresses + " with distinct ports and none of them matched the publish address (" + publishInetAddress + "). " +
- "Please specify a unique port by setting " + TransportSettings.PORT.getKey() + " or " +
- TransportSettings.PUBLISH_PORT.getKey());
- }
- return publishPort;
- }
-
private void createServerBootstrap(String name, Settings settings) {
boolean blockingServer = TCP_BLOCKING_SERVER.get(settings);
String port = settings.get("port");
@@ -609,10 +280,10 @@ public class NettyTransport extends AbstractLifecycleComponent implem
if (logger.isDebugEnabled()) {
logger.debug("using profile[{}], worker_count[{}], port[{}], bind_host[{}], publish_host[{}], compress[{}], "
- + "connect_timeout[{}], connections_per_node[{}/{}/{}/{}/{}], receive_predictor[{}->{}]",
- name, workerCount, port, bindHost, publishHost, compress, connectTimeout, connectionsPerNodeRecovery,
- connectionsPerNodeBulk, connectionsPerNodeReg, connectionsPerNodeState, connectionsPerNodePing, receivePredictorMin,
- receivePredictorMax);
+ + "connect_timeout[{}], connections_per_node[{}/{}/{}/{}/{}], receive_predictor[{}->{}]",
+ name, workerCount, port, bindHost, publishHost, compress, connectTimeout, connectionsPerNodeRecovery,
+ connectionsPerNodeBulk, connectionsPerNodeReg, connectionsPerNodeState, connectionsPerNodePing, receivePredictorMin,
+ receivePredictorMax);
}
final ThreadFactory bossFactory = daemonThreadFactory(this.settings, HTTP_SERVER_BOSS_THREAD_NAME_PREFIX, name);
@@ -620,14 +291,14 @@ public class NettyTransport extends AbstractLifecycleComponent implem
ServerBootstrap serverBootstrap;
if (blockingServer) {
serverBootstrap = new ServerBootstrap(new OioServerSocketChannelFactory(
- Executors.newCachedThreadPool(bossFactory),
- Executors.newCachedThreadPool(workerFactory)
+ Executors.newCachedThreadPool(bossFactory),
+ Executors.newCachedThreadPool(workerFactory)
));
} else {
serverBootstrap = new ServerBootstrap(new NioServerSocketChannelFactory(
- Executors.newCachedThreadPool(bossFactory),
- Executors.newCachedThreadPool(workerFactory),
- workerCount));
+ Executors.newCachedThreadPool(bossFactory),
+ Executors.newCachedThreadPool(workerFactory),
+ workerCount));
}
serverBootstrap.setPipelineFactory(configureServerChannelPipelineFactory(name, settings));
if (!"default".equals(tcpNoDelay)) {
@@ -649,202 +320,8 @@ public class NettyTransport extends AbstractLifecycleComponent implem
serverBootstraps.put(name, serverBootstrap);
}
- @Override
- protected void doStop() {
- final CountDownLatch latch = new CountDownLatch(1);
- // make sure we run it on another thread than a possible IO handler thread
- threadPool.generic().execute(new Runnable() {
- @Override
- public void run() {
- globalLock.writeLock().lock();
- try {
- for (Iterator it = connectedNodes.values().iterator(); it.hasNext(); ) {
- NodeChannels nodeChannels = it.next();
- it.remove();
- nodeChannels.close();
- }
-
- Iterator>> serverChannelIterator = serverChannels.entrySet().iterator();
- while (serverChannelIterator.hasNext()) {
- Map.Entry> serverChannelEntry = serverChannelIterator.next();
- String name = serverChannelEntry.getKey();
- List serverChannels = serverChannelEntry.getValue();
- for (Channel serverChannel : serverChannels) {
- try {
- serverChannel.close().awaitUninterruptibly();
- } catch (Throwable t) {
- logger.debug("Error closing serverChannel for profile [{}]", t, name);
- }
- }
- serverChannelIterator.remove();
- }
-
- if (serverOpenChannels != null) {
- serverOpenChannels.close();
- serverOpenChannels = null;
- }
-
- Iterator> serverBootstrapIterator = serverBootstraps.entrySet().iterator();
- while (serverBootstrapIterator.hasNext()) {
- Map.Entry serverBootstrapEntry = serverBootstrapIterator.next();
- String name = serverBootstrapEntry.getKey();
- ServerBootstrap serverBootstrap = serverBootstrapEntry.getValue();
-
- try {
- serverBootstrap.releaseExternalResources();
- } catch (Throwable t) {
- logger.debug("Error closing serverBootstrap for profile [{}]", t, name);
- }
-
- serverBootstrapIterator.remove();
- }
-
- for (Iterator it = connectedNodes.values().iterator(); it.hasNext(); ) {
- NodeChannels nodeChannels = it.next();
- it.remove();
- nodeChannels.close();
- }
-
- if (clientBootstrap != null) {
- clientBootstrap.releaseExternalResources();
- clientBootstrap = null;
- }
- } finally {
- globalLock.writeLock().unlock();
- latch.countDown();
- }
- }
- });
-
- try {
- latch.await(30, TimeUnit.SECONDS);
- } catch (InterruptedException e) {
- // ignore
- }
- }
-
- @Override
- protected void doClose() {
- }
-
- @Override
- public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws Exception {
- return parse(address, settings.get("transport.profiles.default.port", TransportSettings.PORT.get(settings)), perAddressLimit);
- }
-
- // this code is a take on guava's HostAndPort, like a HostAndPortRange
-
- // pattern for validating ipv6 bracket addresses.
- // not perfect, but PortsRange should take care of any port range validation, not a regex
- private static final Pattern BRACKET_PATTERN = Pattern.compile("^\\[(.*:.*)\\](?::([\\d\\-]*))?$");
-
- /** parse a hostname+port range spec into its equivalent addresses */
- static TransportAddress[] parse(String hostPortString, String defaultPortRange, int perAddressLimit) throws UnknownHostException {
- Objects.requireNonNull(hostPortString);
- String host;
- String portString = null;
-
- if (hostPortString.startsWith("[")) {
- // Parse a bracketed host, typically an IPv6 literal.
- Matcher matcher = BRACKET_PATTERN.matcher(hostPortString);
- if (!matcher.matches()) {
- throw new IllegalArgumentException("Invalid bracketed host/port range: " + hostPortString);
- }
- host = matcher.group(1);
- portString = matcher.group(2); // could be null
- } else {
- int colonPos = hostPortString.indexOf(':');
- if (colonPos >= 0 && hostPortString.indexOf(':', colonPos + 1) == -1) {
- // Exactly 1 colon. Split into host:port.
- host = hostPortString.substring(0, colonPos);
- portString = hostPortString.substring(colonPos + 1);
- } else {
- // 0 or 2+ colons. Bare hostname or IPv6 literal.
- host = hostPortString;
- // 2+ colons and not bracketed: exception
- if (colonPos >= 0) {
- throw new IllegalArgumentException("IPv6 addresses must be bracketed: " + hostPortString);
- }
- }
- }
-
- // if port isn't specified, fill with the default
- if (portString == null || portString.isEmpty()) {
- portString = defaultPortRange;
- }
-
- // generate address for each port in the range
- Set addresses = new HashSet<>(Arrays.asList(InetAddress.getAllByName(host)));
- List transportAddresses = new ArrayList<>();
- int[] ports = new PortsRange(portString).ports();
- int limit = Math.min(ports.length, perAddressLimit);
- for (int i = 0; i < limit; i++) {
- for (InetAddress address : addresses) {
- transportAddresses.add(new InetSocketTransportAddress(address, ports[i]));
- }
- }
- return transportAddresses.toArray(new TransportAddress[transportAddresses.size()]);
- }
-
- @Override
- public boolean addressSupported(Class extends TransportAddress> address) {
- return InetSocketTransportAddress.class.equals(address);
- }
-
- @Override
- public BoundTransportAddress boundAddress() {
- return this.boundAddress;
- }
-
- protected void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
- if (!lifecycle.started()) {
- // ignore
- return;
- }
- if (isCloseConnectionException(e.getCause())) {
- logger.trace("close connection exception caught on transport layer [{}], disconnecting from relevant node", e.getCause(),
- ctx.getChannel());
- // close the channel, which will cause a node to be disconnected if relevant
- ctx.getChannel().close();
- disconnectFromNodeChannel(ctx.getChannel(), e.getCause());
- } else if (isConnectException(e.getCause())) {
- logger.trace("connect exception caught on transport layer [{}]", e.getCause(), ctx.getChannel());
- // close the channel as safe measure, which will cause a node to be disconnected if relevant
- ctx.getChannel().close();
- disconnectFromNodeChannel(ctx.getChannel(), e.getCause());
- } else if (e.getCause() instanceof BindException) {
- logger.trace("bind exception caught on transport layer [{}]", e.getCause(), ctx.getChannel());
- // close the channel as safe measure, which will cause a node to be disconnected if relevant
- ctx.getChannel().close();
- disconnectFromNodeChannel(ctx.getChannel(), e.getCause());
- } else if (e.getCause() instanceof CancelledKeyException) {
- logger.trace("cancelled key exception caught on transport layer [{}], disconnecting from relevant node", e.getCause(),
- ctx.getChannel());
- // close the channel as safe measure, which will cause a node to be disconnected if relevant
- ctx.getChannel().close();
- disconnectFromNodeChannel(ctx.getChannel(), e.getCause());
- } else if (e.getCause() instanceof SizeHeaderFrameDecoder.HttpOnTransportException) {
- // in case we are able to return data, serialize the exception content and sent it back to the client
- if (ctx.getChannel().isOpen()) {
- ChannelBuffer buffer = ChannelBuffers.wrappedBuffer(e.getCause().getMessage().getBytes(StandardCharsets.UTF_8));
- ChannelFuture channelFuture = ctx.getChannel().write(buffer);
- channelFuture.addListener(new ChannelFutureListener() {
- @Override
- public void operationComplete(ChannelFuture future) throws Exception {
- future.getChannel().close();
- }
- });
- }
- } else {
- logger.warn("exception caught on transport layer [{}], closing connection", e.getCause(), ctx.getChannel());
- // close the channel, which will cause a node to be disconnected if relevant
- ctx.getChannel().close();
- disconnectFromNodeChannel(ctx.getChannel(), e.getCause());
- }
- }
-
- TransportAddress wrapAddress(SocketAddress socketAddress) {
- return new InetSocketTransportAddress((InetSocketAddress) socketAddress);
+ protected final void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
+ onException(ctx.getChannel(), e.getCause());
}
@Override
@@ -853,153 +330,6 @@ public class NettyTransport extends AbstractLifecycleComponent implem
return channels == null ? 0 : channels.numberOfOpenChannels();
}
- @Override
- public List getLocalAddresses() {
- List local = new ArrayList<>();
- local.add("127.0.0.1");
- // check if v6 is supported, if so, v4 will also work via mapped addresses.
- if (NetworkUtils.SUPPORTS_V6) {
- local.add("[::1]"); // may get ports appended!
- }
- return local;
- }
-
- @Override
- public void sendRequest(final DiscoveryNode node, final long requestId, final String action, final TransportRequest request,
- TransportRequestOptions options) throws IOException, TransportException {
-
- Channel targetChannel = nodeChannel(node, options);
-
- if (compress) {
- options = TransportRequestOptions.builder(options).withCompress(true).build();
- }
-
- byte status = 0;
- status = TransportStatus.setRequest(status);
-
- ReleasableBytesStreamOutput bStream = new ReleasableBytesStreamOutput(bigArrays);
- boolean addedReleaseListener = false;
- try {
- bStream.skip(NettyHeader.HEADER_SIZE);
- StreamOutput stream = bStream;
- // only compress if asked, and, the request is not bytes, since then only
- // the header part is compressed, and the "body" can't be extracted as compressed
- if (options.compress() && (!(request instanceof BytesTransportRequest))) {
- status = TransportStatus.setCompress(status);
- stream = CompressorFactory.COMPRESSOR.streamOutput(stream);
- }
-
- // 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.smallest(getCurrentVersion(), node.getVersion());
-
- stream.setVersion(version);
- threadPool.getThreadContext().writeTo(stream);
- stream.writeString(action);
-
- ReleasablePagedBytesReference bytes;
- ChannelBuffer buffer;
- // it might be nice to somehow generalize this optimization, maybe a smart "paged" bytes output
- // that create paged channel buffers, but its tricky to know when to do it (where this option is
- // more explicit).
- if (request instanceof BytesTransportRequest) {
- BytesTransportRequest bRequest = (BytesTransportRequest) request;
- assert node.getVersion().equals(bRequest.version());
- bRequest.writeThin(stream);
- stream.close();
- bytes = bStream.bytes();
- ChannelBuffer headerBuffer = NettyUtils.toChannelBuffer(bytes);
- ChannelBuffer contentBuffer = NettyUtils.toChannelBuffer(bRequest.bytes());
- buffer = ChannelBuffers.wrappedBuffer(NettyUtils.DEFAULT_GATHERING, headerBuffer, contentBuffer);
- } else {
- request.writeTo(stream);
- stream.close();
- bytes = bStream.bytes();
- buffer = NettyUtils.toChannelBuffer(bytes);
- }
- NettyHeader.writeHeader(buffer, requestId, status, version);
- ChannelFuture future = targetChannel.write(buffer);
- ReleaseChannelFutureListener listener = new ReleaseChannelFutureListener(bytes);
- future.addListener(listener);
- addedReleaseListener = true;
- final TransportRequestOptions finalOptions = options;
- ChannelFutureListener channelFutureListener =
- f -> transportServiceAdapter.onRequestSent(node, requestId, action, request, finalOptions);
- future.addListener(channelFutureListener);
- } finally {
- if (!addedReleaseListener) {
- Releasables.close(bStream.bytes());
- }
- }
- }
-
- @Override
- public boolean nodeConnected(DiscoveryNode node) {
- return connectedNodes.containsKey(node);
- }
-
- @Override
- public void connectToNodeLight(DiscoveryNode node) throws ConnectTransportException {
- connectToNode(node, true);
- }
-
- @Override
- public void connectToNode(DiscoveryNode node) {
- connectToNode(node, false);
- }
-
- public void connectToNode(DiscoveryNode node, boolean light) {
- if (!lifecycle.started()) {
- throw new IllegalStateException("can't add nodes to a stopped transport");
- }
- if (node == null) {
- throw new ConnectTransportException(null, "can't connect to a null node");
- }
- globalLock.readLock().lock();
- try {
-
- try (Releasable ignored = connectionLock.acquire(node.getId())) {
- if (!lifecycle.started()) {
- throw new IllegalStateException("can't add nodes to a stopped transport");
- }
- NodeChannels nodeChannels = connectedNodes.get(node);
- if (nodeChannels != null) {
- return;
- }
- try {
- if (light) {
- nodeChannels = connectToChannelsLight(node);
- } else {
- nodeChannels = new NodeChannels(new Channel[connectionsPerNodeRecovery], new Channel[connectionsPerNodeBulk],
- new Channel[connectionsPerNodeReg], new Channel[connectionsPerNodeState],
- new Channel[connectionsPerNodePing]);
- try {
- connectToChannels(nodeChannels, node);
- } catch (Throwable e) {
- logger.trace("failed to connect to [{}], cleaning dangling connections", e, node);
- nodeChannels.close();
- throw e;
- }
- }
- // we acquire a connection lock, so no way there is an existing connection
- nodeChannels.start();
- connectedNodes.put(node, nodeChannels);
- if (logger.isDebugEnabled()) {
- logger.debug("connected to node [{}]", node);
- }
- transportServiceAdapter.raiseNodeConnected(node);
- } catch (ConnectTransportException e) {
- throw e;
- } catch (Exception e) {
- throw new ConnectTransportException(node, "general node connection failure", e);
- }
- }
- } finally {
- globalLock.readLock().unlock();
- }
- }
-
protected NodeChannels connectToChannelsLight(DiscoveryNode node) {
InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address();
ChannelFuture connect = clientBootstrap.connect(address);
@@ -1013,176 +343,117 @@ public class NettyTransport extends AbstractLifecycleComponent implem
return new NodeChannels(channels, channels, channels, channels, channels);
}
- protected void connectToChannels(NodeChannels nodeChannels, DiscoveryNode node) {
- ChannelFuture[] connectRecovery = new ChannelFuture[nodeChannels.recovery.length];
- ChannelFuture[] connectBulk = new ChannelFuture[nodeChannels.bulk.length];
- ChannelFuture[] connectReg = new ChannelFuture[nodeChannels.reg.length];
- ChannelFuture[] connectState = new ChannelFuture[nodeChannels.state.length];
- ChannelFuture[] connectPing = new ChannelFuture[nodeChannels.ping.length];
- InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address();
- for (int i = 0; i < connectRecovery.length; i++) {
- connectRecovery[i] = clientBootstrap.connect(address);
- }
- for (int i = 0; i < connectBulk.length; i++) {
- connectBulk[i] = clientBootstrap.connect(address);
- }
- for (int i = 0; i < connectReg.length; i++) {
- connectReg[i] = clientBootstrap.connect(address);
- }
- for (int i = 0; i < connectState.length; i++) {
- connectState[i] = clientBootstrap.connect(address);
- }
- for (int i = 0; i < connectPing.length; i++) {
- connectPing[i] = clientBootstrap.connect(address);
- }
-
+ protected NodeChannels connectToChannels(DiscoveryNode node) {
+ final NodeChannels nodeChannels = new NodeChannels(new Channel[connectionsPerNodeRecovery], new Channel[connectionsPerNodeBulk],
+ new Channel[connectionsPerNodeReg], new Channel[connectionsPerNodeState],
+ new Channel[connectionsPerNodePing]);
+ boolean success = false;
try {
+ ChannelFuture[] connectRecovery = new ChannelFuture[nodeChannels.recovery.length];
+ ChannelFuture[] connectBulk = new ChannelFuture[nodeChannels.bulk.length];
+ ChannelFuture[] connectReg = new ChannelFuture[nodeChannels.reg.length];
+ ChannelFuture[] connectState = new ChannelFuture[nodeChannels.state.length];
+ ChannelFuture[] connectPing = new ChannelFuture[nodeChannels.ping.length];
+ InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address();
for (int i = 0; i < connectRecovery.length; i++) {
- connectRecovery[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
- if (!connectRecovery[i].isSuccess()) {
- throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectRecovery[i].getCause());
- }
- nodeChannels.recovery[i] = connectRecovery[i].getChannel();
- nodeChannels.recovery[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ connectRecovery[i] = clientBootstrap.connect(address);
}
-
for (int i = 0; i < connectBulk.length; i++) {
- connectBulk[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
- if (!connectBulk[i].isSuccess()) {
- throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectBulk[i].getCause());
- }
- nodeChannels.bulk[i] = connectBulk[i].getChannel();
- nodeChannels.bulk[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ connectBulk[i] = clientBootstrap.connect(address);
}
-
for (int i = 0; i < connectReg.length; i++) {
- connectReg[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
- if (!connectReg[i].isSuccess()) {
- throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectReg[i].getCause());
- }
- nodeChannels.reg[i] = connectReg[i].getChannel();
- nodeChannels.reg[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ connectReg[i] = clientBootstrap.connect(address);
}
-
for (int i = 0; i < connectState.length; i++) {
- connectState[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
- if (!connectState[i].isSuccess()) {
- throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectState[i].getCause());
- }
- nodeChannels.state[i] = connectState[i].getChannel();
- nodeChannels.state[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ connectState[i] = clientBootstrap.connect(address);
}
-
for (int i = 0; i < connectPing.length; i++) {
- connectPing[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
- if (!connectPing[i].isSuccess()) {
- throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectPing[i].getCause());
- }
- nodeChannels.ping[i] = connectPing[i].getChannel();
- nodeChannels.ping[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ connectPing[i] = clientBootstrap.connect(address);
}
- if (nodeChannels.recovery.length == 0) {
- if (nodeChannels.bulk.length > 0) {
- nodeChannels.recovery = nodeChannels.bulk;
- } else {
- nodeChannels.recovery = nodeChannels.reg;
+ try {
+ for (int i = 0; i < connectRecovery.length; i++) {
+ connectRecovery[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
+ if (!connectRecovery[i].isSuccess()) {
+ throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectRecovery[i].getCause());
+ }
+ nodeChannels.recovery[i] = connectRecovery[i].getChannel();
+ nodeChannels.recovery[i].getCloseFuture().addListener(new ChannelCloseListener(node));
}
- }
- if (nodeChannels.bulk.length == 0) {
- nodeChannels.bulk = nodeChannels.reg;
- }
- } catch (RuntimeException e) {
- // clean the futures
- List futures = new ArrayList<>();
- futures.addAll(Arrays.asList(connectRecovery));
- futures.addAll(Arrays.asList(connectBulk));
- futures.addAll(Arrays.asList(connectReg));
- futures.addAll(Arrays.asList(connectState));
- futures.addAll(Arrays.asList(connectPing));
- for (ChannelFuture future : Collections.unmodifiableList(futures)) {
- future.cancel();
- if (future.getChannel() != null && future.getChannel().isOpen()) {
- try {
- future.getChannel().close();
- } catch (Exception e1) {
- // ignore
+
+ for (int i = 0; i < connectBulk.length; i++) {
+ connectBulk[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
+ if (!connectBulk[i].isSuccess()) {
+ throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectBulk[i].getCause());
+ }
+ nodeChannels.bulk[i] = connectBulk[i].getChannel();
+ nodeChannels.bulk[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ }
+
+ for (int i = 0; i < connectReg.length; i++) {
+ connectReg[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
+ if (!connectReg[i].isSuccess()) {
+ throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectReg[i].getCause());
+ }
+ nodeChannels.reg[i] = connectReg[i].getChannel();
+ nodeChannels.reg[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ }
+
+ for (int i = 0; i < connectState.length; i++) {
+ connectState[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
+ if (!connectState[i].isSuccess()) {
+ throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectState[i].getCause());
+ }
+ nodeChannels.state[i] = connectState[i].getChannel();
+ nodeChannels.state[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ }
+
+ for (int i = 0; i < connectPing.length; i++) {
+ connectPing[i].awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
+ if (!connectPing[i].isSuccess()) {
+ throw new ConnectTransportException(node, "connect_timeout[" + connectTimeout + "]", connectPing[i].getCause());
+ }
+ nodeChannels.ping[i] = connectPing[i].getChannel();
+ nodeChannels.ping[i].getCloseFuture().addListener(new ChannelCloseListener(node));
+ }
+
+ if (nodeChannels.recovery.length == 0) {
+ if (nodeChannels.bulk.length > 0) {
+ nodeChannels.recovery = nodeChannels.bulk;
+ } else {
+ nodeChannels.recovery = nodeChannels.reg;
}
}
- }
- throw e;
- }
- }
-
- @Override
- public void disconnectFromNode(DiscoveryNode node) {
-
- try (Releasable ignored = connectionLock.acquire(node.getId())) {
- NodeChannels nodeChannels = connectedNodes.remove(node);
- if (nodeChannels != null) {
- try {
- logger.debug("disconnecting from [{}] due to explicit disconnect call", node);
- nodeChannels.close();
- } finally {
- logger.trace("disconnected from [{}] due to explicit disconnect call", node);
- transportServiceAdapter.raiseNodeDisconnected(node);
+ if (nodeChannels.bulk.length == 0) {
+ nodeChannels.bulk = nodeChannels.reg;
}
- }
- }
- }
-
- /**
- * Disconnects from a node, only if the relevant channel is found to be part of the node channels.
- */
- protected boolean disconnectFromNode(DiscoveryNode node, Channel channel, String reason) {
- // this might be called multiple times from all the node channels, so do a lightweight
- // check outside of the lock
- NodeChannels nodeChannels = connectedNodes.get(node);
- if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
- try (Releasable ignored = connectionLock.acquire(node.getId())) {
- nodeChannels = connectedNodes.get(node);
- // check again within the connection lock, if its still applicable to remove it
- if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
- connectedNodes.remove(node);
- try {
- logger.debug("disconnecting from [{}], {}", node, reason);
- nodeChannels.close();
- } finally {
- logger.trace("disconnected from [{}], {}", node, reason);
- transportServiceAdapter.raiseNodeDisconnected(node);
- }
- return true;
- }
- }
- }
- return false;
- }
-
- /**
- * Disconnects from a node if a channel is found as part of that nodes channels.
- */
- protected void disconnectFromNodeChannel(final Channel channel, final Throwable failure) {
- threadPool().generic().execute(new Runnable() {
-
- @Override
- public void run() {
- for (DiscoveryNode node : connectedNodes.keySet()) {
- if (disconnectFromNode(node, channel, ExceptionsHelper.detailedMessage(failure))) {
- // if we managed to find this channel and disconnect from it, then break, no need to check on
- // the rest of the nodes
- break;
+ } catch (RuntimeException e) {
+ // clean the futures
+ List futures = new ArrayList<>();
+ futures.addAll(Arrays.asList(connectRecovery));
+ futures.addAll(Arrays.asList(connectBulk));
+ futures.addAll(Arrays.asList(connectReg));
+ futures.addAll(Arrays.asList(connectState));
+ futures.addAll(Arrays.asList(connectPing));
+ for (ChannelFuture future : Collections.unmodifiableList(futures)) {
+ future.cancel();
+ if (future.getChannel() != null && future.getChannel().isOpen()) {
+ try {
+ future.getChannel().close();
+ } catch (Exception e1) {
+ // ignore
+ }
}
}
+ throw e;
+ }
+ success = true;
+ } finally {
+ if (success == false) {
+ nodeChannels.close();
}
- });
- }
-
- protected Channel nodeChannel(DiscoveryNode node, TransportRequestOptions options) throws ConnectTransportException {
- NodeChannels nodeChannels = connectedNodes.get(node);
- if (nodeChannels == null) {
- throw new NodeNotConnectedException(node, "Node not connected");
}
- return nodeChannels.channel(options.type());
+ return nodeChannels;
}
public ChannelPipelineFactory configureClientChannelPipelineFactory() {
@@ -1212,7 +483,7 @@ public class NettyTransport extends AbstractLifecycleComponent implem
}
channelPipeline.addLast("size", sizeHeader);
// using a dot as a prefix means, this cannot come from any settings parsed
- channelPipeline.addLast("dispatcher", new MessageChannelHandler(nettyTransport, nettyTransport.logger, ".client"));
+ channelPipeline.addLast("dispatcher", new NettyMessageChannelHandler(nettyTransport, ".client"));
return channelPipeline;
}
}
@@ -1249,7 +520,7 @@ public class NettyTransport extends AbstractLifecycleComponent implem
sizeHeader.setMaxCumulationBufferComponents(nettyTransport.maxCompositeBufferComponents);
}
channelPipeline.addLast("size", sizeHeader);
- channelPipeline.addLast("dispatcher", new MessageChannelHandler(nettyTransport, nettyTransport.logger, name));
+ channelPipeline.addLast("dispatcher", new NettyMessageChannelHandler(nettyTransport, name));
return channelPipeline;
}
}
@@ -1266,142 +537,169 @@ public class NettyTransport extends AbstractLifecycleComponent implem
public void operationComplete(final ChannelFuture future) throws Exception {
NodeChannels nodeChannels = connectedNodes.get(node);
if (nodeChannels != null && nodeChannels.hasChannel(future.getChannel())) {
- threadPool().generic().execute(new Runnable() {
- @Override
- public void run() {
- disconnectFromNode(node, future.getChannel(), "channel closed event");
- }
- });
+ threadPool.generic().execute(() -> disconnectFromNode(node, future.getChannel(), "channel closed event"));
}
}
}
- public static class NodeChannels {
-
- List allChannels = Collections.emptyList();
- private Channel[] recovery;
- private final AtomicInteger recoveryCounter = new AtomicInteger();
- private Channel[] bulk;
- private final AtomicInteger bulkCounter = new AtomicInteger();
- private Channel[] reg;
- private final AtomicInteger regCounter = new AtomicInteger();
- private Channel[] state;
- private final AtomicInteger stateCounter = new AtomicInteger();
- private Channel[] ping;
- private final AtomicInteger pingCounter = new AtomicInteger();
-
- public NodeChannels(Channel[] recovery, Channel[] bulk, Channel[] reg, Channel[] state, Channel[] ping) {
- this.recovery = recovery;
- this.bulk = bulk;
- this.reg = reg;
- this.state = state;
- this.ping = ping;
- }
-
- public void start() {
- List newAllChannels = new ArrayList<>();
- newAllChannels.addAll(Arrays.asList(recovery));
- newAllChannels.addAll(Arrays.asList(bulk));
- newAllChannels.addAll(Arrays.asList(reg));
- newAllChannels.addAll(Arrays.asList(state));
- newAllChannels.addAll(Arrays.asList(ping));
- this.allChannels = Collections.unmodifiableList(newAllChannels);
- }
-
- public boolean hasChannel(Channel channel) {
- for (Channel channel1 : allChannels) {
- if (channel.equals(channel1)) {
- return true;
- }
- }
- return false;
- }
-
- public Channel channel(TransportRequestOptions.Type type) {
- if (type == TransportRequestOptions.Type.REG) {
- return reg[Math.floorMod(regCounter.incrementAndGet(), reg.length)];
- } else if (type == TransportRequestOptions.Type.STATE) {
- return state[Math.floorMod(stateCounter.incrementAndGet(), state.length)];
- } else if (type == TransportRequestOptions.Type.PING) {
- return ping[Math.floorMod(pingCounter.incrementAndGet(), ping.length)];
- } else if (type == TransportRequestOptions.Type.BULK) {
- return bulk[Math.floorMod(bulkCounter.incrementAndGet(), bulk.length)];
- } else if (type == TransportRequestOptions.Type.RECOVERY) {
- return recovery[Math.floorMod(recoveryCounter.incrementAndGet(), recovery.length)];
- } else {
- throw new IllegalArgumentException("no type channel for [" + type + "]");
- }
- }
-
- public synchronized void close() {
- List futures = new ArrayList<>();
- for (Channel channel : allChannels) {
+ protected void sendMessage(Channel channel, BytesReference reference, Runnable sendListener, boolean close) {
+ final ChannelFuture future = channel.write(NettyUtils.toChannelBuffer(reference));
+ if (close) {
+ future.addListener(f -> {
try {
- if (channel != null && channel.isOpen()) {
- futures.add(channel.close());
- }
- } catch (Exception e) {
- //ignore
+ sendListener.run();
+ } finally {
+ f.getChannel().close();
}
- }
- for (ChannelFuture future : futures) {
- future.awaitUninterruptibly();
- }
+ });
+ } else {
+ future.addListener(future1 -> sendListener.run());
}
}
- class ScheduledPing extends AbstractLifecycleRunnable {
+ @Override
+ protected void closeChannels(List channels) {
+ List futures = new ArrayList<>();
- final CounterMetric successfulPings = new CounterMetric();
- final CounterMetric failedPings = new CounterMetric();
-
- public ScheduledPing() {
- super(lifecycle, logger);
- }
-
- @Override
- protected void doRunInLifecycle() throws Exception {
- for (Map.Entry entry : connectedNodes.entrySet()) {
- DiscoveryNode node = entry.getKey();
- NodeChannels channels = entry.getValue();
- for (Channel channel : channels.allChannels) {
- try {
- ChannelFuture future = channel.write(NettyHeader.pingHeader());
- future.addListener(new ChannelFutureListener() {
- @Override
- public void operationComplete(ChannelFuture future) throws Exception {
- successfulPings.inc();
- }
- });
- } catch (Throwable t) {
- if (channel.isOpen()) {
- logger.debug("[{}] failed to send ping transport message", t, node);
- failedPings.inc();
- } else {
- logger.trace("[{}] failed to send ping transport message (channel closed)", t, node);
- }
- }
+ for (Channel channel : channels) {
+ try {
+ if (channel != null && channel.isOpen()) {
+ futures.add(channel.close());
}
+ } catch (Exception e) {
+ logger.trace("failed to close channel", e);
}
}
-
- @Override
- protected void onAfterInLifecycle() {
- threadPool.schedule(pingSchedule, ThreadPool.Names.GENERIC, this);
+ for (ChannelFuture future : futures) {
+ future.awaitUninterruptibly();
}
+ }
- @Override
- public void onFailure(Throwable t) {
- if (lifecycle.stoppedOrClosed()) {
- logger.trace("failed to send ping transport message", t);
+ @Override
+ protected InetSocketAddress getLocalAddress(Channel channel) {
+ return (InetSocketAddress) channel.getLocalAddress();
+ }
+
+ @Override
+ protected Channel bind(String name, InetSocketAddress address) {
+ return serverBootstraps.get(name).bind(address);
+ }
+
+ ScheduledPing getPing() {
+ return scheduledPing;
+ }
+
+ @Override
+ protected boolean isOpen(Channel channel) {
+ return channel.isOpen();
+ }
+
+ @Override
+ protected void stopInternal() {
+ Releasables.close(serverOpenChannels, () ->{
+ for (Map.Entry entry : serverBootstraps.entrySet()) {
+ String name = entry.getKey();
+ ServerBootstrap serverBootstrap = entry.getValue();
+ try {
+ serverBootstrap.releaseExternalResources();
+ } catch (Throwable t) {
+ logger.debug("Error closing serverBootstrap for profile [{}]", t, name);
+ }
+ }
+ serverBootstraps.clear();
+ if (clientBootstrap != null) {
+ clientBootstrap.releaseExternalResources();
+ clientBootstrap = null;
+ }
+ });
+ }
+
+ @Override
+ public Message prepareSend(Version nodeVersion, TransportMessage message, StreamOutput stream,
+ ReleasableBytesStream writtenBytes) throws IOException {
+ // it might be nice to somehow generalize this optimization, maybe a smart "paged" bytes output
+ // that create paged channel buffers, but its tricky to know when to do it (where this option is
+ // more explicit).
+ if (message instanceof BytesTransportRequest) {
+ BytesTransportRequest bRequest = (BytesTransportRequest) message;
+ assert nodeVersion.equals(bRequest.version());
+ bRequest.writeThin(stream);
+ stream.close();
+ ReleasablePagedBytesReference bytes = writtenBytes.bytes();
+ ChannelBuffer headerBuffer = NettyUtils.toChannelBuffer(bytes);
+ ChannelBuffer contentBuffer = NettyUtils.toChannelBuffer(bRequest.bytes());
+ ChannelBuffer buffer = ChannelBuffers.wrappedBuffer(NettyUtils.DEFAULT_GATHERING, headerBuffer, contentBuffer);
+ return new NettyMessage(buffer);
} else {
- logger.warn("failed to send ping transport message", t);
+ return super.prepareSend(nodeVersion, message, stream, writtenBytes);
}
+ }
+
+ @Override
+ public Message prepareSend(Version nodeVersion, BytesReference bytesReference) {
+ return new NettyMessage(NettyUtils.toChannelBuffer(bytesReference));
+ }
+
+ @Override
+ public boolean canCompress(TransportRequest request) {
+ return super.canCompress(request) && (!(request instanceof BytesTransportRequest));
+ }
+
+ private class NettyMessage implements Message {
+ private final ChannelBuffer buffer;
+
+ public NettyMessage(ChannelBuffer buffer) {
+ this.buffer = buffer;
+ }
+
+ public StreamOutput getHeaderOutput() {
+ return new ChannelBufferStreamOutput(buffer);
+ }
+
+ public int size() {
+ return buffer.readableBytes();
+ }
+
+ @Override
+ public void send(Channel channel, Runnable onRequestSent) {
+ ChannelFuture future = channel.write(buffer);
+ ChannelFutureListener channelFutureListener = f -> onRequestSent.run();
+ future.addListener(channelFutureListener);
}
}
- protected Version getCurrentVersion() {
- // this is just for tests to mock stuff like the nodes version - tests can override this internally
- return Version.CURRENT;
+ private final static class ChannelBufferStreamOutput extends StreamOutput {
+
+ private final ChannelBuffer buffer;
+ private int offset;
+
+ public ChannelBufferStreamOutput(ChannelBuffer buffer) {
+ this.buffer = buffer;
+ this.offset = buffer.readerIndex();
+ }
+
+ @Override
+ public void writeByte(byte b) throws IOException {
+ buffer.setByte(offset++, b);
+ }
+
+ @Override
+ public void writeBytes(byte[] b, int offset, int length) throws IOException {
+ buffer.setBytes(this.offset, b, offset, length);
+ this.offset += length;
+ }
+
+ @Override
+ public void flush() throws IOException {
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+
+ @Override
+ public void reset() throws IOException {
+ throw new UnsupportedOperationException();
+ }
}
}
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/NettyTransportChannel.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyTransportChannel.java
deleted file mode 100644
index 0d5666408ea..00000000000
--- a/core/src/main/java/org/elasticsearch/transport/netty/NettyTransportChannel.java
+++ /dev/null
@@ -1,176 +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.netty;
-
-import org.elasticsearch.Version;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.bytes.ReleasablePagedBytesReference;
-import org.elasticsearch.common.compress.CompressorFactory;
-import org.elasticsearch.common.io.stream.BytesStreamOutput;
-import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
-import org.elasticsearch.common.io.stream.StreamOutput;
-import org.elasticsearch.common.lease.Releasables;
-import org.elasticsearch.common.netty.NettyUtils;
-import org.elasticsearch.common.netty.ReleaseChannelFutureListener;
-import org.elasticsearch.transport.RemoteTransportException;
-import org.elasticsearch.transport.TransportChannel;
-import org.elasticsearch.transport.TransportResponse;
-import org.elasticsearch.transport.TransportResponseOptions;
-import org.elasticsearch.transport.TransportServiceAdapter;
-import org.elasticsearch.transport.support.TransportStatus;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class NettyTransportChannel implements TransportChannel {
-
- private final NettyTransport transport;
- private final TransportServiceAdapter transportServiceAdapter;
- private final Version version;
- private final String action;
- private final Channel channel;
- private final long requestId;
- private final String profileName;
- private final long reservedBytes;
- private final AtomicBoolean released = new AtomicBoolean();
-
- public NettyTransportChannel(NettyTransport transport, TransportServiceAdapter transportServiceAdapter, String action, Channel channel,
- long requestId, Version version, String profileName, long reservedBytes) {
- this.transportServiceAdapter = transportServiceAdapter;
- this.version = version;
- this.transport = transport;
- this.action = action;
- this.channel = channel;
- this.requestId = requestId;
- this.profileName = profileName;
- this.reservedBytes = reservedBytes;
- }
-
- @Override
- public String getProfileName() {
- return profileName;
- }
-
- @Override
- public String action() {
- return this.action;
- }
-
- @Override
- public void sendResponse(TransportResponse response) throws IOException {
- sendResponse(response, TransportResponseOptions.EMPTY);
- }
-
- @Override
- public void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException {
- release();
- if (transport.compress) {
- options = TransportResponseOptions.builder(options).withCompress(transport.compress).build();
- }
-
- byte status = 0;
- status = TransportStatus.setResponse(status);
-
- ReleasableBytesStreamOutput bStream = null;
- boolean addedReleaseListener = false;
- try {
- bStream = new ReleasableBytesStreamOutput(transport.bigArrays);
- bStream.skip(NettyHeader.HEADER_SIZE);
- StreamOutput stream = bStream;
- if (options.compress()) {
- status = TransportStatus.setCompress(status);
- stream = CompressorFactory.COMPRESSOR.streamOutput(stream);
- }
- stream.setVersion(version);
- response.writeTo(stream);
- stream.close();
-
- ReleasablePagedBytesReference bytes = bStream.bytes();
- ChannelBuffer buffer = NettyUtils.toChannelBuffer(bytes);
- NettyHeader.writeHeader(buffer, requestId, status, version);
- ChannelFuture future = channel.write(buffer);
- ReleaseChannelFutureListener listener = new ReleaseChannelFutureListener(bytes);
- future.addListener(listener);
- addedReleaseListener = true;
- final TransportResponseOptions finalOptions = options;
- ChannelFutureListener onResponseSentListener =
- f -> transportServiceAdapter.onResponseSent(requestId, action, response, finalOptions);
- future.addListener(onResponseSentListener);
- } finally {
- if (!addedReleaseListener && bStream != null) {
- Releasables.close(bStream.bytes());
- }
- }
- }
-
- @Override
- public void sendResponse(Throwable error) throws IOException {
- release();
- BytesStreamOutput stream = new BytesStreamOutput();
- stream.skip(NettyHeader.HEADER_SIZE);
- RemoteTransportException tx = new RemoteTransportException(
- transport.nodeName(), transport.wrapAddress(channel.getLocalAddress()), action, error);
- stream.writeThrowable(tx);
- byte status = 0;
- status = TransportStatus.setResponse(status);
- status = TransportStatus.setError(status);
-
- BytesReference bytes = stream.bytes();
- ChannelBuffer buffer = NettyUtils.toChannelBuffer(bytes);
- NettyHeader.writeHeader(buffer, requestId, status, version);
- ChannelFuture future = channel.write(buffer);
- ChannelFutureListener onResponseSentListener =
- f -> transportServiceAdapter.onResponseSent(requestId, action, error);
- future.addListener(onResponseSentListener);
- }
-
- private void release() {
- // attempt to release once atomically
- if (released.compareAndSet(false, true) == false) {
- throw new IllegalStateException("reserved bytes are already released");
- }
- transport.inFlightRequestsBreaker().addWithoutBreaking(-reservedBytes);
- }
-
- @Override
- public long getRequestId() {
- return requestId;
- }
-
- @Override
- public String getChannelType() {
- return "netty";
- }
-
- /**
- * Returns the underlying netty channel. This method is intended be used for access to netty to get additional
- * details when processing the request and may be used by plugins. Responses should be sent using the methods
- * defined in this class and not directly on the channel.
- * @return underlying netty channel
- */
- public Channel getChannel() {
- return channel;
- }
-
-}
diff --git a/core/src/main/java/org/elasticsearch/common/netty/NettyUtils.java b/core/src/main/java/org/elasticsearch/transport/netty/NettyUtils.java
similarity index 90%
rename from core/src/main/java/org/elasticsearch/common/netty/NettyUtils.java
rename to core/src/main/java/org/elasticsearch/transport/netty/NettyUtils.java
index c37ca3ad6fb..f3fdde5e91c 100644
--- a/core/src/main/java/org/elasticsearch/common/netty/NettyUtils.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/NettyUtils.java
@@ -16,12 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.elasticsearch.common.netty;
+package org.elasticsearch.transport.netty;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.transport.netty.NettyInternalESLoggerFactory;
+import org.elasticsearch.common.logging.Loggers;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.logging.InternalLogger;
@@ -93,10 +93,11 @@ public class NettyUtils {
}
static {
- InternalLoggerFactory.setDefaultFactory(new NettyInternalESLoggerFactory() {
+ InternalLoggerFactory.setDefaultFactory(new InternalLoggerFactory() {
@Override
public InternalLogger newInstance(String name) {
- return super.newInstance(name.replace("org.jboss.netty.", "netty.").replace("org.jboss.netty.", "netty."));
+ name = name.replace("org.jboss.netty.", "netty.").replace("org.jboss.netty.", "netty.");
+ return new NettyInternalESLogger(Loggers.getLogger(name));
}
});
@@ -136,6 +137,13 @@ public class NettyUtils {
* Wraps the given ChannelBuffer with a BytesReference
*/
public static BytesReference toBytesReference(ChannelBuffer channelBuffer) {
- return new ChannelBufferBytesReference(channelBuffer);
+ return toBytesReference(channelBuffer, channelBuffer.readableBytes());
+ }
+
+ /**
+ * Wraps the given ChannelBuffer with a BytesReference of a given size
+ */
+ public static BytesReference toBytesReference(ChannelBuffer channelBuffer, int size) {
+ return new ChannelBufferBytesReference(channelBuffer, size);
}
}
diff --git a/core/src/main/java/org/elasticsearch/common/netty/OpenChannelsHandler.java b/core/src/main/java/org/elasticsearch/transport/netty/OpenChannelsHandler.java
similarity index 95%
rename from core/src/main/java/org/elasticsearch/common/netty/OpenChannelsHandler.java
rename to core/src/main/java/org/elasticsearch/transport/netty/OpenChannelsHandler.java
index 324db75dc90..df7cd73a42b 100644
--- a/core/src/main/java/org/elasticsearch/common/netty/OpenChannelsHandler.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/OpenChannelsHandler.java
@@ -17,8 +17,9 @@
* under the License.
*/
-package org.elasticsearch.common.netty;
+package org.elasticsearch.transport.netty;
+import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@@ -32,13 +33,14 @@ import org.jboss.netty.channel.ChannelState;
import org.jboss.netty.channel.ChannelStateEvent;
import org.jboss.netty.channel.ChannelUpstreamHandler;
+import java.io.Closeable;
import java.util.Set;
/**
*
*/
@ChannelHandler.Sharable
-public class OpenChannelsHandler implements ChannelUpstreamHandler {
+public class OpenChannelsHandler implements ChannelUpstreamHandler, Releasable {
final Set openChannels = ConcurrentCollections.newConcurrentSet();
final CounterMetric openChannelsMetric = new CounterMetric();
@@ -91,6 +93,7 @@ public class OpenChannelsHandler implements ChannelUpstreamHandler {
return totalChannelsMetric.count();
}
+ @Override
public void close() {
for (Channel channel : openChannels) {
channel.close().awaitUninterruptibly();
diff --git a/core/src/main/java/org/elasticsearch/transport/netty/SizeHeaderFrameDecoder.java b/core/src/main/java/org/elasticsearch/transport/netty/SizeHeaderFrameDecoder.java
index 9c410e4b912..d098fae6c78 100644
--- a/core/src/main/java/org/elasticsearch/transport/netty/SizeHeaderFrameDecoder.java
+++ b/core/src/main/java/org/elasticsearch/transport/netty/SizeHeaderFrameDecoder.java
@@ -19,107 +19,29 @@
package org.elasticsearch.transport.netty;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.monitor.jvm.JvmInfo;
-import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.transport.TcpHeader;
+import org.elasticsearch.transport.TcpTransport;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.channel.Channel;
import org.jboss.netty.channel.ChannelHandlerContext;
import org.jboss.netty.handler.codec.frame.FrameDecoder;
import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import java.io.IOException;
-import java.io.StreamCorruptedException;
-
/**
*/
-public class SizeHeaderFrameDecoder extends FrameDecoder {
-
- private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.9);
+final class SizeHeaderFrameDecoder extends FrameDecoder {
@Override
protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buffer) throws Exception {
- final int sizeHeaderLength = NettyHeader.MARKER_BYTES_SIZE + NettyHeader.MESSAGE_LENGTH_SIZE;
- if (buffer.readableBytes() < sizeHeaderLength) {
+ try {
+ boolean continueProcessing = TcpTransport.validateMessageHeader(NettyUtils.toBytesReference(buffer));
+ buffer.skipBytes(TcpHeader.MARKER_BYTES_SIZE + TcpHeader.MESSAGE_LENGTH_SIZE);
+ return continueProcessing ? buffer : null;
+ } catch (IllegalArgumentException ex) {
+ throw new TooLongFrameException(ex.getMessage(), ex);
+ } catch (IllegalStateException ex) {
return null;
}
-
- int readerIndex = buffer.readerIndex();
- if (buffer.getByte(readerIndex) != 'E' || buffer.getByte(readerIndex + 1) != 'S') {
- // special handling for what is probably HTTP
- if (bufferStartsWith(buffer, readerIndex, "GET ") ||
- bufferStartsWith(buffer, readerIndex, "POST ") ||
- bufferStartsWith(buffer, readerIndex, "PUT ") ||
- bufferStartsWith(buffer, readerIndex, "HEAD ") ||
- bufferStartsWith(buffer, readerIndex, "DELETE ") ||
- bufferStartsWith(buffer, readerIndex, "OPTIONS ") ||
- bufferStartsWith(buffer, readerIndex, "PATCH ") ||
- bufferStartsWith(buffer, readerIndex, "TRACE ")) {
-
- throw new HttpOnTransportException("This is not a HTTP port");
- }
-
- // we have 6 readable bytes, show 4 (should be enough)
- throw new StreamCorruptedException("invalid internal transport message format, got ("
- + Integer.toHexString(buffer.getByte(readerIndex) & 0xFF) + ","
- + Integer.toHexString(buffer.getByte(readerIndex + 1) & 0xFF) + ","
- + Integer.toHexString(buffer.getByte(readerIndex + 2) & 0xFF) + ","
- + Integer.toHexString(buffer.getByte(readerIndex + 3) & 0xFF) + ")");
- }
-
- int dataLen = buffer.getInt(buffer.readerIndex() + NettyHeader.MARKER_BYTES_SIZE);
- if (dataLen == NettyHeader.PING_DATA_SIZE) {
- // discard the messages we read and continue, this is achieved by skipping the bytes
- // and returning null
- buffer.skipBytes(sizeHeaderLength);
- return null;
- }
- if (dataLen <= 0) {
- throw new StreamCorruptedException("invalid data length: " + dataLen);
- }
- // safety against too large frames being sent
- if (dataLen > NINETY_PER_HEAP_SIZE) {
- throw new TooLongFrameException("transport content length received [" + new ByteSizeValue(dataLen) + "] exceeded ["
- + new ByteSizeValue(NINETY_PER_HEAP_SIZE) + "]");
- }
-
- if (buffer.readableBytes() < dataLen + sizeHeaderLength) {
- return null;
- }
- buffer.skipBytes(sizeHeaderLength);
- return buffer;
}
- private boolean bufferStartsWith(ChannelBuffer buffer, int readerIndex, String method) {
- char[] chars = method.toCharArray();
- for (int i = 0; i < chars.length; i++) {
- if (buffer.getByte(readerIndex + i) != chars[i]) {
- return false;
- }
- }
-
- return true;
- }
-
- /**
- * A helper exception to mark an incoming connection as potentially being HTTP
- * so an appropriate error code can be returned
- */
- public static class HttpOnTransportException extends ElasticsearchException {
-
- public HttpOnTransportException(String msg) {
- super(msg);
- }
-
- @Override
- public RestStatus status() {
- return RestStatus.BAD_REQUEST;
- }
-
- public HttpOnTransportException(StreamInput in) throws IOException{
- super(in);
- }
- }
}
diff --git a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
index d10fc890848..c0b5f2e9cf8 100644
--- a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
+++ b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
@@ -80,6 +80,7 @@ import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
import org.elasticsearch.transport.ActionNotFoundTransportException;
import org.elasticsearch.transport.ActionTransportException;
import org.elasticsearch.transport.ConnectTransportException;
+import org.elasticsearch.transport.TcpTransport;
import java.io.IOException;
import java.net.URISyntaxException;
@@ -763,7 +764,7 @@ public class ExceptionSerializationTests extends ESTestCase {
ids.put(122, null);
ids.put(123, org.elasticsearch.indices.IndexAlreadyExistsException.class);
ids.put(124, org.elasticsearch.script.Script.ScriptParseException.class);
- ids.put(125, org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.HttpOnTransportException.class);
+ ids.put(125, TcpTransport.HttpOnTransportException.class);
ids.put(126, org.elasticsearch.index.mapper.MapperParsingException.class);
ids.put(127, org.elasticsearch.search.SearchContextException.class);
ids.put(128, org.elasticsearch.search.builder.SearchSourceBuilderException.class);
diff --git a/core/src/test/java/org/elasticsearch/common/ChannelsTests.java b/core/src/test/java/org/elasticsearch/common/ChannelsTests.java
index 5bb9c614b84..4f2bad36d4a 100644
--- a/core/src/test/java/org/elasticsearch/common/ChannelsTests.java
+++ b/core/src/test/java/org/elasticsearch/common/ChannelsTests.java
@@ -162,20 +162,6 @@ public class ChannelsTests extends ESTestCase {
assertTrue("read bytes didn't match written bytes", sourceRef.equals(copyRef));
}
-
- public void testWriteFromChannel() throws IOException {
- int length = randomIntBetween(1, randomBytes.length / 2);
- int offset = randomIntBetween(0, randomBytes.length - length);
- ByteBuffer byteBuffer = ByteBuffer.wrap(randomBytes);
- ChannelBuffer source = new ByteBufferBackedChannelBuffer(byteBuffer);
- Channels.writeToChannel(source, offset, length, fileChannel);
-
- BytesReference copyRef = new BytesArray(Channels.readFromFileChannel(fileChannel, 0, length));
- BytesReference sourceRef = new BytesArray(randomBytes, offset, length);
-
- assertTrue("read bytes didn't match written bytes", sourceRef.equals(copyRef));
- }
-
class MockFileChannel extends FileChannel {
FileChannel delegate;
diff --git a/core/src/test/java/org/elasticsearch/transport/netty/KeyedLockTests.java b/core/src/test/java/org/elasticsearch/common/util/concurrent/KeyedLockTests.java
similarity index 99%
rename from core/src/test/java/org/elasticsearch/transport/netty/KeyedLockTests.java
rename to core/src/test/java/org/elasticsearch/common/util/concurrent/KeyedLockTests.java
index f9451375590..36335ee78d0 100644
--- a/core/src/test/java/org/elasticsearch/transport/netty/KeyedLockTests.java
+++ b/core/src/test/java/org/elasticsearch/common/util/concurrent/KeyedLockTests.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.elasticsearch.transport.netty;
+package org.elasticsearch.common.util.concurrent;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.concurrent.KeyedLock;
diff --git a/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
index 4fa99b3b80c..9ae029a4aa4 100644
--- a/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
+++ b/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
@@ -324,16 +324,13 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
public void testVoidMessageCompressed() {
serviceA.registerRequestHandler("sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
- new TransportRequestHandler() {
- @Override
- public void messageReceived(TransportRequest.Empty request, TransportChannel channel) {
- try {
- TransportResponseOptions responseOptions = TransportResponseOptions.builder().withCompress(true).build();
- channel.sendResponse(TransportResponse.Empty.INSTANCE, responseOptions);
- } catch (IOException e) {
- logger.error("Unexpected failure", e);
- fail(e.getMessage());
- }
+ (request, channel) -> {
+ try {
+ TransportResponseOptions responseOptions = TransportResponseOptions.builder().withCompress(true).build();
+ channel.sendResponse(TransportResponse.Empty.INSTANCE, responseOptions);
+ } catch (IOException e) {
+ logger.error("Unexpected failure", e);
+ fail(e.getMessage());
}
});
diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyPublishPortTests.java b/core/src/test/java/org/elasticsearch/transport/PublishPortTests.java
similarity index 94%
rename from core/src/test/java/org/elasticsearch/transport/netty/NettyPublishPortTests.java
rename to core/src/test/java/org/elasticsearch/transport/PublishPortTests.java
index 6f602dafc99..ffe7a2d7ce2 100644
--- a/core/src/test/java/org/elasticsearch/transport/netty/NettyPublishPortTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/PublishPortTests.java
@@ -17,13 +17,11 @@
* under the License.
*/
-package org.elasticsearch.transport.netty;
+package org.elasticsearch.transport;
import org.elasticsearch.common.network.NetworkUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.transport.BindTransportException;
-import org.elasticsearch.transport.TransportSettings;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
@@ -32,11 +30,11 @@ import java.util.List;
import static java.net.InetAddress.getByName;
import static java.util.Arrays.asList;
-import static org.elasticsearch.transport.netty.NettyTransport.resolvePublishPort;
+import static org.elasticsearch.transport.TcpTransport.resolvePublishPort;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
-public class NettyPublishPortTests extends ESTestCase {
+public class PublishPortTests extends ESTestCase {
public void testPublishPort() throws Exception {
int boundPort = randomIntBetween(9000, 9100);
diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportTests.java b/core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java
similarity index 78%
rename from core/src/test/java/org/elasticsearch/transport/netty/NettyTransportTests.java
rename to core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java
index a5bd6612cdf..da1dcf43e5d 100644
--- a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/TCPTransportTests.java
@@ -17,17 +17,17 @@
* under the License.
*/
-package org.elasticsearch.transport.netty;
+package org.elasticsearch.transport;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.test.ESTestCase;
-/** Unit tests for NettyTransport */
-public class NettyTransportTests extends ESTestCase {
-
+/** Unit tests for TCPTransport */
+public class TCPTransportTests extends ESTestCase {
+
/** Test ipv4 host with a default port works */
public void testParseV4DefaultPort() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("127.0.0.1", "1234", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("127.0.0.1", "1234", Integer.MAX_VALUE);
assertEquals(1, addresses.length);
assertEquals("127.0.0.1", addresses[0].getAddress());
@@ -36,19 +36,19 @@ public class NettyTransportTests extends ESTestCase {
/** Test ipv4 host with a default port range works */
public void testParseV4DefaultRange() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("127.0.0.1", "1234-1235", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("127.0.0.1", "1234-1235", Integer.MAX_VALUE);
assertEquals(2, addresses.length);
assertEquals("127.0.0.1", addresses[0].getAddress());
assertEquals(1234, addresses[0].getPort());
-
+
assertEquals("127.0.0.1", addresses[1].getAddress());
assertEquals(1235, addresses[1].getPort());
}
/** Test ipv4 host with port works */
public void testParseV4WithPort() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("127.0.0.1:2345", "1234", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("127.0.0.1:2345", "1234", Integer.MAX_VALUE);
assertEquals(1, addresses.length);
assertEquals("127.0.0.1", addresses[0].getAddress());
@@ -57,7 +57,7 @@ public class NettyTransportTests extends ESTestCase {
/** Test ipv4 host with port range works */
public void testParseV4WithPortRange() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("127.0.0.1:2345-2346", "1234", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("127.0.0.1:2345-2346", "1234", Integer.MAX_VALUE);
assertEquals(2, addresses.length);
assertEquals("127.0.0.1", addresses[0].getAddress());
@@ -70,7 +70,7 @@ public class NettyTransportTests extends ESTestCase {
/** Test unbracketed ipv6 hosts in configuration fail. Leave no ambiguity */
public void testParseV6UnBracketed() throws Exception {
try {
- NettyTransport.parse("::1", "1234", Integer.MAX_VALUE);
+ TcpTransport.parse("::1", "1234", Integer.MAX_VALUE);
fail("should have gotten exception");
} catch (IllegalArgumentException expected) {
assertTrue(expected.getMessage().contains("must be bracketed"));
@@ -79,7 +79,7 @@ public class NettyTransportTests extends ESTestCase {
/** Test ipv6 host with a default port works */
public void testParseV6DefaultPort() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("[::1]", "1234", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("[::1]", "1234", Integer.MAX_VALUE);
assertEquals(1, addresses.length);
assertEquals("::1", addresses[0].getAddress());
@@ -88,19 +88,19 @@ public class NettyTransportTests extends ESTestCase {
/** Test ipv6 host with a default port range works */
public void testParseV6DefaultRange() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("[::1]", "1234-1235", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("[::1]", "1234-1235", Integer.MAX_VALUE);
assertEquals(2, addresses.length);
assertEquals("::1", addresses[0].getAddress());
assertEquals(1234, addresses[0].getPort());
-
+
assertEquals("::1", addresses[1].getAddress());
assertEquals(1235, addresses[1].getPort());
}
/** Test ipv6 host with port works */
public void testParseV6WithPort() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("[::1]:2345", "1234", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("[::1]:2345", "1234", Integer.MAX_VALUE);
assertEquals(1, addresses.length);
assertEquals("::1", addresses[0].getAddress());
@@ -109,7 +109,7 @@ public class NettyTransportTests extends ESTestCase {
/** Test ipv6 host with port range works */
public void testParseV6WithPortRange() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("[::1]:2345-2346", "1234", Integer.MAX_VALUE);
+ TransportAddress[] addresses = TcpTransport.parse("[::1]:2345-2346", "1234", Integer.MAX_VALUE);
assertEquals(2, addresses.length);
assertEquals("::1", addresses[0].getAddress());
@@ -118,10 +118,10 @@ public class NettyTransportTests extends ESTestCase {
assertEquals("::1", addresses[1].getAddress());
assertEquals(2346, addresses[1].getPort());
}
-
+
/** Test per-address limit */
public void testAddressLimit() throws Exception {
- TransportAddress[] addresses = NettyTransport.parse("[::1]:100-200", "1000", 3);
+ TransportAddress[] addresses = TcpTransport.parse("[::1]:100-200", "1000", 3);
assertEquals(3, addresses.length);
assertEquals(100, addresses[0].getPort());
assertEquals(101, addresses[1].getPort());
diff --git a/core/src/test/java/org/elasticsearch/common/netty/ChannelBufferBytesReferenceTests.java b/core/src/test/java/org/elasticsearch/transport/netty/ChannelBufferBytesReferenceTests.java
similarity index 96%
rename from core/src/test/java/org/elasticsearch/common/netty/ChannelBufferBytesReferenceTests.java
rename to core/src/test/java/org/elasticsearch/transport/netty/ChannelBufferBytesReferenceTests.java
index 76a8626fee5..a284f6ea911 100644
--- a/core/src/test/java/org/elasticsearch/common/netty/ChannelBufferBytesReferenceTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/netty/ChannelBufferBytesReferenceTests.java
@@ -16,12 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.elasticsearch.common.netty;
+package org.elasticsearch.transport.netty;
import org.elasticsearch.common.bytes.AbstractBytesReferenceTestCase;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
+import org.elasticsearch.transport.netty.NettyUtils;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java
index df7dcb0714b..c69f56c2cbd 100644
--- a/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java
@@ -19,7 +19,6 @@
package org.elasticsearch.transport.netty;
import org.elasticsearch.Version;
-import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.lease.Releasables;
@@ -33,6 +32,7 @@ import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
+import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
@@ -56,7 +56,7 @@ public class NettyScheduledPingTests extends ESTestCase {
ThreadPool threadPool = new TestThreadPool(getClass().getName());
Settings settings = Settings.builder()
- .put(NettyTransport.PING_SCHEDULE.getKey(), "5ms")
+ .put(TcpTransport.PING_SCHEDULE.getKey(), "5ms")
.put(TransportSettings.PORT.getKey(), 0)
.put("cluster.name", "test")
.build();
@@ -89,12 +89,12 @@ public class NettyScheduledPingTests extends ESTestCase {
assertBusy(new Runnable() {
@Override
public void run() {
- assertThat(nettyA.scheduledPing.successfulPings.count(), greaterThan(100L));
- assertThat(nettyB.scheduledPing.successfulPings.count(), greaterThan(100L));
+ assertThat(nettyA.getPing().getSuccessfulPings(), greaterThan(100L));
+ assertThat(nettyB.getPing().getSuccessfulPings(), greaterThan(100L));
}
});
- assertThat(nettyA.scheduledPing.failedPings.count(), equalTo(0L));
- assertThat(nettyB.scheduledPing.failedPings.count(), equalTo(0L));
+ assertThat(nettyA.getPing().getFailedPings(), equalTo(0L));
+ assertThat(nettyB.getPing().getFailedPings(), equalTo(0L));
serviceA.registerRequestHandler("sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
new TransportRequestHandler() {
@@ -137,15 +137,12 @@ public class NettyScheduledPingTests extends ESTestCase {
}).txGet();
}
- assertBusy(new Runnable() {
- @Override
- public void run() {
- assertThat(nettyA.scheduledPing.successfulPings.count(), greaterThan(200L));
- assertThat(nettyB.scheduledPing.successfulPings.count(), greaterThan(200L));
- }
+ assertBusy(() -> {
+ assertThat(nettyA.getPing().getSuccessfulPings(), greaterThan(200L));
+ assertThat(nettyB.getPing().getSuccessfulPings(), greaterThan(200L));
});
- assertThat(nettyA.scheduledPing.failedPings.count(), equalTo(0L));
- assertThat(nettyB.scheduledPing.failedPings.count(), equalTo(0L));
+ assertThat(nettyA.getPing().getFailedPings(), equalTo(0L));
+ assertThat(nettyB.getPing().getFailedPings(), equalTo(0L));
Releasables.close(serviceA, serviceB);
terminate(threadPool);
diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportIT.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportIT.java
index 2f89435c6df..310f804ef7c 100644
--- a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportIT.java
+++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportIT.java
@@ -44,6 +44,7 @@ import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.ChannelPipelineFactory;
import java.io.IOException;
+import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.Collections;
@@ -98,45 +99,24 @@ public class NettyTransportIT extends ESIntegTestCase {
super(settings, threadPool, networkService, bigArrays, namedWriteableRegistry, circuitBreakerService);
}
+ protected String handleRequest(Channel channel, String profileName,
+ StreamInput stream, long requestId, int messageLengthBytes, Version version,
+ InetSocketAddress remoteAddress) throws IOException {
+ String action = super.handleRequest(channel, profileName, stream, requestId, messageLengthBytes, version,
+ remoteAddress);
+ channelProfileName = TransportSettings.DEFAULT_PROFILE;
+ return action;
+ }
+
@Override
- public ChannelPipelineFactory configureServerChannelPipelineFactory(String name, Settings groupSettings) {
- return new ErrorPipelineFactory(this, name, groupSettings);
- }
-
- private static class ErrorPipelineFactory extends ServerChannelPipelineFactory {
-
- private final ESLogger logger;
-
- public ErrorPipelineFactory(ExceptionThrowingNettyTransport nettyTransport, String name, Settings groupSettings) {
- super(nettyTransport, name, groupSettings);
- this.logger = nettyTransport.logger;
- }
-
- @Override
- public ChannelPipeline getPipeline() throws Exception {
- ChannelPipeline pipeline = super.getPipeline();
- pipeline.replace("dispatcher", "dispatcher",
- new MessageChannelHandler(nettyTransport, logger, TransportSettings.DEFAULT_PROFILE) {
-
- @Override
- protected String handleRequest(Channel channel, Marker marker, StreamInput buffer, long requestId,
- int messageLengthBytes, Version version) throws IOException {
- String action = super.handleRequest(channel, marker, buffer, requestId, messageLengthBytes, version);
- channelProfileName = this.profileName;
- return action;
- }
-
- @Override
- protected void validateRequest(Marker marker, StreamInput buffer, long requestId, String action) throws IOException {
- super.validateRequest(marker, buffer, requestId, action);
- String error = threadPool.getThreadContext().getHeader("ERROR");
- if (error != null) {
- throw new ElasticsearchException(error);
- }
- }
- });
- return pipeline;
+ 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/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java
index 6fdc214d18d..352c90d2317 100644
--- a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java
@@ -18,7 +18,6 @@
*/
package org.elasticsearch.transport.netty;
-import org.elasticsearch.Version;
import org.elasticsearch.common.component.Lifecycle;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.network.NetworkService;
@@ -30,6 +29,7 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.TransportSettings;
import org.junit.Before;
@@ -58,7 +58,7 @@ public class NettyTransportMultiPortTests extends ESTestCase {
.build();
ThreadPool threadPool = new TestThreadPool("tst");
- try (NettyTransport transport = startNettyTransport(settings, threadPool)) {
+ try (TcpTransport> transport = startTransport(settings, threadPool)) {
assertEquals(1, transport.profileBoundAddresses().size());
assertEquals(1, transport.boundAddress().boundAddresses().length);
} finally {
@@ -74,7 +74,7 @@ public class NettyTransportMultiPortTests extends ESTestCase {
.build();
ThreadPool threadPool = new TestThreadPool("tst");
- try (NettyTransport transport = startNettyTransport(settings, threadPool)) {
+ try (TcpTransport> transport = startTransport(settings, threadPool)) {
assertEquals(1, transport.profileBoundAddresses().size());
assertEquals(1, transport.boundAddress().boundAddresses().length);
} finally {
@@ -91,7 +91,7 @@ public class NettyTransportMultiPortTests extends ESTestCase {
.build();
ThreadPool threadPool = new TestThreadPool("tst");
- try (NettyTransport transport = startNettyTransport(settings, threadPool)) {
+ try (TcpTransport> transport = startTransport(settings, threadPool)) {
assertEquals(0, transport.profileBoundAddresses().size());
assertEquals(1, transport.boundAddress().boundAddresses().length);
} finally {
@@ -107,7 +107,7 @@ public class NettyTransportMultiPortTests extends ESTestCase {
.build();
ThreadPool threadPool = new TestThreadPool("tst");
- try (NettyTransport transport = startNettyTransport(settings, threadPool)) {
+ try (TcpTransport> transport = startTransport(settings, threadPool)) {
assertEquals(0, transport.profileBoundAddresses().size());
assertEquals(1, transport.boundAddress().boundAddresses().length);
} finally {
@@ -125,7 +125,7 @@ public class NettyTransportMultiPortTests extends ESTestCase {
.build();
ThreadPool threadPool = new TestThreadPool("tst");
- try (NettyTransport transport = startNettyTransport(settings, threadPool)) {
+ try (TcpTransport> transport = startTransport(settings, threadPool)) {
assertEquals(0, transport.profileBoundAddresses().size());
assertEquals(1, transport.boundAddress().boundAddresses().length);
} finally {
@@ -133,14 +133,13 @@ public class NettyTransportMultiPortTests extends ESTestCase {
}
}
- private NettyTransport startNettyTransport(Settings settings, ThreadPool threadPool) {
+ private TcpTransport> startTransport(Settings settings, ThreadPool threadPool) {
BigArrays bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
-
- NettyTransport nettyTransport = new NettyTransport(settings, threadPool, new NetworkService(settings), bigArrays,
+ TcpTransport> transport = new NettyTransport(settings, threadPool, new NetworkService(settings), bigArrays,
new NamedWriteableRegistry(), new NoneCircuitBreakerService());
- nettyTransport.start();
+ transport.start();
- assertThat(nettyTransport.lifecycleState(), is(Lifecycle.State.STARTED));
- return nettyTransport;
+ assertThat(transport.lifecycleState(), is(Lifecycle.State.STARTED));
+ return transport;
}
}
diff --git a/core/src/test/java/org/elasticsearch/common/netty/NettyUtilsTests.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyUtilsTests.java
similarity index 98%
rename from core/src/test/java/org/elasticsearch/common/netty/NettyUtilsTests.java
rename to core/src/test/java/org/elasticsearch/transport/netty/NettyUtilsTests.java
index 2d981dc9eae..fa8f30249bb 100644
--- a/core/src/test/java/org/elasticsearch/common/netty/NettyUtilsTests.java
+++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyUtilsTests.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.elasticsearch.common.netty;
+package org.elasticsearch.transport.netty;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
@@ -27,7 +27,6 @@ import org.elasticsearch.test.ESTestCase;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.buffer.CompositeChannelBuffer;
-import org.junit.Before;
import java.io.IOException;
diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
index 20af5cf50b2..b6b75b1ec64 100644
--- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
+++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
@@ -89,6 +89,7 @@ import org.elasticsearch.search.SearchService;
import org.elasticsearch.test.disruption.ServiceDisruptionScheme;
import org.elasticsearch.test.transport.AssertingLocalTransport;
import org.elasticsearch.test.transport.MockTransportService;
+import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.TransportSettings;
@@ -421,9 +422,9 @@ public final class InternalTestCluster extends TestCluster {
// randomize netty settings
if (random.nextBoolean()) {
builder.put(NettyTransport.WORKER_COUNT.getKey(), random.nextInt(3) + 1);
- builder.put(NettyTransport.CONNECTIONS_PER_NODE_RECOVERY.getKey(), random.nextInt(2) + 1);
- builder.put(NettyTransport.CONNECTIONS_PER_NODE_BULK.getKey(), random.nextInt(3) + 1);
- builder.put(NettyTransport.CONNECTIONS_PER_NODE_REG.getKey(), random.nextInt(6) + 1);
+ builder.put(TcpTransport.CONNECTIONS_PER_NODE_RECOVERY.getKey(), random.nextInt(2) + 1);
+ builder.put(TcpTransport.CONNECTIONS_PER_NODE_BULK.getKey(), random.nextInt(3) + 1);
+ builder.put(TcpTransport.CONNECTIONS_PER_NODE_REG.getKey(), random.nextInt(6) + 1);
}
if (random.nextBoolean()) {
@@ -455,7 +456,7 @@ public final class InternalTestCluster extends TestCluster {
}
if (random.nextBoolean()) {
- builder.put(NettyTransport.PING_SCHEDULE.getKey(), RandomInts.randomIntBetween(random, 100, 2000) + "ms");
+ builder.put(TcpTransport.PING_SCHEDULE.getKey(), RandomInts.randomIntBetween(random, 100, 2000) + "ms");
}
if (random.nextBoolean()) {