diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 5a08a834c3c..51215ba6ce9 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -1168,7 +1168,6 @@ - @@ -1180,10 +1179,10 @@ - - - - + + + + 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 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 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/dev-tools/smoke_test_rc.py b/dev-tools/smoke_test_rc.py index 259081ddc10..33abbf96345 100644 --- a/dev-tools/smoke_test_rc.py +++ b/dev-tools/smoke_test_rc.py @@ -63,7 +63,7 @@ DEFAULT_PLUGINS = ["analysis-icu", "analysis-phonetic", "analysis-smartcn", "analysis-stempel", - "discovery-azure", + "discovery-azure-classic", "discovery-ec2", "discovery-gce", "ingest-attachment", diff --git a/docs/plugins/discovery-azure.asciidoc b/docs/plugins/discovery-azure-classic.asciidoc similarity index 93% rename from docs/plugins/discovery-azure.asciidoc rename to docs/plugins/discovery-azure-classic.asciidoc index f49f1568cab..d794adbea7e 100644 --- a/docs/plugins/discovery-azure.asciidoc +++ b/docs/plugins/discovery-azure-classic.asciidoc @@ -1,9 +1,13 @@ -[[discovery-azure]] -=== Azure Discovery Plugin +[[discovery-azure-classic]] +=== Azure Classic Discovery Plugin -The Azure Discovery plugin uses the Azure API for unicast discovery. +The Azure Classic Discovery plugin uses the Azure Classic API for unicast discovery. -[[discovery-azure-install]] +// TODO: Link to ARM plugin when ready +// See issue https://github.com/elastic/elasticsearch/issues/19146 +deprecated[5.0.0, Use coming Azure ARM Discovery plugin instead] + +[[discovery-azure-classic-install]] [float] ==== Installation @@ -11,13 +15,13 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/elasticsearch-plugin install discovery-azure +sudo bin/elasticsearch-plugin install discovery-azure-classic ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must be restarted after installation. -[[discovery-azure-remove]] +[[discovery-azure-classic-remove]] [float] ==== Removal @@ -25,12 +29,12 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/elasticsearch-plugin remove discovery-azure +sudo bin/elasticsearch-plugin remove discovery-azure-classic ---------------------------------------------------------------- The node must be stopped before removing the plugin. -[[discovery-azure-usage]] +[[discovery-azure-classic-usage]] ==== Azure Virtual Machine Discovery Azure VM discovery allows to use the azure APIs to perform automatic discovery (similar to multicast in non hostile @@ -64,7 +68,7 @@ You can use {ref}/modules-network.html[core network host settings]. For example ============================================== -[[discovery-azure-short]] +[[discovery-azure-classic-short]] ===== How to start (short story) * Create Azure instances @@ -73,7 +77,7 @@ You can use {ref}/modules-network.html[core network host settings]. For example * Modify `elasticsearch.yml` file * Start Elasticsearch -[[discovery-azure-settings]] +[[discovery-azure-classic-settings]] ===== Azure credential API settings The following are a list of settings that can further control the credential API: @@ -100,7 +104,7 @@ The following are a list of settings that can further control the credential API your_azure_cloud_service_name -[[discovery-azure-settings-advanced]] +[[discovery-azure-classic-settings-advanced]] ===== Advanced settings The following are a list of settings that can further control the discovery: @@ -143,7 +147,7 @@ discovery: slot: production ---- -[[discovery-azure-long]] +[[discovery-azure-classic-long]] ==== Setup process for Azure Discovery We will expose here one strategy which is to hide our Elasticsearch cluster from outside. @@ -153,7 +157,7 @@ other. That means that with this mode, you can use elasticsearch unicast discovery to build a cluster, using the Azure API to retrieve information about your nodes. -[[discovery-azure-long-prerequisites]] +[[discovery-azure-classic-long-prerequisites]] ===== Prerequisites Before starting, you need to have: @@ -243,7 +247,7 @@ azure account download azure account import /tmp/azure.publishsettings ---- -[[discovery-azure-long-instance]] +[[discovery-azure-classic-long-instance]] ===== Creating your first instance You need to have a storage account available. Check http://www.windowsazure.com/en-us/develop/net/how-to-guides/blob-storage/#create-account[Azure Blob Storage documentation] @@ -396,7 +400,7 @@ This command should give you a JSON result: } ---- -[[discovery-azure-long-plugin]] +[[discovery-azure-classic-long-plugin]] ===== Install elasticsearch cloud azure plugin [source,sh] @@ -405,7 +409,7 @@ This command should give you a JSON result: sudo service elasticsearch stop # Install the plugin -sudo /usr/share/elasticsearch/bin/elasticsearch-plugin install discovery-azure +sudo /usr/share/elasticsearch/bin/elasticsearch-plugin install discovery-azure-classic # Configure it sudo vi /etc/elasticsearch/elasticsearch.yml @@ -441,7 +445,7 @@ sudo service elasticsearch start If anything goes wrong, check your logs in `/var/log/elasticsearch`. -[[discovery-azure-scale]] +[[discovery-azure-classic-scale]] ==== Scaling Out! You need first to create an image of your previous machine. diff --git a/docs/plugins/discovery.asciidoc b/docs/plugins/discovery.asciidoc index 62c5b4551ac..999bf9c0e1f 100644 --- a/docs/plugins/discovery.asciidoc +++ b/docs/plugins/discovery.asciidoc @@ -13,9 +13,9 @@ The core discovery plugins are: The EC2 discovery plugin uses the https://github.com/aws/aws-sdk-java[AWS API] for unicast discovery. -<>:: +<>:: -The Azure discovery plugin uses the Azure API for unicast discovery. +The Azure Classic discovery plugin uses the Azure Classic API for unicast discovery. <>:: @@ -33,7 +33,7 @@ A number of discovery plugins have been contributed by our community: include::discovery-ec2.asciidoc[] -include::discovery-azure.asciidoc[] +include::discovery-azure-classic.asciidoc[] include::discovery-gce.asciidoc[] diff --git a/docs/plugins/redirects.asciidoc b/docs/plugins/redirects.asciidoc index c8cf10c6319..0f9c0b40f2c 100644 --- a/docs/plugins/redirects.asciidoc +++ b/docs/plugins/redirects.asciidoc @@ -24,7 +24,7 @@ The `cloud-aws` plugin has been split into two separate plugins: The `cloud-azure` plugin has been split into two separate plugins: -* <> (`discovery-azure`) +* <> (`discovery-azure-classic`) * <> (`repository-azure`) diff --git a/docs/reference/cat/plugins.asciidoc b/docs/reference/cat/plugins.asciidoc index 81df5cfb127..0af1faa5c9c 100644 --- a/docs/reference/cat/plugins.asciidoc +++ b/docs/reference/cat/plugins.asciidoc @@ -6,14 +6,9 @@ The `plugins` command provides a view per node of running plugins. This informat [source,sh] ------------------------------------------------------------------------------ % curl 'localhost:9200/_cat/plugins?v' -name component version type isolation url -Abraxas discovery-azure 2.1.0-SNAPSHOT j x -Abraxas lang-javascript 2.0.0-SNAPSHOT j x -Abraxas marvel NA j/s x /_plugin/marvel/ -Abraxas lang-python 2.0.0-SNAPSHOT j x -Abraxas inquisitor NA s /_plugin/inquisitor/ -Abraxas kopf 0.5.2 s /_plugin/kopf/ -Abraxas segmentspy NA s /_plugin/segmentspy/ +name component version description +Abraxas discovery-gce 5.0.0 The Google Compute Engine (GCE) Discovery plugin allows to use GCE API for the unicast discovery mechanism. +Abraxas lang-javascript 5.0.0 The JavaScript language plugin allows to have javascript as the language of scripts to execute. ------------------------------------------------------------------------------- We can tell quickly how many plugins per node we have and which versions. diff --git a/docs/reference/index.asciidoc b/docs/reference/index.asciidoc index 3a625ac1a2c..636475dfb47 100644 --- a/docs/reference/index.asciidoc +++ b/docs/reference/index.asciidoc @@ -1,7 +1,7 @@ [[elasticsearch-reference]] = Elasticsearch Reference -:version: 5.0.0-alpha3 +:version: 5.0.0-alpha4 :major-version: 5.x :branch: master :jdk: 1.8.0_73 diff --git a/docs/reference/migration/migrate_5_0/plugins.asciidoc b/docs/reference/migration/migrate_5_0/plugins.asciidoc index 2826c822d15..e1ff497a8f3 100644 --- a/docs/reference/migration/migrate_5_0/plugins.asciidoc +++ b/docs/reference/migration/migrate_5_0/plugins.asciidoc @@ -63,7 +63,7 @@ Proxy settings for both plugins have been renamed: Cloud Azure plugin has been split in three plugins: -* {plugins}/discovery-azure.html[Discovery Azure plugin] +* {plugins}/discovery-azure-classic.html[Discovery Azure plugin] * {plugins}/repository-azure.html[Repository Azure plugin] * {plugins}/store-smb.html[Store SMB plugin] diff --git a/docs/reference/modules/discovery/azure.asciidoc b/docs/reference/modules/discovery/azure.asciidoc index 87d072564b3..1343819b02a 100644 --- a/docs/reference/modules/discovery/azure.asciidoc +++ b/docs/reference/modules/discovery/azure.asciidoc @@ -1,5 +1,5 @@ -[[modules-discovery-azure]] -=== Azure Discovery +[[modules-discovery-azure-classic]] +=== Azure Classic Discovery -Azure discovery allows to use the Azure APIs to perform automatic discovery (similar to multicast). -It is available as a plugin. See {plugins}/discovery-azure.html[discovery-azure] for more information. +Azure classic discovery allows to use the Azure Classic APIs to perform automatic discovery (similar to multicast). +It is available as a plugin. See {plugins}/discovery-azure-classic.html[discovery-azure-classic] for more information. diff --git a/docs/reference/query-dsl/function-score-query.asciidoc b/docs/reference/query-dsl/function-score-query.asciidoc index b6e4dedbc88..c6477b78d8b 100644 --- a/docs/reference/query-dsl/function-score-query.asciidoc +++ b/docs/reference/query-dsl/function-score-query.asciidoc @@ -83,9 +83,16 @@ First, each document is scored by the defined functions. The parameter `max`:: maximum score is used `min`:: minimum score is used -Because scores can be on different scales (for example, between 0 and 1 for decay functions but arbitrary for `field_value_factor`) and also because sometimes a different impact of functions on the score is desirable, the score of each function can be adjusted with a user defined `weight` (). The `weight` can be defined per function in the `functions` array (example above) and is multiplied with the score computed by the respective function. +Because scores can be on different scales (for example, between 0 and 1 for decay functions but arbitrary for `field_value_factor`) and also +because sometimes a different impact of functions on the score is desirable, the score of each function can be adjusted with a user defined +`weight`. The `weight` can be defined per function in the `functions` array (example above) and is multiplied with the score computed by +the respective function. If weight is given without any other function declaration, `weight` acts as a function that simply returns the `weight`. +In case `score_mode` is set to `avg` the individual scores will be combined by a **weighted** average. +For example, if two functions return score 1 and 2 and their respective weights are 3 and 4, then their scores will be combined as +`(1*3+2*4)/(3+4)` and **not** `(1*3+2*4)/2`. + The new score can be restricted to not exceed a certain limit by setting the `max_boost` parameter. The default for `max_boost` is FLT_MAX. diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 62e04e669a1..150b1b93a36 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -699,7 +699,9 @@ The meaning of the stats are as follows: This is not currently used and will always report `0`. Currently aggregation profiling only times the shard level parts of the aggregation execution. Timing of the reduce phase will be added later. -=== Performance Notes +=== Profiling Considerations + +==== Performance Notes Like any profiler, the Profile API introduces a non-negligible overhead to search execution. The act of instrumenting low-level method calls such as `collect`, `advance` and `next_doc` can be fairly expensive, since these methods are called @@ -710,7 +712,7 @@ There are also cases where special Lucene optimizations are disabled, since they could cause some queries to report larger relative times than their non-profiled counterparts, but in general should not have a drastic effect compared to other components in the profiled query. -=== Limitations +==== Limitations - Profiling statistics are currently not available for suggestions, highlighting, `dfs_query_then_fetch` - Profiling of the reduce phase of aggregation is currently not available diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorPlugin.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorPlugin.java index a15acf67c02..4359568b3f6 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorPlugin.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorPlugin.java @@ -21,8 +21,6 @@ package org.elasticsearch.percolator; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.mapper.Mapper; @@ -39,8 +37,6 @@ import java.util.Map; public class PercolatorPlugin extends Plugin implements MapperPlugin, ActionPlugin { - public static final String NAME = "percolator"; - private final Settings settings; public PercolatorPlugin(Settings settings) { @@ -65,7 +61,7 @@ public class PercolatorPlugin extends Plugin implements MapperPlugin, ActionPlug @Override public List> getSettings() { - return Arrays.asList(PercolatorFieldMapper.INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING); + return Collections.singletonList(PercolatorFieldMapper.INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING); } @Override @@ -73,7 +69,4 @@ public class PercolatorPlugin extends Plugin implements MapperPlugin, ActionPlug return Collections.singletonMap(PercolatorFieldMapper.CONTENT_TYPE, new PercolatorFieldMapper.TypeParser()); } - static boolean transportClientMode(Settings settings) { - return TransportClient.CLIENT_TYPE.equals(settings.get(Client.CLIENT_TYPE_SETTING_S.getKey())); - } } diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/RestMultiPercolateAction.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/RestMultiPercolateAction.java index a2902a9a7c2..41de2de42d1 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/RestMultiPercolateAction.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/RestMultiPercolateAction.java @@ -36,13 +36,10 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; public class RestMultiPercolateAction extends BaseRestHandler { private final boolean allowExplicitIndex; - private final TransportMultiPercolateAction action; @Inject - public RestMultiPercolateAction(Settings settings, RestController controller, Client client, - TransportMultiPercolateAction action) { + public RestMultiPercolateAction(Settings settings, RestController controller, Client client) { super(settings, client); - this.action = action; controller.registerHandler(POST, "/_mpercolate", this); controller.registerHandler(POST, "/{index}/_mpercolate", this); controller.registerHandler(POST, "/{index}/{type}/_mpercolate", this); @@ -61,7 +58,8 @@ public class RestMultiPercolateAction extends BaseRestHandler { multiPercolateRequest.indices(Strings.splitStringByCommaToArray(restRequest.param("index"))); multiPercolateRequest.documentType(restRequest.param("type")); multiPercolateRequest.add(RestActions.getRestContent(restRequest), allowExplicitIndex); - action.execute(multiPercolateRequest, new RestToXContentListener(restChannel)); + client.execute(MultiPercolateAction.INSTANCE, multiPercolateRequest, + new RestToXContentListener(restChannel)); } } diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/RestPercolateAction.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/RestPercolateAction.java index b752cc55f6c..6dffd5518c8 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/RestPercolateAction.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/RestPercolateAction.java @@ -36,13 +36,9 @@ import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.POST; public class RestPercolateAction extends BaseRestHandler { - - private final TransportPercolateAction action; - @Inject - public RestPercolateAction(Settings settings, RestController controller, Client client, TransportPercolateAction action) { + public RestPercolateAction(Settings settings, RestController controller, Client client) { super(settings, client); - this.action = action; controller.registerHandler(GET, "/{index}/{type}/_percolate", this); controller.registerHandler(POST, "/{index}/{type}/_percolate", this); @@ -54,7 +50,8 @@ public class RestPercolateAction extends BaseRestHandler { controller.registerHandler(GET, "/{index}/{type}/_percolate/count", countHandler); controller.registerHandler(POST, "/{index}/{type}/_percolate/count", countHandler); - RestCountPercolateExistingDocHandler countExistingDocHandler = new RestCountPercolateExistingDocHandler(settings, controller, client); + RestCountPercolateExistingDocHandler countExistingDocHandler = new RestCountPercolateExistingDocHandler(settings, controller, + client); controller.registerHandler(GET, "/{index}/{type}/{id}/_percolate/count", countExistingDocHandler); controller.registerHandler(POST, "/{index}/{type}/{id}/_percolate/count", countExistingDocHandler); } @@ -67,10 +64,11 @@ public class RestPercolateAction extends BaseRestHandler { percolateRequest.source(RestActions.getRestContent(restRequest)); percolateRequest.indicesOptions(IndicesOptions.fromRequest(restRequest, percolateRequest.indicesOptions())); - executePercolate(percolateRequest, restChannel); + executePercolate(client, percolateRequest, restChannel); } - void parseExistingDocPercolate(PercolateRequest percolateRequest, RestRequest restRequest, RestChannel restChannel, final Client client) { + void parseExistingDocPercolate(PercolateRequest percolateRequest, RestRequest restRequest, RestChannel restChannel, + final Client client) { String index = restRequest.param("index"); String type = restRequest.param("type"); percolateRequest.indices(Strings.splitStringByCommaToArray(restRequest.param("percolate_index", index))); @@ -91,11 +89,11 @@ public class RestPercolateAction extends BaseRestHandler { percolateRequest.source(RestActions.getRestContent(restRequest)); percolateRequest.indicesOptions(IndicesOptions.fromRequest(restRequest, percolateRequest.indicesOptions())); - executePercolate(percolateRequest, restChannel); + executePercolate(client, percolateRequest, restChannel); } - void executePercolate(final PercolateRequest percolateRequest, final RestChannel restChannel) { - action.execute(percolateRequest, new RestToXContentListener<>(restChannel)); + void executePercolate(final Client client, final PercolateRequest percolateRequest, final RestChannel restChannel) { + client.execute(PercolateAction.INSTANCE, percolateRequest, new RestToXContentListener<>(restChannel)); } @Override diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestRethrottleAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestRethrottleAction.java index 9841794ca2a..cb53509a461 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestRethrottleAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestRethrottleAction.java @@ -36,14 +36,11 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.action.admin.cluster.node.tasks.RestListTasksAction.nodeSettingListener; public class RestRethrottleAction extends BaseRestHandler { - private final TransportRethrottleAction action; private final ClusterService clusterService; @Inject - public RestRethrottleAction(Settings settings, RestController controller, Client client, TransportRethrottleAction action, - ClusterService clusterService) { + public RestRethrottleAction(Settings settings, RestController controller, Client client, ClusterService clusterService) { super(settings, client); - this.action = action; this.clusterService = clusterService; controller.registerHandler(POST, "/_update_by_query/{taskId}/_rethrottle", this); controller.registerHandler(POST, "/_delete_by_query/{taskId}/_rethrottle", this); @@ -60,6 +57,6 @@ public class RestRethrottleAction extends BaseRestHandler { } internalRequest.setRequestsPerSecond(requestsPerSecond); ActionListener listener = nodeSettingListener(clusterService, new RestToXContentListener<>(channel)); - action.execute(internalRequest, listener); + client.execute(RethrottleAction.INSTANCE, internalRequest, listener); } } diff --git a/plugins/discovery-azure/build.gradle b/plugins/discovery-azure-classic/build.gradle similarity index 97% rename from plugins/discovery-azure/build.gradle rename to plugins/discovery-azure-classic/build.gradle index 8f0c641e150..88874968b21 100644 --- a/plugins/discovery-azure/build.gradle +++ b/plugins/discovery-azure-classic/build.gradle @@ -20,8 +20,8 @@ import org.elasticsearch.gradle.LoggedExec */ esplugin { - description 'The Azure Discovery plugin allows to use Azure API for the unicast discovery mechanism' - classname 'org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin' + description 'The Azure Classic Discovery plugin allows to use Azure Classic API for the unicast discovery mechanism' + classname 'org.elasticsearch.plugin.discovery.azure.classic.AzureDiscoveryPlugin' } versions << [ diff --git a/plugins/discovery-azure/licenses/azure-LICENSE.txt b/plugins/discovery-azure-classic/licenses/azure-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/azure-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/azure-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/azure-NOTICE.txt b/plugins/discovery-azure-classic/licenses/azure-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/azure-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/azure-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/azure-core-0.9.3.jar.sha1 b/plugins/discovery-azure-classic/licenses/azure-core-0.9.3.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/azure-core-0.9.3.jar.sha1 rename to plugins/discovery-azure-classic/licenses/azure-core-0.9.3.jar.sha1 diff --git a/plugins/discovery-azure/licenses/azure-svc-mgmt-compute-0.9.3.jar.sha1 b/plugins/discovery-azure-classic/licenses/azure-svc-mgmt-compute-0.9.3.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/azure-svc-mgmt-compute-0.9.3.jar.sha1 rename to plugins/discovery-azure-classic/licenses/azure-svc-mgmt-compute-0.9.3.jar.sha1 diff --git a/plugins/discovery-azure/licenses/commons-codec-1.10.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-codec-1.10.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/commons-codec-1.10.jar.sha1 rename to plugins/discovery-azure-classic/licenses/commons-codec-1.10.jar.sha1 diff --git a/plugins/discovery-azure/licenses/commons-codec-LICENSE.txt b/plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-codec-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/commons-codec-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/commons-codec-NOTICE.txt b/plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-codec-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/commons-codec-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/commons-io-2.4.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-io-2.4.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/commons-io-2.4.jar.sha1 rename to plugins/discovery-azure-classic/licenses/commons-io-2.4.jar.sha1 diff --git a/plugins/discovery-azure/licenses/commons-io-LICENSE.txt b/plugins/discovery-azure-classic/licenses/commons-io-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-io-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/commons-io-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/commons-io-NOTICE.txt b/plugins/discovery-azure-classic/licenses/commons-io-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-io-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/commons-io-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/commons-lang-2.6.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-lang-2.6.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/commons-lang-2.6.jar.sha1 rename to plugins/discovery-azure-classic/licenses/commons-lang-2.6.jar.sha1 diff --git a/plugins/discovery-azure/licenses/commons-lang-LICENSE.txt b/plugins/discovery-azure-classic/licenses/commons-lang-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-lang-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/commons-lang-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/commons-lang-NOTICE.txt b/plugins/discovery-azure-classic/licenses/commons-lang-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-lang-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/commons-lang-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/commons-logging-1.1.3.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-logging-1.1.3.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/commons-logging-1.1.3.jar.sha1 rename to plugins/discovery-azure-classic/licenses/commons-logging-1.1.3.jar.sha1 diff --git a/plugins/discovery-azure/licenses/commons-logging-LICENSE.txt b/plugins/discovery-azure-classic/licenses/commons-logging-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-logging-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/commons-logging-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/commons-logging-NOTICE.txt b/plugins/discovery-azure-classic/licenses/commons-logging-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/commons-logging-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/commons-logging-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/httpclient-4.5.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.2.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/httpclient-4.5.2.jar.sha1 rename to plugins/discovery-azure-classic/licenses/httpclient-4.5.2.jar.sha1 diff --git a/plugins/discovery-azure/licenses/httpclient-LICENSE.txt b/plugins/discovery-azure-classic/licenses/httpclient-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/httpclient-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/httpclient-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/httpclient-NOTICE.txt b/plugins/discovery-azure-classic/licenses/httpclient-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/httpclient-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/httpclient-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/httpcore-4.4.4.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.4.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/httpcore-4.4.4.jar.sha1 rename to plugins/discovery-azure-classic/licenses/httpcore-4.4.4.jar.sha1 diff --git a/plugins/discovery-azure/licenses/httpcore-LICENSE.txt b/plugins/discovery-azure-classic/licenses/httpcore-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/httpcore-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/httpcore-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/httpcore-NOTICE.txt b/plugins/discovery-azure-classic/licenses/httpcore-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/httpcore-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/httpcore-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/jackson-LICENSE b/plugins/discovery-azure-classic/licenses/jackson-LICENSE similarity index 100% rename from plugins/discovery-azure/licenses/jackson-LICENSE rename to plugins/discovery-azure-classic/licenses/jackson-LICENSE diff --git a/plugins/discovery-azure/licenses/jackson-NOTICE b/plugins/discovery-azure-classic/licenses/jackson-NOTICE similarity index 100% rename from plugins/discovery-azure/licenses/jackson-NOTICE rename to plugins/discovery-azure-classic/licenses/jackson-NOTICE diff --git a/plugins/discovery-azure/licenses/jackson-core-asl-1.9.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/jackson-core-asl-1.9.2.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jackson-core-asl-1.9.2.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jackson-core-asl-1.9.2.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jackson-jaxrs-1.9.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/jackson-jaxrs-1.9.2.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jackson-jaxrs-1.9.2.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jackson-jaxrs-1.9.2.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jackson-mapper-asl-1.9.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/jackson-mapper-asl-1.9.2.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jackson-mapper-asl-1.9.2.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jackson-mapper-asl-1.9.2.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jackson-xc-1.9.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/jackson-xc-1.9.2.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jackson-xc-1.9.2.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jackson-xc-1.9.2.jar.sha1 diff --git a/plugins/discovery-azure/licenses/javax.inject-1.jar.sha1 b/plugins/discovery-azure-classic/licenses/javax.inject-1.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/javax.inject-1.jar.sha1 rename to plugins/discovery-azure-classic/licenses/javax.inject-1.jar.sha1 diff --git a/plugins/discovery-azure/licenses/javax.inject-LICENSE.txt b/plugins/discovery-azure-classic/licenses/javax.inject-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/javax.inject-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/javax.inject-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/javax.inject-NOTICE.txt b/plugins/discovery-azure-classic/licenses/javax.inject-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/javax.inject-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/javax.inject-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/jaxb-LICENSE.txt b/plugins/discovery-azure-classic/licenses/jaxb-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/jaxb-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/jaxb-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/jaxb-NOTICE.txt b/plugins/discovery-azure-classic/licenses/jaxb-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/jaxb-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/jaxb-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/jaxb-api-2.2.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/jaxb-api-2.2.2.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jaxb-api-2.2.2.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jaxb-api-2.2.2.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jaxb-impl-2.2.3-1.jar.sha1 b/plugins/discovery-azure-classic/licenses/jaxb-impl-2.2.3-1.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jaxb-impl-2.2.3-1.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jaxb-impl-2.2.3-1.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jersey-LICENSE.txt b/plugins/discovery-azure-classic/licenses/jersey-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/jersey-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/jersey-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/jersey-NOTICE.txt b/plugins/discovery-azure-classic/licenses/jersey-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/jersey-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/jersey-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/jersey-client-1.13.jar.sha1 b/plugins/discovery-azure-classic/licenses/jersey-client-1.13.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jersey-client-1.13.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jersey-client-1.13.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jersey-core-1.13.jar.sha1 b/plugins/discovery-azure-classic/licenses/jersey-core-1.13.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jersey-core-1.13.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jersey-core-1.13.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jersey-json-1.13.jar.sha1 b/plugins/discovery-azure-classic/licenses/jersey-json-1.13.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jersey-json-1.13.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jersey-json-1.13.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jettison-1.1.jar.sha1 b/plugins/discovery-azure-classic/licenses/jettison-1.1.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/jettison-1.1.jar.sha1 rename to plugins/discovery-azure-classic/licenses/jettison-1.1.jar.sha1 diff --git a/plugins/discovery-azure/licenses/jettison-LICENSE.txt b/plugins/discovery-azure-classic/licenses/jettison-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/jettison-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/jettison-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/jettison-NOTICE.txt b/plugins/discovery-azure-classic/licenses/jettison-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/jettison-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/jettison-NOTICE.txt diff --git a/plugins/discovery-azure/licenses/mail-1.4.5.jar.sha1 b/plugins/discovery-azure-classic/licenses/mail-1.4.5.jar.sha1 similarity index 100% rename from plugins/discovery-azure/licenses/mail-1.4.5.jar.sha1 rename to plugins/discovery-azure-classic/licenses/mail-1.4.5.jar.sha1 diff --git a/plugins/discovery-azure/licenses/mail-LICENSE.txt b/plugins/discovery-azure-classic/licenses/mail-LICENSE.txt similarity index 100% rename from plugins/discovery-azure/licenses/mail-LICENSE.txt rename to plugins/discovery-azure-classic/licenses/mail-LICENSE.txt diff --git a/plugins/discovery-azure/licenses/mail-NOTICE.txt b/plugins/discovery-azure-classic/licenses/mail-NOTICE.txt similarity index 100% rename from plugins/discovery-azure/licenses/mail-NOTICE.txt rename to plugins/discovery-azure-classic/licenses/mail-NOTICE.txt diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureDiscoveryModule.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureDiscoveryModule.java similarity index 76% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureDiscoveryModule.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureDiscoveryModule.java index 2c9c6e0a486..da684fd824d 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureDiscoveryModule.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureDiscoveryModule.java @@ -17,12 +17,11 @@ * under the License. */ -package org.elasticsearch.cloud.azure; +package org.elasticsearch.cloud.azure.classic; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.cloud.azure.management.AzureComputeService; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Management; -import org.elasticsearch.cloud.azure.management.AzureComputeServiceImpl; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceImpl; import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.Inject; @@ -31,7 +30,7 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin; +import org.elasticsearch.plugin.discovery.azure.classic.AzureDiscoveryPlugin; /** * Azure Module @@ -41,7 +40,7 @@ import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin; * to AzureComputeServiceImpl. * * - * @see org.elasticsearch.cloud.azure.management.AzureComputeServiceImpl + * @see AzureComputeServiceImpl */ public class AzureDiscoveryModule extends AbstractModule { protected final ESLogger logger; @@ -77,19 +76,19 @@ public class AzureDiscoveryModule extends AbstractModule { return false; } - if (isDefined(settings, Management.SUBSCRIPTION_ID_SETTING) && - isDefined(settings, Management.SERVICE_NAME_SETTING) && - isDefined(settings, Management.KEYSTORE_PATH_SETTING) && - isDefined(settings, Management.KEYSTORE_PASSWORD_SETTING)) { + if (isDefined(settings, AzureComputeService.Management.SUBSCRIPTION_ID_SETTING) && + isDefined(settings, AzureComputeService.Management.SERVICE_NAME_SETTING) && + isDefined(settings, AzureComputeService.Management.KEYSTORE_PATH_SETTING) && + isDefined(settings, AzureComputeService.Management.KEYSTORE_PASSWORD_SETTING)) { logger.trace("All required properties for Azure discovery are set!"); return true; } else { logger.debug("One or more Azure discovery settings are missing. " + "Check elasticsearch.yml file. Should have [{}], [{}], [{}] and [{}].", - Management.SUBSCRIPTION_ID_SETTING.getKey(), - Management.SERVICE_NAME_SETTING.getKey(), - Management.KEYSTORE_PATH_SETTING.getKey(), - Management.KEYSTORE_PASSWORD_SETTING.getKey()); + AzureComputeService.Management.SUBSCRIPTION_ID_SETTING.getKey(), + AzureComputeService.Management.SERVICE_NAME_SETTING.getKey(), + AzureComputeService.Management.KEYSTORE_PATH_SETTING.getKey(), + AzureComputeService.Management.KEYSTORE_PASSWORD_SETTING.getKey()); return false; } } diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureServiceDisableException.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureServiceDisableException.java similarity index 95% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureServiceDisableException.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureServiceDisableException.java index 487997d71b6..66488f90c31 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureServiceDisableException.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureServiceDisableException.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.cloud.azure; +package org.elasticsearch.cloud.azure.classic; public class AzureServiceDisableException extends IllegalStateException { public AzureServiceDisableException(String msg) { diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureServiceRemoteException.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureServiceRemoteException.java similarity index 95% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureServiceRemoteException.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureServiceRemoteException.java index 4bd4f1d67f1..c961c03ba71 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureServiceRemoteException.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/AzureServiceRemoteException.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.cloud.azure; +package org.elasticsearch.cloud.azure.classic; public class AzureServiceRemoteException extends IllegalStateException { public AzureServiceRemoteException(String msg) { diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/management/AzureComputeService.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeService.java similarity index 94% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/management/AzureComputeService.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeService.java index 526f98025b7..49e609aad80 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/management/AzureComputeService.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeService.java @@ -17,15 +17,15 @@ * under the License. */ -package org.elasticsearch.cloud.azure.management; +package org.elasticsearch.cloud.azure.classic.management; import com.microsoft.windowsazure.core.utils.KeyStoreType; import com.microsoft.windowsazure.management.compute.models.HostedServiceGetDetailedResponse; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.azure.AzureUnicastHostsProvider; -import org.elasticsearch.discovery.azure.AzureUnicastHostsProvider.Deployment; +import org.elasticsearch.discovery.azure.classic.AzureUnicastHostsProvider; +import org.elasticsearch.discovery.azure.classic.AzureUnicastHostsProvider.Deployment; import java.net.URI; import java.net.URISyntaxException; diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/management/AzureComputeServiceImpl.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceImpl.java similarity index 97% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/management/AzureComputeServiceImpl.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceImpl.java index 076ce52ff80..7d6f81ad0b6 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/management/AzureComputeServiceImpl.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceImpl.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.cloud.azure.management; +package org.elasticsearch.cloud.azure.classic.management; import com.microsoft.windowsazure.Configuration; import com.microsoft.windowsazure.core.Builder; @@ -28,7 +28,7 @@ import com.microsoft.windowsazure.management.compute.ComputeManagementService; import com.microsoft.windowsazure.management.compute.models.HostedServiceGetDetailedResponse; import com.microsoft.windowsazure.management.configuration.ManagementConfiguration; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.cloud.azure.AzureServiceRemoteException; +import org.elasticsearch.cloud.azure.classic.AzureServiceRemoteException; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/discovery/azure/AzureUnicastHostsProvider.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/discovery/azure/classic/AzureUnicastHostsProvider.java similarity index 96% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/discovery/azure/AzureUnicastHostsProvider.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/discovery/azure/classic/AzureUnicastHostsProvider.java index cb6c8238bf5..ed327a3a727 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/discovery/azure/AzureUnicastHostsProvider.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/discovery/azure/classic/AzureUnicastHostsProvider.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.discovery.azure; +package org.elasticsearch.discovery.azure.classic; import com.microsoft.windowsazure.management.compute.models.DeploymentSlot; import com.microsoft.windowsazure.management.compute.models.DeploymentStatus; @@ -25,10 +25,10 @@ import com.microsoft.windowsazure.management.compute.models.HostedServiceGetDeta import com.microsoft.windowsazure.management.compute.models.InstanceEndpoint; import com.microsoft.windowsazure.management.compute.models.RoleInstance; import org.elasticsearch.Version; -import org.elasticsearch.cloud.azure.AzureServiceDisableException; -import org.elasticsearch.cloud.azure.AzureServiceRemoteException; -import org.elasticsearch.cloud.azure.management.AzureComputeService; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Discovery; +import org.elasticsearch.cloud.azure.classic.AzureServiceDisableException; +import org.elasticsearch.cloud.azure.classic.AzureServiceRemoteException; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Discovery; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractComponent; diff --git a/plugins/discovery-azure/src/main/java/org/elasticsearch/plugin/discovery/azure/AzureDiscoveryPlugin.java b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java similarity index 82% rename from plugins/discovery-azure/src/main/java/org/elasticsearch/plugin/discovery/azure/AzureDiscoveryPlugin.java rename to plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java index 9c1df5b4c8d..a7e1816fff0 100644 --- a/plugins/discovery-azure/src/main/java/org/elasticsearch/plugin/discovery/azure/AzureDiscoveryPlugin.java +++ b/plugins/discovery-azure-classic/src/main/java/org/elasticsearch/plugin/discovery/azure/classic/AzureDiscoveryPlugin.java @@ -17,18 +17,18 @@ * under the License. */ -package org.elasticsearch.plugin.discovery.azure; +package org.elasticsearch.plugin.discovery.azure.classic; -import org.elasticsearch.cloud.azure.AzureDiscoveryModule; -import org.elasticsearch.cloud.azure.management.AzureComputeService; +import org.elasticsearch.cloud.azure.classic.AzureDiscoveryModule; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService; import org.elasticsearch.common.inject.Module; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.azure.AzureUnicastHostsProvider; +import org.elasticsearch.discovery.azure.classic.AzureUnicastHostsProvider; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.plugins.Plugin; @@ -45,7 +45,9 @@ public class AzureDiscoveryPlugin extends Plugin { public AzureDiscoveryPlugin(Settings settings) { this.settings = settings; - logger.trace("starting azure discovery plugin..."); + DeprecationLogger deprecationLogger = new DeprecationLogger(logger); + deprecationLogger.deprecated("azure classic discovery plugin is deprecated. Use azure arm discovery plugin instead"); + logger.trace("starting azure classic discovery plugin..."); } @Override diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureComputeServiceTestCase.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java similarity index 89% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureComputeServiceTestCase.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java index 0c57ec3f16e..e9d16408537 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureComputeServiceTestCase.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java @@ -17,14 +17,14 @@ * under the License. */ -package org.elasticsearch.cloud.azure; +package org.elasticsearch.cloud.azure.classic; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Discovery; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Management; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Discovery; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Management; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.Node; -import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin; +import org.elasticsearch.plugin.discovery.azure.classic.AzureDiscoveryPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AzureComputeServiceSimpleMock.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AzureComputeServiceSimpleMock.java similarity index 95% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AzureComputeServiceSimpleMock.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AzureComputeServiceSimpleMock.java index 26843eba532..66e853b5953 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AzureComputeServiceSimpleMock.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AzureComputeServiceSimpleMock.java @@ -17,14 +17,14 @@ * under the License. */ -package org.elasticsearch.cloud.azure; +package org.elasticsearch.cloud.azure.classic; import com.microsoft.windowsazure.management.compute.models.DeploymentSlot; import com.microsoft.windowsazure.management.compute.models.DeploymentStatus; import com.microsoft.windowsazure.management.compute.models.HostedServiceGetDetailedResponse; import com.microsoft.windowsazure.management.compute.models.InstanceEndpoint; import com.microsoft.windowsazure.management.compute.models.RoleInstance; -import org.elasticsearch.cloud.azure.management.AzureComputeServiceAbstractMock; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceAbstractMock; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.CollectionUtils; diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AzureComputeServiceTwoNodesMock.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AzureComputeServiceTwoNodesMock.java similarity index 96% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AzureComputeServiceTwoNodesMock.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AzureComputeServiceTwoNodesMock.java index bf7589c8836..d75ce22d55c 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AzureComputeServiceTwoNodesMock.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/AzureComputeServiceTwoNodesMock.java @@ -17,14 +17,14 @@ * under the License. */ -package org.elasticsearch.cloud.azure; +package org.elasticsearch.cloud.azure.classic; import com.microsoft.windowsazure.management.compute.models.DeploymentSlot; import com.microsoft.windowsazure.management.compute.models.DeploymentStatus; import com.microsoft.windowsazure.management.compute.models.HostedServiceGetDetailedResponse; import com.microsoft.windowsazure.management.compute.models.InstanceEndpoint; import com.microsoft.windowsazure.management.compute.models.RoleInstance; -import org.elasticsearch.cloud.azure.management.AzureComputeServiceAbstractMock; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeServiceAbstractMock; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/management/AzureComputeServiceAbstractMock.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceAbstractMock.java similarity index 96% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/management/AzureComputeServiceAbstractMock.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceAbstractMock.java index c11060a84a9..33f40a9159a 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/management/AzureComputeServiceAbstractMock.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceAbstractMock.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.cloud.azure.management; +package org.elasticsearch.cloud.azure.classic.management; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.component.AbstractLifecycleComponent; diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryClusterFormationTests.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java similarity index 98% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryClusterFormationTests.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java index 0d1de07ed64..505f2d8b0db 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryClusterFormationTests.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java @@ -17,14 +17,14 @@ * under the License. */ -package org.elasticsearch.discovery.azure; +package org.elasticsearch.discovery.azure.classic; import com.microsoft.windowsazure.management.compute.models.DeploymentSlot; import com.microsoft.windowsazure.management.compute.models.DeploymentStatus; import com.sun.net.httpserver.Headers; import com.sun.net.httpserver.HttpsConfigurator; import com.sun.net.httpserver.HttpsServer; -import org.elasticsearch.cloud.azure.management.AzureComputeService; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.logging.Loggers; @@ -33,7 +33,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.env.Environment; import org.elasticsearch.node.Node; -import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin; +import org.elasticsearch.plugin.discovery.azure.classic.AzureDiscoveryPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.transport.TransportSettings; diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryRestIT.java similarity index 96% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryRestIT.java index 131f73d1ca9..cb04842cb4a 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureDiscoveryRestIT.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryRestIT.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.discovery.azure; +package org.elasticsearch.discovery.azure.classic; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureMinimumMasterNodesTests.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureMinimumMasterNodesTests.java similarity index 84% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureMinimumMasterNodesTests.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureMinimumMasterNodesTests.java index 46c3f8af7c2..72e1f2da791 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureMinimumMasterNodesTests.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureMinimumMasterNodesTests.java @@ -17,11 +17,11 @@ * under the License. */ -package org.elasticsearch.discovery.azure; +package org.elasticsearch.discovery.azure.classic; import org.apache.lucene.util.LuceneTestCase.AwaitsFix; -import org.elasticsearch.cloud.azure.AbstractAzureComputeServiceTestCase; -import org.elasticsearch.cloud.azure.AzureComputeServiceTwoNodesMock; +import org.elasticsearch.cloud.azure.classic.AbstractAzureComputeServiceTestCase; +import org.elasticsearch.cloud.azure.classic.AzureComputeServiceTwoNodesMock; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.MasterNotDiscoveredException; import org.elasticsearch.discovery.zen.ZenDiscovery; @@ -63,20 +63,23 @@ public class AzureMinimumMasterNodesTests extends AbstractAzureComputeServiceTes logger.info("--> start data node / non master node"); internalCluster().startNode(); try { - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").get().getState().nodes().getMasterNodeId(), + nullValue()); fail("should not be able to find master"); } catch (MasterNotDiscoveredException e) { // all is well, no master elected } logger.info("--> start another node"); internalCluster().startNode(); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); logger.info("--> stop master node"); internalCluster().stopCurrentMasterNode(); try { - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + nullValue()); fail("should not be able to find master"); } catch (MasterNotDiscoveredException e) { // all is well, no master elected @@ -84,6 +87,7 @@ public class AzureMinimumMasterNodesTests extends AbstractAzureComputeServiceTes logger.info("--> start another node"); internalCluster().startNode(); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); } } diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureSimpleTests.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureSimpleTests.java similarity index 82% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureSimpleTests.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureSimpleTests.java index 7a85909a13e..d2234632122 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureSimpleTests.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureSimpleTests.java @@ -17,12 +17,12 @@ * under the License. */ -package org.elasticsearch.discovery.azure; +package org.elasticsearch.discovery.azure.classic; -import org.elasticsearch.cloud.azure.AbstractAzureComputeServiceTestCase; -import org.elasticsearch.cloud.azure.AzureComputeServiceSimpleMock; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Discovery; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Management; +import org.elasticsearch.cloud.azure.classic.AbstractAzureComputeServiceTestCase; +import org.elasticsearch.cloud.azure.classic.AzureComputeServiceSimpleMock; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Discovery; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Management; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; @@ -38,14 +38,15 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase { super(AzureComputeServiceSimpleMock.TestPlugin.class); } - public void testOneNodeDhouldRunUsingPrivateIp() { + public void testOneNodeShouldRunUsingPrivateIp() { Settings.Builder settings = Settings.builder() .put(Management.SERVICE_NAME_SETTING.getKey(), "dummy") .put(Discovery.HOST_TYPE_SETTING.getKey(), "private_ip"); logger.info("--> start one node"); internalCluster().startNode(settings); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); // We expect having 1 node as part of the cluster, let's test that checkNumberOfNodes(1); @@ -58,7 +59,8 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase { logger.info("--> start one node"); internalCluster().startNode(settings); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); // We expect having 1 node as part of the cluster, let's test that checkNumberOfNodes(1); diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTests.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureTwoStartedNodesTests.java similarity index 78% rename from plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTests.java rename to plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureTwoStartedNodesTests.java index 6431696c2eb..35844c9b383 100644 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/discovery/azure/AzureTwoStartedNodesTests.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureTwoStartedNodesTests.java @@ -17,12 +17,12 @@ * under the License. */ -package org.elasticsearch.discovery.azure; +package org.elasticsearch.discovery.azure.classic; -import org.elasticsearch.cloud.azure.AbstractAzureComputeServiceTestCase; -import org.elasticsearch.cloud.azure.AzureComputeServiceTwoNodesMock; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Discovery; -import org.elasticsearch.cloud.azure.management.AzureComputeService.Management; +import org.elasticsearch.cloud.azure.classic.AbstractAzureComputeServiceTestCase; +import org.elasticsearch.cloud.azure.classic.AzureComputeServiceTwoNodesMock; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Discovery; +import org.elasticsearch.cloud.azure.classic.management.AzureComputeService.Management; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; @@ -46,11 +46,13 @@ public class AzureTwoStartedNodesTests extends AbstractAzureComputeServiceTestCa logger.info("--> start first node"); internalCluster().startNode(settings); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); logger.info("--> start another node"); internalCluster().startNode(settings); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); // We expect having 2 nodes as part of the cluster, let's test that checkNumberOfNodes(2); @@ -64,11 +66,13 @@ public class AzureTwoStartedNodesTests extends AbstractAzureComputeServiceTestCa logger.info("--> start first node"); internalCluster().startNode(settings); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); logger.info("--> start another node"); internalCluster().startNode(settings); - assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue()); + assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").get().getState().nodes().getMasterNodeId(), + notNullValue()); // We expect having 2 nodes as part of the cluster, let's test that checkNumberOfNodes(2); diff --git a/plugins/discovery-azure-classic/src/test/resources/rest-api-spec/test/discovery_azure_classic/10_basic.yaml b/plugins/discovery-azure-classic/src/test/resources/rest-api-spec/test/discovery_azure_classic/10_basic.yaml new file mode 100644 index 00000000000..ea042d8a52d --- /dev/null +++ b/plugins/discovery-azure-classic/src/test/resources/rest-api-spec/test/discovery_azure_classic/10_basic.yaml @@ -0,0 +1,13 @@ +# Integration tests for Azure Classic Discovery component +# +"Discovery Azure Classic loaded": + - do: + cluster.state: {} + + # Get master node id + - set: { master_node: master } + + - do: + nodes.info: {} + + - match: { nodes.$master.plugins.0.name: discovery-azure-classic } diff --git a/plugins/discovery-azure/LICENSE.txt b/plugins/discovery-azure/LICENSE.txt deleted file mode 100644 index d6456956733..00000000000 --- a/plugins/discovery-azure/LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/plugins/discovery-azure/src/test/resources/rest-api-spec/test/discovery_azure/10_basic.yaml b/plugins/discovery-azure/src/test/resources/rest-api-spec/test/discovery_azure/10_basic.yaml deleted file mode 100644 index 7a5acd1f001..00000000000 --- a/plugins/discovery-azure/src/test/resources/rest-api-spec/test/discovery_azure/10_basic.yaml +++ /dev/null @@ -1,13 +0,0 @@ -# Integration tests for Azure Discovery component -# -"Discovery Azure loaded": - - do: - cluster.state: {} - - # Get master node id - - set: { master_node: master } - - - do: - nodes.info: {} - - - match: { nodes.$master.plugins.0.name: discovery-azure } diff --git a/plugins/repository-azure/LICENSE.txt b/plugins/repository-azure/LICENSE.txt deleted file mode 100644 index d6456956733..00000000000 --- a/plugins/repository-azure/LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/plugins/store-smb/LICENSE.txt b/plugins/store-smb/LICENSE.txt deleted file mode 100644 index d6456956733..00000000000 --- a/plugins/store-smb/LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash index 336fa3ee305..c17df96937a 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash @@ -209,7 +209,7 @@ fi install_and_check_plugin discovery gce google-api-client-*.jar } -@test "[$GROUP] install discovery-azure plugin" { +@test "[$GROUP] install discovery-azure-classic plugin" { install_and_check_plugin discovery azure azure-core-*.jar } @@ -341,8 +341,8 @@ fi remove_plugin discovery-gce } -@test "[$GROUP] remove discovery-azure plugin" { - remove_plugin discovery-azure +@test "[$GROUP] remove discovery-azure-classic plugin" { + remove_plugin discovery-azure-classic } @test "[$GROUP] remove discovery-ec2 plugin" { diff --git a/settings.gradle b/settings.gradle index c0d7a72b2a5..6588b605a9d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -31,7 +31,7 @@ List projects = [ 'plugins:analysis-phonetic', 'plugins:analysis-smartcn', 'plugins:analysis-stempel', - 'plugins:discovery-azure', + 'plugins:discovery-azure-classic', 'plugins:discovery-ec2', 'plugins:discovery-gce', 'plugins:ingest-geoip', 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()) {