From 194a6b1df031a95977b6ca395a74a8b74d9af31d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 7 Oct 2016 11:27:47 +0200 Subject: [PATCH] Remove LocalTransport in favor of MockTcpTransport (#20695) This change proposes the removal of all non-tcp transport implementations. The mock transport can be used by default to run tests instead of local transport that has roughly the same performance compared to TCP or at least not noticeably slower. This is a master only change, deprecation notice in 5.x will be committed as a separate change. --- TESTING.asciidoc | 16 - .../routing/allocation/Allocators.java | 10 +- .../transport/TransportClientBenchmark.java | 4 +- .../cluster/stats/ClusterStatsNodes.java | 6 +- .../cluster/node/DiscoveryNode.java | 6 +- .../cluster/node/DiscoveryNodeFilters.java | 10 +- .../common/network/NetworkModule.java | 5 +- .../transport/BoundTransportAddress.java | 8 +- .../transport/InetSocketTransportAddress.java | 127 ------ .../transport/LocalTransportAddress.java | 116 ----- .../common/transport/TransportAddress.java | 107 ++++- .../TransportAddressSerializers.java | 67 --- .../discovery/zen/NodeJoinController.java | 7 +- .../elasticsearch/plugins/NetworkPlugin.java | 2 +- .../rest/action/cat/RestNodeAttrsAction.java | 8 +- .../rest/action/cat/RestNodesAction.java | 13 +- .../rest/action/cat/RestTasksAction.java | 8 +- .../rest/action/cat/RestThreadPoolAction.java | 8 +- .../transport/ActionTransportException.java | 14 +- .../elasticsearch/transport/TcpTransport.java | 22 +- .../elasticsearch/transport/Transports.java | 2 - .../transport/local/LocalTransport.java | 430 ------------------ .../local/LocalTransportChannel.java | 139 ------ .../ExceptionSerializationTests.java | 16 +- .../action/IndicesRequestIT.java | 3 +- .../ClusterAllocationExplanationTests.java | 3 +- .../cluster/node/stats/NodeStatsTests.java | 3 +- .../node/tasks/TaskManagerTestCase.java | 9 +- .../admin/cluster/node/tasks/TasksIT.java | 7 +- .../IndicesShardStoreResponseTests.java | 7 +- .../shrink/TransportShrinkActionTests.java | 3 +- .../ingest/IngestProxyActionFilterTests.java | 3 +- .../TransportMultiSearchActionTests.java | 7 +- .../TransportBroadcastByNodeActionTests.java | 3 +- .../master/IndexingMasterFailoverIT.java | 3 +- .../TransportMasterNodeActionTests.java | 5 +- .../nodes/TransportNodesActionTests.java | 3 +- .../BroadcastReplicationTests.java | 12 +- .../ClusterStateCreationUtils.java | 4 +- .../bootstrap/BootstrapCheckTests.java | 25 +- .../TransportClientHeadersTests.java | 17 +- .../client/transport/TransportClientIT.java | 9 +- .../TransportClientNodesServiceTests.java | 20 +- .../cluster/ClusterChangedEventTests.java | 3 +- .../cluster/ClusterStateDiffIT.java | 9 +- .../cluster/ClusterStateTests.java | 5 +- .../elasticsearch/cluster/DiskUsageTests.java | 13 +- .../cluster/MinimumMasterNodesIT.java | 8 +- .../elasticsearch/cluster/NoMasterNodeIT.java | 9 +- .../cluster/NodeConnectionsServiceTests.java | 3 +- .../cluster/SpecificMasterNodesIT.java | 8 +- .../MetaDataCreateIndexServiceTests.java | 3 +- .../node/DiscoveryNodeFiltersTests.java | 31 +- .../cluster/node/DiscoveryNodesTests.java | 3 +- .../cluster/routing/PrimaryAllocationIT.java | 17 +- .../NodeVersionAllocationDeciderTests.java | 13 +- .../allocation/SameShardRoutingTests.java | 7 +- .../decider/DiskThresholdDeciderTests.java | 11 +- .../DiskThresholdDeciderUnitTests.java | 13 +- .../ClusterSerializationTests.java | 4 +- .../ClusterStateToStringTests.java | 3 +- .../cluster/service/ClusterServiceIT.java | 8 +- .../cluster/service/ClusterServiceTests.java | 3 +- .../common/network/NetworkModuleTests.java | 17 +- .../transport/BoundTransportAddressTests.java | 9 +- ...usterStatePublishResponseHandlerTests.java | 3 +- .../DiscoveryWithServiceDisruptionsIT.java | 1 - .../discovery/ZenFaultDetectionTests.java | 12 +- .../discovery/ZenUnicastDiscoveryIT.java | 1 - .../zen/ElectMasterServiceTests.java | 5 +- .../zen/NodeJoinControllerTests.java | 13 +- ...eRemovalClusterStateTaskExecutorTests.java | 3 +- .../discovery/zen/ZenDiscoveryIT.java | 11 +- .../discovery/zen/ZenDiscoveryUnitTests.java | 13 +- .../discovery/zen/ZenPingTests.java | 3 +- .../zen/ping/unicast/UnicastZenPingTests.java | 7 +- .../PendingClusterStatesQueueTests.java | 3 +- .../PublishClusterStateActionTests.java | 2 +- .../gateway/AsyncShardFetchTests.java | 5 +- .../elasticsearch/http/HttpServerTests.java | 3 +- .../index/TransportIndexFailuresIT.java | 1 - .../mapper/DynamicMappingDisabledTests.java | 12 +- .../ESIndexLevelReplicationTestCase.java | 3 +- .../index/shard/IndexShardIT.java | 3 +- .../index/shard/IndexShardTests.java | 17 +- .../IndexingMemoryControllerTests.java | 3 +- ...dicesLifecycleListenerSingleNodeTests.java | 3 +- ...ClusterStateServiceRandomUpdatesTests.java | 3 +- .../recovery/RecoverySourceHandlerTests.java | 22 +- .../indices/recovery/RecoveryStatusTests.java | 3 +- .../indices/recovery/RecoveryTargetTests.java | 3 +- .../recovery/StartRecoveryRequestTests.java | 5 +- .../indices/state/RareClusterStateIT.java | 11 +- .../indices/store/IndicesStoreTests.java | 11 +- .../ingest/PipelineStoreTests.java | 7 +- .../nodesinfo/NodeInfoStreamingTests.java | 5 +- .../bucket/DateHistogramOffsetIT.java | 9 - .../DedicatedClusterSnapshotRestoreIT.java | 10 +- .../local/SimpleLocalTransportTests.java | 37 -- .../java/org/elasticsearch/tribe/TribeIT.java | 9 +- docs/reference/modules/transport.asciidoc | 7 - .../reindex/AsyncBulkByScrollActionTests.java | 3 +- .../ReindexFromRemoteWhitelistTests.java | 5 +- .../netty3/Netty3HttpServerTransport.java | 17 +- .../transport/netty3/Netty3Transport.java | 6 +- .../elasticsearch/ESNetty3IntegTestCase.java | 1 - .../netty3/Netty3HttpPublishPortTests.java | 12 +- .../netty3/Netty3HttpRequestSizeLimitIT.java | 12 +- .../Netty3HttpServerPipeliningTests.java | 6 +- .../netty3/Netty3PipeliningDisabledIT.java | 5 +- .../Netty3SizeHeaderFrameDecoderTests.java | 3 +- ...Netty3TransportMultiPortIntegrationIT.java | 13 +- .../Netty3TransportPublishAddressIT.java | 5 +- .../netty3/SimpleNetty3TransportTests.java | 4 +- .../netty4/Netty4HttpServerTransport.java | 17 +- .../transport/netty4/Netty4Transport.java | 6 +- .../elasticsearch/ESNetty4IntegTestCase.java | 1 - .../netty4/Netty4HttpPublishPortTests.java | 12 +- .../netty4/Netty4HttpRequestSizeLimitIT.java | 14 +- .../Netty4HttpServerPipeliningTests.java | 10 +- .../Netty4HttpServerTransportTests.java | 4 +- .../netty4/Netty4PipeliningDisabledIT.java | 5 +- .../netty4/Netty4PipeliningEnabledIT.java | 6 +- .../Netty4SizeHeaderFrameDecoderTests.java | 3 +- ...Netty4TransportMultiPortIntegrationIT.java | 13 +- .../Netty4TransportPublishAddressIT.java | 5 +- .../netty4/SimpleNetty4TransportTests.java | 4 +- .../AzureDiscoveryClusterFormationTests.java | 1 - .../Ec2DiscoveryClusterFormationTests.java | 1 - .../discovery/ec2/Ec2DiscoveryTests.java | 57 ++- .../discovery/gce/GceDiscoverTests.java | 1 - .../discovery/gce/GceDiscoveryTests.java | 2 +- .../elasticsearch/tribe/TribeUnitTests.java | 15 +- .../smoketest/ESSmokeClientTestCase.java | 4 +- .../cluster/ESAllocationTestCase.java | 9 +- .../MockInternalClusterInfoService.java | 4 +- .../index/shard/IndexShardTestCase.java | 3 +- .../test/ClusterServiceUtils.java | 3 +- .../elasticsearch/test/ESIntegTestCase.java | 61 +-- .../test/ESSingleNodeTestCase.java | 11 +- .../org/elasticsearch/test/ESTestCase.java | 18 + .../test/ExternalTestCluster.java | 3 +- .../test/InternalTestCluster.java | 3 +- .../transport/AssertingLocalTransport.java | 110 ----- .../test/transport/MockTransportService.java | 19 +- .../AbstractSimpleTransportTestCase.java | 7 +- .../AssertingTransportInterceptor.java | 126 +++++ .../transport/MockTcpTransport.java | 4 +- .../transport/MockTransportClient.java | 17 +- .../org/elasticsearch/node/MockNodeTests.java | 4 +- .../test/test/InternalTestClusterTests.java | 22 +- 151 files changed, 797 insertions(+), 1666 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java delete mode 100644 core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java delete mode 100644 core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java delete mode 100644 core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java delete mode 100644 core/src/main/java/org/elasticsearch/transport/local/LocalTransportChannel.java delete mode 100644 core/src/test/java/org/elasticsearch/transport/local/SimpleLocalTransportTests.java delete mode 100644 test/framework/src/main/java/org/elasticsearch/test/transport/AssertingLocalTransport.java create mode 100644 test/framework/src/main/java/org/elasticsearch/transport/AssertingTransportInterceptor.java diff --git a/TESTING.asciidoc b/TESTING.asciidoc index dd6c093047a..40e148260f3 100644 --- a/TESTING.asciidoc +++ b/TESTING.asciidoc @@ -16,22 +16,6 @@ following: gradle assemble ----------------------------- -== Other test options - -To disable and enable network transport, set the `tests.es.node.mode` system property. - -Use network transport: - ------------------------------------- --Dtests.es.node.mode=network ------------------------------------- - -Use local transport (default since 1.3): - -------------------------------------- --Dtests.es.node.mode=local -------------------------------------- - === Running Elasticsearch from a checkout In order to run Elasticsearch from source without building a package, you can diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java index 860137cf559..4d8f7cfeaac 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java @@ -31,15 +31,18 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.gateway.GatewayAllocator; import java.lang.reflect.InvocationTargetException; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; public final class Allocators { private static class NoopGatewayAllocator extends GatewayAllocator { @@ -91,8 +94,11 @@ public final class Allocators { } + private static final AtomicInteger portGenerator = new AtomicInteger(); + public static DiscoveryNode newNode(String nodeId, Map attributes) { - return new DiscoveryNode("", nodeId, LocalTransportAddress.buildUnique(), attributes, Sets.newHashSet(DiscoveryNode.Role.MASTER, + return new DiscoveryNode("", nodeId, new TransportAddress(TransportAddress.META_ADDRESS, + portGenerator.incrementAndGet()), attributes, Sets.newHashSet(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA), Version.CURRENT); } } diff --git a/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java b/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java index c38234ef302..6d6e5ade827 100644 --- a/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java +++ b/client/benchmark/src/main/java/org/elasticsearch/client/benchmark/transport/TransportClientBenchmark.java @@ -27,7 +27,7 @@ import org.elasticsearch.client.benchmark.ops.bulk.BulkRequestExecutor; import org.elasticsearch.client.benchmark.ops.search.SearchRequestExecutor; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugin.noop.NoopPlugin; import org.elasticsearch.plugin.noop.action.bulk.NoopBulkAction; @@ -51,7 +51,7 @@ public final class TransportClientBenchmark extends AbstractBenchmark(size); for (int i = 0; i < size; i++) { @@ -242,7 +240,7 @@ public class DiscoveryNode implements Writeable, ToXContent { out.writeString(ephemeralId); out.writeString(hostName); out.writeString(hostAddress); - addressToStream(out, address); + address.writeTo(out); out.writeVInt(attributes.size()); for (Map.Entry entry : attributes.entrySet()) { out.writeString(entry.getKey()); diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeFilters.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeFilters.java index e8ede54f4a2..7247dff5d8c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeFilters.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeFilters.java @@ -24,7 +24,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import java.util.HashMap; import java.util.Map; @@ -82,8 +82,8 @@ public class DiscoveryNodeFilters { if ("_ip".equals(attr)) { // We check both the host_ip or the publish_ip String publishAddress = null; - if (node.getAddress() instanceof InetSocketTransportAddress) { - publishAddress = NetworkAddress.format(((InetSocketTransportAddress) node.getAddress()).address().getAddress()); + if (node.getAddress() instanceof TransportAddress) { + publishAddress = NetworkAddress.format(node.getAddress().address().getAddress()); } boolean match = matchByIP(values, node.getHostAddress(), publishAddress); @@ -116,8 +116,8 @@ public class DiscoveryNodeFilters { } else if ("_publish_ip".equals(attr)) { // We check explicitly only the publish_ip String address = null; - if (node.getAddress() instanceof InetSocketTransportAddress) { - address = NetworkAddress.format(((InetSocketTransportAddress) node.getAddress()).address().getAddress()); + if (node.getAddress() instanceof TransportAddress) { + address = NetworkAddress.format(node.getAddress().address().getAddress()); } boolean match = matchByIP(values, address, null); diff --git a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java index 60bb35d8f1c..5c8f3ec65e8 100644 --- a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java +++ b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java @@ -45,10 +45,8 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.local.LocalTransport; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -112,7 +110,6 @@ public final class NetworkModule { NetworkService networkService) { this.settings = settings; this.transportClient = transportClient; - registerTransport(LOCAL_TRANSPORT, () -> new LocalTransport(settings, threadPool, namedWriteableRegistry, circuitBreakerService)); for (NetworkPlugin plugin : plugins) { if (transportClient == false && HTTP_ENABLED.get(settings)) { Map> httpTransportFactory = plugin.getHttpTransports(settings, threadPool, bigArrays, @@ -126,7 +123,7 @@ public final class NetworkModule { for (Map.Entry> entry : httpTransportFactory.entrySet()) { registerTransport(entry.getKey(), entry.getValue()); } - List transportInterceptors = plugin.getTransportInterceptors(); + List transportInterceptors = plugin.getTransportInterceptors(namedWriteableRegistry); for (TransportInterceptor interceptor : transportInterceptors) { registerTransportInterceptor(interceptor); } diff --git a/core/src/main/java/org/elasticsearch/common/transport/BoundTransportAddress.java b/core/src/main/java/org/elasticsearch/common/transport/BoundTransportAddress.java index 28e4506b953..336b9c536a1 100644 --- a/core/src/main/java/org/elasticsearch/common/transport/BoundTransportAddress.java +++ b/core/src/main/java/org/elasticsearch/common/transport/BoundTransportAddress.java @@ -68,18 +68,18 @@ public class BoundTransportAddress implements Streamable { int boundAddressLength = in.readInt(); boundAddresses = new TransportAddress[boundAddressLength]; for (int i = 0; i < boundAddressLength; i++) { - boundAddresses[i] = TransportAddressSerializers.addressFromStream(in); + boundAddresses[i] = new TransportAddress(in); } - publishAddress = TransportAddressSerializers.addressFromStream(in); + publishAddress = new TransportAddress(in); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeInt(boundAddresses.length); for (TransportAddress address : boundAddresses) { - TransportAddressSerializers.addressToStream(out, address); + address.writeTo(out); } - TransportAddressSerializers.addressToStream(out, publishAddress); + publishAddress.writeTo(out); } @Override diff --git a/core/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java b/core/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java deleted file mode 100644 index 94c1a2390ac..00000000000 --- a/core/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java +++ /dev/null @@ -1,127 +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.transport; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.network.NetworkAddress; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; - -/** - * A transport address used for IP socket address (wraps {@link java.net.InetSocketAddress}). - */ -public final class InetSocketTransportAddress implements TransportAddress { - public static final short TYPE_ID = 1; - - private final InetSocketAddress address; - - public InetSocketTransportAddress(InetAddress address, int port) { - this(new InetSocketAddress(address, port)); - } - - public InetSocketTransportAddress(InetSocketAddress address) { - if (address == null) { - throw new IllegalArgumentException("InetSocketAddress must not be null"); - } - if (address.getAddress() == null) { - throw new IllegalArgumentException("Address must be resolved but wasn't - InetSocketAddress#getAddress() returned null"); - } - this.address = address; - } - - /** - * Read from a stream. - */ - public InetSocketTransportAddress(StreamInput in) throws IOException { - final int len = in.readByte(); - final byte[] a = new byte[len]; // 4 bytes (IPv4) or 16 bytes (IPv6) - in.readFully(a); - InetAddress inetAddress = InetAddress.getByAddress(a); - int port = in.readInt(); - this.address = new InetSocketAddress(inetAddress, port); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - byte[] bytes = address().getAddress().getAddress(); // 4 bytes (IPv4) or 16 bytes (IPv6) - out.writeByte((byte) bytes.length); // 1 byte - out.write(bytes, 0, bytes.length); - // don't serialize scope ids over the network!!!! - // these only make sense with respect to the local machine, and will only formulate - // the address incorrectly remotely. - out.writeInt(address.getPort()); - } - - @Override - public short uniqueAddressTypeId() { - return TYPE_ID; - } - - @Override - public boolean sameHost(TransportAddress other) { - return other instanceof InetSocketTransportAddress && - address.getAddress().equals(((InetSocketTransportAddress) other).address.getAddress()); - } - - @Override - public boolean isLoopbackOrLinkLocalAddress() { - return address.getAddress().isLinkLocalAddress() || address.getAddress().isLoopbackAddress(); - } - - @Override - public String getHost() { - return getAddress(); // just delegate no resolving - } - - @Override - public String getAddress() { - return NetworkAddress.format(address.getAddress()); - } - - @Override - public int getPort() { - return address.getPort(); - } - - public InetSocketAddress address() { - return this.address; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - InetSocketTransportAddress address1 = (InetSocketTransportAddress) o; - return address.equals(address1.address); - } - - @Override - public int hashCode() { - return address != null ? address.hashCode() : 0; - } - - @Override - public String toString() { - return NetworkAddress.format(address); - } -} diff --git a/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java b/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java deleted file mode 100644 index 48becc832da..00000000000 --- a/core/src/main/java/org/elasticsearch/common/transport/LocalTransportAddress.java +++ /dev/null @@ -1,116 +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.transport; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; - -/** - * - */ -public final class LocalTransportAddress implements TransportAddress { - public static final short TYPE_ID = 2; - - private static final AtomicLong transportAddressIdGenerator = new AtomicLong(); - - /** - * generates a new unique address - */ - public static LocalTransportAddress buildUnique() { - return new LocalTransportAddress(Long.toString(transportAddressIdGenerator.incrementAndGet())); - } - - private String id; - - public LocalTransportAddress(String id) { - this.id = id; - } - - /** - * Read from a stream. - */ - public LocalTransportAddress(StreamInput in) throws IOException { - id = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(id); - } - - public String id() { - return this.id; - } - - @Override - public short uniqueAddressTypeId() { - return TYPE_ID; - } - - @Override - public boolean sameHost(TransportAddress other) { - return other instanceof LocalTransportAddress && id.equals(((LocalTransportAddress) other).id); - } - - @Override - public boolean isLoopbackOrLinkLocalAddress() { - return false; - } - - @Override - public String getHost() { - return "local"; - } - - @Override - public String getAddress() { - return "0.0.0.0"; // see https://en.wikipedia.org/wiki/0.0.0.0 - } - - @Override - public int getPort() { - return 0; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - LocalTransportAddress that = (LocalTransportAddress) o; - - if (id != null ? !id.equals(that.id) : that.id != null) return false; - - return true; - } - - @Override - public int hashCode() { - return id != null ? id.hashCode() : 0; - } - - @Override - public String toString() { - return "local[" + id + "]"; - } -} diff --git a/core/src/main/java/org/elasticsearch/common/transport/TransportAddress.java b/core/src/main/java/org/elasticsearch/common/transport/TransportAddress.java index 08e8af2bffe..3656fc251c4 100644 --- a/core/src/main/java/org/elasticsearch/common/transport/TransportAddress.java +++ b/core/src/main/java/org/elasticsearch/common/transport/TransportAddress.java @@ -19,34 +19,113 @@ package org.elasticsearch.common.transport; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.network.NetworkAddress; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; /** - * + * A transport address used for IP socket address (wraps {@link java.net.InetSocketAddress}). */ -public interface TransportAddress extends Writeable { +public final class TransportAddress implements Writeable { /** - * Returns the host string for this transport address + * A non-routeable v4 meta transport address that can be used for + * testing or in scenarios where targets should be marked as non-applicable from a transport perspective. */ - String getHost(); + public static final InetAddress META_ADDRESS; + + static { + try { + META_ADDRESS = InetAddress.getByName("0.0.0.0"); + } catch (UnknownHostException e) { + throw new AssertionError(e); + } + } + + private final InetSocketAddress address; + + public TransportAddress(InetAddress address, int port) { + this(new InetSocketAddress(address, port)); + } + + public TransportAddress(InetSocketAddress address) { + if (address == null) { + throw new IllegalArgumentException("InetSocketAddress must not be null"); + } + if (address.getAddress() == null) { + throw new IllegalArgumentException("Address must be resolved but wasn't - InetSocketAddress#getAddress() returned null"); + } + this.address = address; + } /** - * Returns the address string for this transport address + * Read from a stream. */ - String getAddress(); + public TransportAddress(StreamInput in) throws IOException { + final int len = in.readByte(); + final byte[] a = new byte[len]; // 4 bytes (IPv4) or 16 bytes (IPv6) + in.readFully(a); + InetAddress inetAddress = InetAddress.getByAddress(a); + int port = in.readInt(); + this.address = new InetSocketAddress(inetAddress, port); + } - /** - * Returns the port of this transport address if applicable - */ - int getPort(); + @Override + public void writeTo(StreamOutput out) throws IOException { + byte[] bytes = address().getAddress().getAddress(); // 4 bytes (IPv4) or 16 bytes (IPv6) + out.writeByte((byte) bytes.length); // 1 byte + out.write(bytes, 0, bytes.length); + // don't serialize scope ids over the network!!!! + // these only make sense with respect to the local machine, and will only formulate + // the address incorrectly remotely. + out.writeInt(address.getPort()); + } - short uniqueAddressTypeId(); + public boolean sameHost(TransportAddress other) { + return address.getAddress().equals(other.address.getAddress()); + } - boolean sameHost(TransportAddress other); + public boolean isLoopbackOrLinkLocalAddress() { + return address.getAddress().isLinkLocalAddress() || address.getAddress().isLoopbackAddress(); + } - boolean isLoopbackOrLinkLocalAddress(); + public String getHost() { + return getAddress(); // just delegate no resolving + } - String toString(); + public String getAddress() { + return NetworkAddress.format(address.getAddress()); + } + + public int getPort() { + return address.getPort(); + } + + public InetSocketAddress address() { + return this.address; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TransportAddress address1 = (TransportAddress) o; + return address.equals(address1.address); + } + + @Override + public int hashCode() { + return address != null ? address.hashCode() : 0; + } + + @Override + public String toString() { + return NetworkAddress.format(address); + } } diff --git a/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java b/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java deleted file mode 100644 index 784bee52d63..00000000000 --- a/core/src/main/java/org/elasticsearch/common/transport/TransportAddressSerializers.java +++ /dev/null @@ -1,67 +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.transport; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import static java.util.Collections.unmodifiableMap; - -/** - * A global registry of all supported types of {@link TransportAddress}s. This registry is not open for modification by plugins. - */ -public abstract class TransportAddressSerializers { - private static final Map> ADDRESS_REGISTRY; - - static { - Map> registry = new HashMap<>(); - addAddressType(registry, InetSocketTransportAddress.TYPE_ID, InetSocketTransportAddress::new); - addAddressType(registry, LocalTransportAddress.TYPE_ID, LocalTransportAddress::new); - ADDRESS_REGISTRY = unmodifiableMap(registry); - } - - private static void addAddressType(Map> registry, short uniqueAddressTypeId, - Writeable.Reader address) { - if (registry.containsKey(uniqueAddressTypeId)) { - throw new IllegalStateException("Address [" + uniqueAddressTypeId + "] already bound"); - } - registry.put(uniqueAddressTypeId, address); - } - - public static TransportAddress addressFromStream(StreamInput input) throws IOException { - // TODO why don't we just use named writeables here? - short addressUniqueId = input.readShort(); - Writeable.Reader addressType = ADDRESS_REGISTRY.get(addressUniqueId); - if (addressType == null) { - throw new IOException("No transport address mapped to [" + addressUniqueId + "]"); - } - return addressType.read(input); - } - - public static void addressToStream(StreamOutput out, TransportAddress address) throws IOException { - out.writeShort(address.uniqueAddressTypeId()); - address.writeTo(out); - } -} diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 75cce695357..a4a51d6ab31 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -37,7 +37,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.membership.MembershipAction; @@ -385,7 +385,8 @@ public class NodeJoinController extends AbstractComponent { /** * a task indicated that the current node should become master, if no current master is known */ - private static final DiscoveryNode BECOME_MASTER_TASK = new DiscoveryNode("_BECOME_MASTER_TASK_", LocalTransportAddress.buildUnique(), + private static final DiscoveryNode BECOME_MASTER_TASK = new DiscoveryNode("_BECOME_MASTER_TASK_", + new TransportAddress(TransportAddress.META_ADDRESS, 0), Collections.emptyMap(), Collections.emptySet(), Version.CURRENT) { @Override public String toString() { @@ -398,7 +399,7 @@ public class NodeJoinController extends AbstractComponent { * it may be use in combination with {@link #BECOME_MASTER_TASK} */ private static final DiscoveryNode FINISH_ELECTION_TASK = new DiscoveryNode("_FINISH_ELECTION_", - LocalTransportAddress.buildUnique(), Collections.emptyMap(), Collections.emptySet(), Version.CURRENT) { + new TransportAddress(TransportAddress.META_ADDRESS, 0), Collections.emptyMap(), Collections.emptySet(), Version.CURRENT) { @Override public String toString() { return ""; // this is not really task , so don't log anything about it... diff --git a/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java b/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java index ee7187c8853..70bac7fac49 100644 --- a/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/NetworkPlugin.java @@ -42,7 +42,7 @@ public interface NetworkPlugin { * Returns a list of {@link TransportInterceptor} instances that are used to intercept incoming and outgoing * transport (inter-node) requests. This must not return null */ - default List getTransportInterceptors() { + default List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry) { return Collections.emptyList(); } diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java index 37c5cf26e42..a5f33d916fd 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java @@ -31,7 +31,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; @@ -108,11 +108,7 @@ public class RestNodeAttrsAction extends AbstractCatAction { table.addCell(info == null ? null : info.getProcess().getId()); table.addCell(node.getHostName()); table.addCell(node.getHostAddress()); - if (node.getAddress() instanceof InetSocketTransportAddress) { - table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort()); - } else { - table.addCell("-"); - } + table.addCell(node.getAddress().address().getPort()); table.addCell(attrEntry.getKey()); table.addCell(attrEntry.getValue()); table.endRow(); diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index d428b9db9c2..87cbe976089 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.Table; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpInfo; import org.elasticsearch.index.cache.query.QueryCacheStats; @@ -236,19 +235,11 @@ public class RestNodesAction extends AbstractCatAction { table.addCell(fullId ? node.getId() : Strings.substring(node.getId(), 0, 4)); table.addCell(info == null ? null : info.getProcess().getId()); table.addCell(node.getHostAddress()); - if (node.getAddress() instanceof InetSocketTransportAddress) { - table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort()); - } else { - table.addCell("-"); - } + table.addCell(node.getAddress().address().getPort()); final HttpInfo httpInfo = info == null ? null : info.getHttp(); if (httpInfo != null) { TransportAddress transportAddress = httpInfo.getAddress().publishAddress(); - if (transportAddress instanceof InetSocketTransportAddress) { - table.addCell(NetworkAddress.format(((InetSocketTransportAddress)transportAddress).address())); - } else { - table.addCell(transportAddress.toString()); - } + table.addCell(NetworkAddress.format(transportAddress.address())); } else { table.addCell("-"); } diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestTasksAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestTasksAction.java index a2bd220239a..2c490debea0 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestTasksAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestTasksAction.java @@ -29,7 +29,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -144,11 +144,7 @@ public class RestTasksAction extends AbstractCatAction { // Node information. Note that the node may be null because it has left the cluster between when we got this response and now. table.addCell(fullId ? nodeId : Strings.substring(nodeId, 0, 4)); table.addCell(node == null ? "-" : node.getHostAddress()); - if (node != null && node.getAddress() instanceof InetSocketTransportAddress) { - table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort()); - } else { - table.addCell("-"); - } + table.addCell(node.getAddress().address().getPort()); table.addCell(node == null ? "-" : node.getName()); table.addCell(node == null ? "-" : node.getVersion().toString()); diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java index ab77ded35e8..30c3de151c2 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java @@ -34,7 +34,7 @@ import org.elasticsearch.common.Table; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; @@ -195,11 +195,7 @@ public class RestThreadPoolAction extends AbstractCatAction { table.addCell(info == null ? null : info.getProcess().getId()); table.addCell(node.getHostName()); table.addCell(node.getHostAddress()); - if (node.getAddress() instanceof InetSocketTransportAddress) { - table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort()); - } else { - table.addCell("-"); - } + table.addCell(node.getAddress().address().getPort()); final ThreadPoolStats.Stats poolStats = entry.getValue(); final ThreadPool.Info poolInfo = poolThreadInfo.get(entry.getKey()); diff --git a/core/src/main/java/org/elasticsearch/transport/ActionTransportException.java b/core/src/main/java/org/elasticsearch/transport/ActionTransportException.java index 9b0e887c219..e35e77cba3e 100644 --- a/core/src/main/java/org/elasticsearch/transport/ActionTransportException.java +++ b/core/src/main/java/org/elasticsearch/transport/ActionTransportException.java @@ -22,7 +22,6 @@ package org.elasticsearch.transport; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.transport.TransportAddressSerializers; import java.io.IOException; @@ -39,11 +38,7 @@ public class ActionTransportException extends TransportException { public ActionTransportException(StreamInput in) throws IOException { super(in); - if (in.readBoolean()) { - address = TransportAddressSerializers.addressFromStream(in); - } else { - address = null; - } + address = in.readOptionalWriteable(TransportAddress::new); action = in.readOptionalString(); } @@ -62,12 +57,7 @@ public class ActionTransportException extends TransportException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - if (address != null) { - out.writeBoolean(true); - TransportAddressSerializers.addressToStream(out, address); - } else { - out.writeBoolean(false); - } + out.writeOptionalWriteable(address); out.writeOptionalString(action); } diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index 0dd0c05dcad..b7a082c30df 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -53,9 +53,8 @@ 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.transport.PortsRange; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; @@ -506,7 +505,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i @Override public boolean addressSupported(Class address) { - return InetSocketTransportAddress.class.equals(address); + return TransportAddress.class.equals(address); } @Override @@ -640,7 +639,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i for (int i = 0; i < boundAddresses.size(); i++) { InetSocketAddress boundAddress = boundAddresses.get(i); boundAddressesHostStrings[i] = boundAddress.getHostString(); - transportBoundAddresses[i] = new InetSocketTransportAddress(boundAddress); + transportBoundAddresses[i] = new TransportAddress(boundAddress); } final String[] publishHosts; @@ -658,7 +657,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i } final int publishPort = resolvePublishPort(name, settings, profileSettings, boundAddresses, publishInetAddress); - final TransportAddress publishAddress = new InetSocketTransportAddress(new InetSocketAddress(publishInetAddress, publishPort)); + final TransportAddress publishAddress = new TransportAddress(new InetSocketAddress(publishInetAddress, publishPort)); return new BoundTransportAddress(transportBoundAddresses, publishAddress); } @@ -757,7 +756,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i 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])); + transportAddresses.add(new TransportAddress(address, ports[i])); } } return transportAddresses.toArray(new TransportAddress[transportAddresses.size()]); @@ -967,7 +966,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i try (BytesStreamOutput stream = new BytesStreamOutput()) { stream.setVersion(nodeVersion); RemoteTransportException tx = new RemoteTransportException( - nodeName(), new InetSocketTransportAddress(getLocalAddress(channel)), action, error); + nodeName(), new TransportAddress(getLocalAddress(channel)), action, error); threadPool.getThreadContext().writeTo(stream); stream.writeException(tx); byte status = 0; @@ -1194,9 +1193,10 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i } streamIn = compressor.streamInput(streamIn); } - if (version.onOrAfter(Version.CURRENT.minimumCompatibilityVersion()) == false || version.major != Version.CURRENT.major) { + if (version.onOrAfter(getCurrentVersion().minimumCompatibilityVersion()) == false + || version.major != getCurrentVersion().major) { throw new IllegalStateException("Received message from unsupported version: [" + version - + "] minimal compatible version is: [" +Version.CURRENT.minimumCompatibilityVersion() + "]"); + + "] minimal compatible version is: [" + getCurrentVersion().minimumCompatibilityVersion() + "]"); } streamIn = new NamedWriteableAwareStreamInput(streamIn, namedWriteableRegistry); streamIn.setVersion(version); @@ -1233,7 +1233,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i private void handleResponse(InetSocketAddress remoteAddress, final StreamInput stream, final TransportResponseHandler handler) { final TransportResponse response = handler.newInstance(); - response.remoteAddress(new InetSocketTransportAddress(remoteAddress)); + response.remoteAddress(new TransportAddress(remoteAddress)); try { response.readFrom(stream); } catch (Exception e) { @@ -1299,7 +1299,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i transportChannel = new TcpTransportChannel<>(this, channel, transportName, action, requestId, version, profileName, messageLengthBytes); final TransportRequest request = reg.newRequest(); - request.remoteAddress(new InetSocketTransportAddress(remoteAddress)); + request.remoteAddress(new TransportAddress(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); diff --git a/core/src/main/java/org/elasticsearch/transport/Transports.java b/core/src/main/java/org/elasticsearch/transport/Transports.java index 9b4dc4d5a6a..c187e3baf23 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.http.HttpServerTransport; -import org.elasticsearch.transport.local.LocalTransport; import java.util.Arrays; @@ -38,7 +37,6 @@ public enum Transports { public static final boolean isTransportThread(Thread t) { final String threadName = t.getName(); for (String s : Arrays.asList( - LocalTransport.LOCAL_TRANSPORT_THREAD_NAME_PREFIX, HttpServerTransport.HTTP_SERVER_BOSS_THREAD_NAME_PREFIX, HttpServerTransport.HTTP_SERVER_WORKER_THREAD_NAME_PREFIX, TcpTransport.TRANSPORT_SERVER_BOSS_THREAD_NAME_PREFIX, diff --git a/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java b/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java deleted file mode 100644 index f65312391d6..00000000000 --- a/core/src/main/java/org/elasticsearch/transport/local/LocalTransport.java +++ /dev/null @@ -1,430 +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.local; - -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.component.Lifecycle; -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.StreamInput; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.LocalTransportAddress; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ActionNotFoundTransportException; -import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.NodeNotConnectedException; -import org.elasticsearch.transport.RemoteTransportException; -import org.elasticsearch.transport.RequestHandlerRegistry; -import org.elasticsearch.transport.ResponseHandlerFailureTransportException; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; -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 java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; - -public class LocalTransport extends AbstractLifecycleComponent implements Transport { - - public static final String LOCAL_TRANSPORT_THREAD_NAME_PREFIX = "local_transport"; - final ThreadPool threadPool; - private final ThreadPoolExecutor workers; - private volatile TransportServiceAdapter transportServiceAdapter; - private volatile BoundTransportAddress boundAddress; - private volatile LocalTransportAddress localAddress; - private static final ConcurrentMap transports = newConcurrentMap(); - private static final AtomicLong transportAddressIdGenerator = new AtomicLong(); - private final ConcurrentMap connectedNodes = newConcurrentMap(); - protected final NamedWriteableRegistry namedWriteableRegistry; - private final CircuitBreakerService circuitBreakerService; - - public static final String TRANSPORT_LOCAL_ADDRESS = "transport.local.address"; - public static final String TRANSPORT_LOCAL_WORKERS = "transport.local.workers"; - public static final String TRANSPORT_LOCAL_QUEUE = "transport.local.queue"; - - public LocalTransport(Settings settings, ThreadPool threadPool, - NamedWriteableRegistry namedWriteableRegistry, CircuitBreakerService circuitBreakerService) { - super(settings); - this.threadPool = threadPool; - int workerCount = this.settings.getAsInt(TRANSPORT_LOCAL_WORKERS, EsExecutors.boundedNumberOfProcessors(settings)); - int queueSize = this.settings.getAsInt(TRANSPORT_LOCAL_QUEUE, -1); - logger.debug("creating [{}] workers, queue_size [{}]", workerCount, queueSize); - final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(this.settings, LOCAL_TRANSPORT_THREAD_NAME_PREFIX); - this.workers = EsExecutors.newFixed(LOCAL_TRANSPORT_THREAD_NAME_PREFIX, workerCount, queueSize, threadFactory, - threadPool.getThreadContext()); - this.namedWriteableRegistry = namedWriteableRegistry; - this.circuitBreakerService = circuitBreakerService; - } - - @Override - public TransportAddress[] addressesFromString(String address, int perAddressLimit) { - return new TransportAddress[]{new LocalTransportAddress(address)}; - } - - @Override - public boolean addressSupported(Class address) { - return LocalTransportAddress.class.equals(address); - } - - @Override - protected void doStart() { - String address = settings.get(TRANSPORT_LOCAL_ADDRESS); - if (address == null) { - address = Long.toString(transportAddressIdGenerator.incrementAndGet()); - } - localAddress = new LocalTransportAddress(address); - LocalTransport previous = transports.put(localAddress, this); - if (previous != null) { - throw new ElasticsearchException("local address [" + address + "] is already bound"); - } - boundAddress = new BoundTransportAddress(new TransportAddress[] { localAddress }, localAddress); - } - - @Override - protected void doStop() { - transports.remove(localAddress); - // now, go over all the transports connected to me, and raise disconnected event - for (final LocalTransport targetTransport : transports.values()) { - for (final Map.Entry entry : targetTransport.connectedNodes.entrySet()) { - if (entry.getValue() == this) { - targetTransport.disconnectFromNode(entry.getKey()); - } - } - } - } - - @Override - protected void doClose() { - ThreadPool.terminate(workers, 10, TimeUnit.SECONDS); - } - - @Override - public void transportServiceAdapter(TransportServiceAdapter transportServiceAdapter) { - this.transportServiceAdapter = transportServiceAdapter; - } - - @Override - public BoundTransportAddress boundAddress() { - return boundAddress; - } - - @Override - public Map profileBoundAddresses() { - return Collections.emptyMap(); - } - - @Override - public boolean nodeConnected(DiscoveryNode node) { - return connectedNodes.containsKey(node); - } - - @Override - public void connectToNodeLight(DiscoveryNode node) throws ConnectTransportException { - connectToNode(node); - } - - @Override - public void connectToNode(DiscoveryNode node) throws ConnectTransportException { - synchronized (this) { - if (connectedNodes.containsKey(node)) { - return; - } - final LocalTransport targetTransport = transports.get(node.getAddress()); - if (targetTransport == null) { - throw new ConnectTransportException(node, "Failed to connect"); - } - connectedNodes.put(node, targetTransport); - transportServiceAdapter.raiseNodeConnected(node); - } - } - - @Override - public void disconnectFromNode(DiscoveryNode node) { - synchronized (this) { - LocalTransport removed = connectedNodes.remove(node); - if (removed != null) { - transportServiceAdapter.raiseNodeDisconnected(node); - } - } - } - - @Override - public long serverOpen() { - return 0; - } - - @Override - public void sendRequest(final DiscoveryNode node, final long requestId, final String action, final TransportRequest request, - TransportRequestOptions options) throws IOException, TransportException { - final Version version = Version.smallest(node.getVersion(), getVersion()); - - try (BytesStreamOutput stream = new BytesStreamOutput()) { - stream.setVersion(version); - - stream.writeLong(requestId); - byte status = 0; - status = TransportStatus.setRequest(status); - stream.writeByte(status); // 0 for request, 1 for response. - - threadPool.getThreadContext().writeTo(stream); - stream.writeString(action); - request.writeTo(stream); - - stream.close(); - - final LocalTransport targetTransport = connectedNodes.get(node); - if (targetTransport == null) { - throw new NodeNotConnectedException(node, "Node not connected"); - } - - final byte[] data = BytesReference.toBytes(stream.bytes()); - transportServiceAdapter.addBytesSent(data.length); - transportServiceAdapter.onRequestSent(node, requestId, action, request, options); - targetTransport.receiveMessage(version, data, action, requestId, this); - } - } - - /** - * entry point for incoming messages - * - * @param version the version used to serialize the message - * @param data message data - * @param action the action associated with this message (only used for error handling when data is not parsable) - * @param requestId requestId if the message is request (only used for error handling when data is not parsable) - * @param sourceTransport the source transport to respond to. - */ - public void receiveMessage(Version version, byte[] data, String action, @Nullable Long requestId, LocalTransport sourceTransport) { - try { - workers().execute(() -> { - ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext context = threadContext.stashContext()) { - processReceivedMessage(data, action, sourceTransport, version, requestId); - } - }); - } catch (EsRejectedExecutionException e) { - assert lifecycle.started() == false; - logger.trace("received request but shutting down. ignoring. action [{}], request id [{}]", action, requestId); - } - } - - ThreadPoolExecutor workers() { - return this.workers; - } - - CircuitBreaker inFlightRequestsBreaker() { - // We always obtain a fresh breaker to reflect changes to the breaker configuration. - return circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); - } - - /** processes received messages, assuming thread passing and thread context have all been dealt with */ - protected void processReceivedMessage(byte[] data, String action, LocalTransport sourceTransport, Version version, - @Nullable final Long sendRequestId) { - Transports.assertTransportThread(); - try { - transportServiceAdapter.addBytesReceived(data.length); - StreamInput stream = StreamInput.wrap(data); - stream.setVersion(version); - - long requestId = stream.readLong(); - byte status = stream.readByte(); - boolean isRequest = TransportStatus.isRequest(status); - threadPool.getThreadContext().readHeaders(stream); - if (isRequest) { - handleRequest(stream, requestId, data.length, sourceTransport, version); - } else { - final TransportResponseHandler handler = transportServiceAdapter.onResponseReceived(requestId); - // ignore if its null, the adapter logs it - if (handler != null) { - if (TransportStatus.isError(status)) { - handleResponseError(stream, handler); - } else { - handleResponse(stream, sourceTransport, handler); - } - } - } - } catch (Exception e) { - if (sendRequestId != null) { - TransportResponseHandler handler = sourceTransport.transportServiceAdapter.onResponseReceived(sendRequestId); - if (handler != null) { - RemoteTransportException error = new RemoteTransportException(nodeName(), localAddress, action, e); - sourceTransport.workers().execute(() -> { - ThreadContext threadContext = sourceTransport.threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - sourceTransport.handleException(handler, error); - } - }); - } - } else { - logger.warn((Supplier) () -> new ParameterizedMessage("Failed to receive message for action [{}]", action), e); - } - } - } - - private void handleRequest(StreamInput stream, long requestId, int messageLengthBytes, LocalTransport sourceTransport, - Version version) throws Exception { - stream = new NamedWriteableAwareStreamInput(stream, namedWriteableRegistry); - final String action = stream.readString(); - final RequestHandlerRegistry reg = transportServiceAdapter.getRequestHandler(action); - transportServiceAdapter.onRequestReceived(requestId, action); - if (reg != null && reg.canTripCircuitBreaker()) { - inFlightRequestsBreaker().addEstimateBytesAndMaybeBreak(messageLengthBytes, ""); - } else { - inFlightRequestsBreaker().addWithoutBreaking(messageLengthBytes); - } - final LocalTransportChannel transportChannel = new LocalTransportChannel(this, transportServiceAdapter, sourceTransport, action, - requestId, version, messageLengthBytes, threadPool.getThreadContext()); - try { - if (reg == null) { - throw new ActionNotFoundTransportException("Action [" + action + "] not found"); - } - final TransportRequest request = reg.newRequest(); - request.remoteAddress(sourceTransport.boundAddress.publishAddress()); - request.readFrom(stream); - if (ThreadPool.Names.SAME.equals(reg.getExecutor())) { - //noinspection unchecked - reg.processMessageReceived(request, transportChannel); - } else { - threadPool.executor(reg.getExecutor()).execute(new AbstractRunnable() { - @Override - protected void doRun() throws Exception { - //noinspection unchecked - reg.processMessageReceived(request, transportChannel); - } - - @Override - public boolean isForceExecution() { - return reg.isForceExecution(); - } - - @Override - public void onFailure(Exception e) { - if (lifecycleState() == Lifecycle.State.STARTED) { - // we can only send a response transport is started.... - try { - transportChannel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.warn( - (Supplier) () -> new ParameterizedMessage( - "Failed to send error message back to client for action [{}]", action), inner); - } - } - } - }); - } - } catch (Exception e) { - try { - transportChannel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.warn( - (Supplier) () -> new ParameterizedMessage( - "Failed to send error message back to client for action [{}]", action), inner); - } - - } - } - - protected void handleResponse(StreamInput buffer, LocalTransport sourceTransport, final TransportResponseHandler handler) { - buffer = new NamedWriteableAwareStreamInput(buffer, namedWriteableRegistry); - final TransportResponse response = handler.newInstance(); - response.remoteAddress(sourceTransport.boundAddress.publishAddress()); - try { - response.readFrom(buffer); - } catch (Exception e) { - handleException(handler, new TransportSerializationException( - "Failed to deserialize response of type [" + response.getClass().getName() + "]", e)); - return; - } - handleParsedResponse(response, handler); - } - - protected void handleParsedResponse(final TransportResponse response, final TransportResponseHandler handler) { - threadPool.executor(handler.executor()).execute(() -> { - try { - handler.handleResponse(response); - } catch (Exception e) { - handleException(handler, new ResponseHandlerFailureTransportException(e)); - } - }); - } - - private void handleResponseError(StreamInput buffer, final TransportResponseHandler handler) { - Exception exception; - try { - exception = buffer.readException(); - } catch (Exception e) { - exception = new TransportSerializationException("Failed to deserialize exception response from stream", e); - } - handleException(handler, exception); - } - - private void handleException(final TransportResponseHandler handler, Exception exception) { - if (!(exception instanceof RemoteTransportException)) { - exception = new RemoteTransportException("Not a remote transport exception", null, null, exception); - } - final RemoteTransportException rtx = (RemoteTransportException) exception; - try { - handler.handleException(rtx); - } catch (Exception e) { - logger.error((Supplier) () -> new ParameterizedMessage("failed to handle exception response [{}]", handler), e); - } - } - - @Override - public List getLocalAddresses() { - return Collections.singletonList("0.0.0.0"); - } - - protected Version getVersion() { // for tests - return Version.CURRENT; - } -} diff --git a/core/src/main/java/org/elasticsearch/transport/local/LocalTransportChannel.java b/core/src/main/java/org/elasticsearch/transport/local/LocalTransportChannel.java deleted file mode 100644 index 0c1e8747a12..00000000000 --- a/core/src/main/java/org/elasticsearch/transport/local/LocalTransportChannel.java +++ /dev/null @@ -1,139 +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.local; - -import org.elasticsearch.Version; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.util.concurrent.ThreadContext; -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 java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * - */ -public class LocalTransportChannel implements TransportChannel { - - private static final String LOCAL_TRANSPORT_PROFILE = "default"; - - private final LocalTransport sourceTransport; - private final TransportServiceAdapter sourceTransportServiceAdapter; - // the transport we will *send to* - private final LocalTransport targetTransport; - private final String action; - private final long requestId; - private final Version version; - private final long reservedBytes; - private final ThreadContext threadContext; - private final AtomicBoolean closed = new AtomicBoolean(); - - public LocalTransportChannel(LocalTransport sourceTransport, TransportServiceAdapter sourceTransportServiceAdapter, - LocalTransport targetTransport, String action, long requestId, Version version, long reservedBytes, - ThreadContext threadContext) { - this.sourceTransport = sourceTransport; - this.sourceTransportServiceAdapter = sourceTransportServiceAdapter; - this.targetTransport = targetTransport; - this.action = action; - this.requestId = requestId; - this.version = version; - this.reservedBytes = reservedBytes; - this.threadContext = threadContext; - } - - @Override - public String action() { - return action; - } - - @Override - public String getProfileName() { - return LOCAL_TRANSPORT_PROFILE; - } - - @Override - public void sendResponse(TransportResponse response) throws IOException { - sendResponse(response, TransportResponseOptions.EMPTY); - } - - @Override - public void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException { - try (BytesStreamOutput stream = new BytesStreamOutput()) { - stream.setVersion(version); - stream.writeLong(requestId); - byte status = 0; - status = TransportStatus.setResponse(status); - stream.writeByte(status); // 0 for request, 1 for response. - threadContext.writeTo(stream); - response.writeTo(stream); - sendResponseData(BytesReference.toBytes(stream.bytes())); - sourceTransportServiceAdapter.onResponseSent(requestId, action, response, options); - } - } - - @Override - public void sendResponse(Exception exception) throws IOException { - BytesStreamOutput stream = new BytesStreamOutput(); - writeResponseExceptionHeader(stream); - RemoteTransportException tx = new RemoteTransportException(targetTransport.nodeName(), - targetTransport.boundAddress().boundAddresses()[0], action, exception); - stream.writeException(tx); - sendResponseData(BytesReference.toBytes(stream.bytes())); - sourceTransportServiceAdapter.onResponseSent(requestId, action, exception); - } - - private void sendResponseData(byte[] data) { - close(); - targetTransport.receiveMessage(version, data, action, null, sourceTransport); - } - - private void close() { - // attempt to close once atomically - if (closed.compareAndSet(false, true) == false) { - throw new IllegalStateException("Channel is already closed"); - } - sourceTransport.inFlightRequestsBreaker().addWithoutBreaking(-reservedBytes); - } - - @Override - public long getRequestId() { - return requestId; - } - - @Override - public String getChannelType() { - return "local"; - } - - private void writeResponseExceptionHeader(BytesStreamOutput stream) throws IOException { - stream.writeLong(requestId); - byte status = 0; - status = TransportStatus.setResponse(status); - status = TransportStatus.setError(status); - stream.writeByte(status); - threadContext.writeTo(stream); - } -} diff --git a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 0611d706acc..0eff8d7e60c 100644 --- a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -40,7 +40,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CancellableThreadsTests; import org.elasticsearch.common.util.set.Sets; @@ -373,11 +373,12 @@ public class ExceptionSerializationTests extends ESTestCase { } public void testActionTransportException() throws IOException { + TransportAddress transportAddress = buildNewFakeTransportAddress(); ActionTransportException ex = serialize( - new ActionTransportException("name?", new LocalTransportAddress("dead.end:666"), "ACTION BABY!", "message?", null)); + new ActionTransportException("name?", transportAddress, "ACTION BABY!", "message?", null)); assertEquals("ACTION BABY!", ex.action()); - assertEquals(new LocalTransportAddress("dead.end:666"), ex.address()); - assertEquals("[name?][local[dead.end:666]][ACTION BABY!] message?", ex.getMessage()); + assertEquals(transportAddress, ex.address()); + assertEquals("[name?][" + transportAddress.toString() +"][ACTION BABY!] message?", ex.getMessage()); } public void testSearchContextMissingException() throws IOException { @@ -440,16 +441,17 @@ public class ExceptionSerializationTests extends ESTestCase { } public void testConnectTransportException() throws IOException { - DiscoveryNode node = new DiscoveryNode("thenode", new LocalTransportAddress("dead.end:666"), + TransportAddress transportAddress = buildNewFakeTransportAddress(); + DiscoveryNode node = new DiscoveryNode("thenode", transportAddress, emptyMap(), emptySet(), Version.CURRENT); ConnectTransportException ex = serialize(new ConnectTransportException(node, "msg", "action", null)); - assertEquals("[][local[dead.end:666]][action] msg", ex.getMessage()); + assertEquals("[][" + transportAddress.toString() + "][action] msg", ex.getMessage()); assertEquals(node, ex.node()); assertEquals("action", ex.action()); assertNull(ex.getCause()); ex = serialize(new ConnectTransportException(node, "msg", "action", new NullPointerException())); - assertEquals("[][local[dead.end:666]][action] msg", ex.getMessage()); + assertEquals("[]["+ transportAddress+ "][action] msg", ex.getMessage()); assertEquals(node, ex.node()); assertEquals("action", ex.action()); assertTrue(ex.getCause() instanceof NullPointerException); diff --git a/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java b/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java index d1d01610f18..92d1768db13 100644 --- a/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/core/src/test/java/org/elasticsearch/action/IndicesRequestIT.java @@ -78,6 +78,7 @@ import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugins.NetworkPlugin; @@ -742,7 +743,7 @@ public class IndicesRequestIT extends ESIntegTestCase { public static class TestPlugin extends Plugin implements NetworkPlugin { public final InterceptingTransportService instance = new InterceptingTransportService(); @Override - public List getTransportInterceptors() { + public List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry) { return Collections.singletonList(instance); } } diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java index 577f73a89ed..d656702f9cd 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplanationTests.java @@ -33,7 +33,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -67,7 +66,7 @@ public final class ClusterAllocationExplanationTests extends ESTestCase { .numberOfShards(1) .numberOfReplicas(1) .build(); - private DiscoveryNode node = new DiscoveryNode("node-0", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + private DiscoveryNode node = new DiscoveryNode("node-0", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); private static Decision.Multi yesDecision = new Decision.Multi(); private static Decision.Multi noDecision = new Decision.Multi(); diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java index 04905de18c2..6e8159589f1 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -22,7 +22,6 @@ package org.elasticsearch.action.admin.cluster.node.stats; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.DiscoveryStats; import org.elasticsearch.discovery.zen.publish.PendingClusterStateStats; import org.elasticsearch.http.HttpStats; @@ -255,7 +254,7 @@ public class NodeStatsTests extends ESTestCase { } private static NodeStats createNodeStats() { - DiscoveryNode node = new DiscoveryNode("test_node", LocalTransportAddress.buildUnique(), + DiscoveryNode node = new DiscoveryNode("test_node", buildNewFakeTransportAddress(), emptyMap(), emptySet(), VersionUtils.randomVersion(random())); OsStats osStats = null; if (frequently()) { diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java index c457d3a30fa..27c0e7a2ec5 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java @@ -44,15 +44,17 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.tasks.MockTaskManager; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.MockTcpTransport; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.local.LocalTransport; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -168,8 +170,9 @@ public abstract class TaskManagerTestCase extends ESTestCase { public TestNode(String name, ThreadPool threadPool, Settings settings) { clusterService = createClusterService(threadPool); transportService = new TransportService(settings, - new LocalTransport(settings, threadPool, new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR) { + new MockTcpTransport(settings, threadPool, BigArrays.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService(), + new NamedWriteableRegistry(Collections.emptyList()), new NetworkService(settings, Collections.emptyList())), + threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR) { @Override protected TaskManager createTaskManager() { if (MockTaskManager.USE_MOCK_TASK_MANAGER_SETTING.get(settings)) { diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index 134477cc204..885a173d908 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -58,6 +58,7 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.tasks.MockTaskManager; import org.elasticsearch.test.tasks.MockTaskManagerListener; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.MockTcpTransportPlugin; import org.elasticsearch.transport.ReceiveTimeoutTransportException; import org.elasticsearch.transport.TransportService; @@ -102,8 +103,10 @@ public class TasksIT extends ESIntegTestCase { private Map, RecordingTaskManagerListener> listeners = new HashMap<>(); @Override - protected boolean addMockTransportService() { - return false; + protected Collection> getMockPlugins() { + Collection> mockPlugins = new ArrayList<>(super.getMockPlugins()); + mockPlugins.remove(MockTransportService.TestPlugin.class); + return mockPlugins; } @Override diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java index 9705009a044..95b847dd658 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -53,8 +52,8 @@ public class IndicesShardStoreResponseTests extends ESTestCase { List failures = new ArrayList<>(); ImmutableOpenIntMap.Builder> storeStatuses = ImmutableOpenIntMap.builder(); - DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); - DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); List storeStatusList = new ArrayList<>(); storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node2, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); @@ -122,7 +121,7 @@ public class IndicesShardStoreResponseTests extends ESTestCase { } public void testStoreStatusOrdering() throws Exception { - DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); List orderedStoreStatuses = new ArrayList<>(); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java index 0c5164aec5b..5446d74911d 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java @@ -39,7 +39,6 @@ import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllo import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.gateway.TestGatewayAllocator; @@ -144,7 +143,7 @@ public class TransportShrinkActionTests extends ESTestCase { } private DiscoveryNode newNode(String nodeId) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), + return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA))), Version.CURRENT); } } diff --git a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java index 50bd3771bc3..f0b7f631505 100644 --- a/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java +++ b/core/src/test/java/org/elasticsearch/action/ingest/IngestProxyActionFilterTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; @@ -79,7 +78,7 @@ public class IngestProxyActionFilterTests extends ESTestCase { if (i < ingestNodes) { roles.add(DiscoveryNode.Role.INGEST); } - DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, LocalTransportAddress.buildUnique(), attributes, roles, VersionUtils.randomVersion(random())); + DiscoveryNode node = new DiscoveryNode(nodeId, nodeId, buildNewFakeTransportAddress(), attributes, roles, VersionUtils.randomVersion(random())); builder.add(node); if (i == totalNodes - 1) { localNode = node; diff --git a/core/src/test/java/org/elasticsearch/action/search/TransportMultiSearchActionTests.java b/core/src/test/java/org/elasticsearch/action/search/TransportMultiSearchActionTests.java index 9df5bc82238..306031eaaf4 100644 --- a/core/src/test/java/org/elasticsearch/action/search/TransportMultiSearchActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/TransportMultiSearchActionTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; @@ -117,12 +116,12 @@ public class TransportMultiSearchActionTests extends ESTestCase { int numDataNodes = randomIntBetween(1, 10); DiscoveryNodes.Builder builder = DiscoveryNodes.builder(); for (int i = 0; i < numDataNodes; i++) { - builder.add(new DiscoveryNode("_id" + i, new LocalTransportAddress("_id" + i), Collections.emptyMap(), + builder.add(new DiscoveryNode("_id" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT)); } - builder.add(new DiscoveryNode("master", new LocalTransportAddress("mater"), Collections.emptyMap(), + builder.add(new DiscoveryNode("master", buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.MASTER), Version.CURRENT)); - builder.add(new DiscoveryNode("ingest", new LocalTransportAddress("ingest"), Collections.emptyMap(), + builder.add(new DiscoveryNode("ingest", buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.INGEST), Version.CURRENT)); ClusterState state = ClusterState.builder(new ClusterName("_name")).nodes(builder).build(); diff --git a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java index a249a0e98ef..de216eb571b 100644 --- a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java @@ -49,7 +49,6 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; @@ -248,7 +247,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase { } static DiscoveryNode newNode(int nodeId) { - return new DiscoveryNode("node_" + nodeId, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + return new DiscoveryNode("node_" + nodeId, buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java b/core/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java index 87f86c3f596..29dd2a150a3 100644 --- a/core/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java +++ b/core/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java @@ -22,6 +22,7 @@ package org.elasticsearch.action.support.master; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.fd.FaultDetection; @@ -43,7 +44,6 @@ import java.util.concurrent.CyclicBarrier; import static org.hamcrest.Matchers.equalTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode public class IndexingMasterFailoverIT extends ESIntegTestCase { @Override @@ -62,6 +62,7 @@ public class IndexingMasterFailoverIT extends ESIntegTestCase { logger.info("--> start 4 nodes, 3 master, 1 data"); final Settings sharedSettings = Settings.builder() + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen") .put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly .put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") // for hitting simulated network failures quickly .put("discovery.zen.join_timeout", "10s") // still long to induce failures but to long so test won't time out diff --git a/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java index a7db99cc201..2cb55686589 100644 --- a/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.MasterNotDiscoveredException; @@ -90,9 +89,9 @@ public class TransportMasterNodeActionTests extends ESTestCase { transportService = new TransportService(clusterService.getSettings(), transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR); transportService.start(); transportService.acceptIncomingRequests(); - localNode = new DiscoveryNode("local_node", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + localNode = new DiscoveryNode("local_node", buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.MASTER), Version.CURRENT); - remoteNode = new DiscoveryNode("remote_node", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + remoteNode = new DiscoveryNode("remote_node", buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.MASTER), Version.CURRENT); allNodes = new DiscoveryNode[]{localNode, remoteNode}; } diff --git a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java index 67cc64cb871..659ce291b1f 100644 --- a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.threadpool.TestThreadPool; @@ -237,7 +236,7 @@ public class TransportNodesActionTests extends ESTestCase { private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { String node = "node_" + nodeId; - return new DiscoveryNode(node, node, LocalTransportAddress.buildUnique(), attributes, roles, Version.CURRENT); + return new DiscoveryNode(node, node, buildNewFakeTransportAddress(), attributes, roles, Version.CURRENT); } private static class TestTransportNodesAction diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index 2d098a065b5..3f6a30a5833 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.support.replication; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.ShardOperationFailedException; @@ -35,7 +36,9 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.breaker.CircuitBreakerService; @@ -45,8 +48,8 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.MockTcpTransport; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.local.LocalTransport; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -89,7 +92,9 @@ public class BroadcastReplicationTests extends ESTestCase { @Before public void setUp() throws Exception { super.setUp(); - LocalTransport transport = new LocalTransport(Settings.EMPTY, threadPool, new NamedWriteableRegistry(Collections.emptyList()), circuitBreakerService); + MockTcpTransport transport = new MockTcpTransport(Settings.EMPTY, + threadPool, BigArrays.NON_RECYCLING_INSTANCE, circuitBreakerService, new NamedWriteableRegistry(Collections.emptyList()), + new NetworkService(Settings.EMPTY, Collections.emptyList())); clusterService = createClusterService(threadPool); transportService = new TransportService(clusterService.getSettings(), transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR); transportService.start(); @@ -100,8 +105,7 @@ public class BroadcastReplicationTests extends ESTestCase { @After public void tearDown() throws Exception { super.tearDown(); - clusterService.close(); - transportService.close(); + IOUtils.close(clusterService, transportService); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java index 646ad23a48b..ab419a3c698 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java @@ -34,8 +34,8 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; import java.util.Arrays; import java.util.Collections; @@ -255,7 +255,7 @@ public class ClusterStateCreationUtils { } private static DiscoveryNode newNode(int nodeId) { - return new DiscoveryNode("node_" + nodeId, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + return new DiscoveryNode("node_" + nodeId, ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java index 9813731017d..2a00ca889d6 100644 --- a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java +++ b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.node.NodeValidationException; import org.elasticsearch.test.ESTestCase; +import java.net.InetAddress; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -55,13 +56,11 @@ public class BootstrapCheckTests extends ESTestCase { // nothing should happen since we are in non-production mode final List transportAddresses = new ArrayList<>(); for (int i = 0; i < randomIntBetween(1, 8); i++) { - TransportAddress localTransportAddress = mock(TransportAddress.class); - when(localTransportAddress.isLoopbackOrLinkLocalAddress()).thenReturn(true); + TransportAddress localTransportAddress = new TransportAddress(InetAddress.getLoopbackAddress(), i); transportAddresses.add(localTransportAddress); } - TransportAddress publishAddress = mock(TransportAddress.class); - when(publishAddress.isLoopbackOrLinkLocalAddress()).thenReturn(true); + TransportAddress publishAddress = new TransportAddress(InetAddress.getLoopbackAddress(), 0); BoundTransportAddress boundTransportAddress = mock(BoundTransportAddress.class); when(boundTransportAddress.boundAddresses()).thenReturn(transportAddresses.toArray(new TransportAddress[0])); when(boundTransportAddress.publishAddress()).thenReturn(publishAddress); @@ -83,18 +82,17 @@ public class BootstrapCheckTests extends ESTestCase { public void testEnforceLimitsWhenBoundToNonLocalAddress() { final List transportAddresses = new ArrayList<>(); - final TransportAddress nonLocalTransportAddress = mock(TransportAddress.class); - when(nonLocalTransportAddress.isLoopbackOrLinkLocalAddress()).thenReturn(false); + final TransportAddress nonLocalTransportAddress = buildNewFakeTransportAddress(); transportAddresses.add(nonLocalTransportAddress); for (int i = 0; i < randomIntBetween(0, 7); i++) { - final TransportAddress randomTransportAddress = mock(TransportAddress.class); - when(randomTransportAddress.isLoopbackOrLinkLocalAddress()).thenReturn(randomBoolean()); + final TransportAddress randomTransportAddress = randomBoolean() ? buildNewFakeTransportAddress() : + new TransportAddress(InetAddress.getLoopbackAddress(), i); transportAddresses.add(randomTransportAddress); } - final TransportAddress publishAddress = mock(TransportAddress.class); - when(publishAddress.isLoopbackOrLinkLocalAddress()).thenReturn(randomBoolean()); + final TransportAddress publishAddress = randomBoolean() ? buildNewFakeTransportAddress() : + new TransportAddress(InetAddress.getLoopbackAddress(), 0); final BoundTransportAddress boundTransportAddress = mock(BoundTransportAddress.class); Collections.shuffle(transportAddresses, random()); @@ -108,14 +106,11 @@ public class BootstrapCheckTests extends ESTestCase { final List transportAddresses = new ArrayList<>(); for (int i = 0; i < randomIntBetween(1, 8); i++) { - final TransportAddress randomTransportAddress = mock(TransportAddress.class); - when(randomTransportAddress.isLoopbackOrLinkLocalAddress()).thenReturn(false); + final TransportAddress randomTransportAddress = buildNewFakeTransportAddress(); transportAddresses.add(randomTransportAddress); } - final TransportAddress publishAddress = mock(TransportAddress.class); - when(publishAddress.isLoopbackOrLinkLocalAddress()).thenReturn(true); - + final TransportAddress publishAddress = new TransportAddress(InetAddress.getLoopbackAddress(), 0); final BoundTransportAddress boundTransportAddress = mock(BoundTransportAddress.class); when(boundTransportAddress.boundAddresses()).thenReturn(transportAddresses.toArray(new TransportAddress[0])); when(boundTransportAddress.publishAddress()).thenReturn(publishAddress); diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientHeadersTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientHeadersTests.java index 02240a6bf24..db195ea0b55 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientHeadersTests.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientHeadersTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.env.Environment; @@ -61,14 +62,20 @@ public class TransportClientHeadersTests extends AbstractClientHeadersTestCase { @Override public void tearDown() throws Exception { - super.tearDown(); - transportService.stop(); - transportService.close(); + try { + // stop this first before we bubble up since + // transportService uses the threadpool that super.tearDown will close + transportService.stop(); + transportService.close(); + } finally { + super.tearDown(); + } + } @Override protected Client buildClient(Settings headersSettings, GenericAction[] testedActions) { - transportService = MockTransportService.local(Settings.EMPTY, Version.CURRENT, threadPool); + transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool); transportService.start(); transportService.acceptIncomingRequests(); TransportClient client = new MockTransportClient(Settings.builder() @@ -120,7 +127,7 @@ public class TransportClientHeadersTests extends AbstractClientHeadersTestCase { private InternalTransportServiceInterceptor instance = new InternalTransportServiceInterceptor(); @Override - public List getTransportInterceptors() { + public List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry) { return Collections.singletonList(new TransportInterceptor() { @Override public TransportRequestHandler interceptHandler(String action, diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java index 3f4fd501171..93c5e29208c 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientIT.java @@ -26,15 +26,18 @@ import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.env.Environment; +import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeValidationException; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; +import org.elasticsearch.transport.MockTcpTransportPlugin; import org.elasticsearch.transport.MockTransportClient; import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.Collections; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -54,15 +57,15 @@ public class TransportClientIT extends ESIntegTestCase { public void testNodeVersionIsUpdated() throws IOException, NodeValidationException { TransportClient client = (TransportClient) internalCluster().client(); - try (Node node = new Node(Settings.builder() + try (Node node = new MockNode(Settings.builder() .put(internalCluster().getDefaultSettings()) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) .put("node.name", "testNodeVersionIsUpdated") - .put("transport.type", "local") + .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put(NetworkModule.HTTP_ENABLED.getKey(), false) .put(Node.NODE_DATA_SETTING.getKey(), false) .put("cluster.name", "foobar") - .build()).start()) { + .build(), Collections.singleton(MockTcpTransportPlugin.class)).start()) { TransportAddress transportAddress = node.injector().getInstance(TransportService.class).boundAddress().publishAddress(); client.addTransportAddress(transportAddress); // since we force transport clients there has to be one node started that we connect to. diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java index 1596519651f..59a4e3a4b76 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java @@ -25,7 +25,7 @@ import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAct import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -39,13 +39,16 @@ import org.elasticsearch.transport.TransportService; import java.io.Closeable; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.CoreMatchers.startsWith; @@ -60,6 +63,9 @@ public class TransportClientNodesServiceTests extends ESTestCase { private final TransportService transportService; private final TransportClientNodesService transportClientNodesService; private final int nodesCount; + private TransportAddress livenessAddress = buildNewFakeTransportAddress(); + public Set nodeAddresses = new HashSet<>(); + TestIteration() { Settings settings = Settings.builder().put("cluster.name", "test").build(); @@ -98,7 +104,9 @@ public class TransportClientNodesServiceTests extends ESTestCase { new TransportClientNodesService(settings, transportService, threadPool); this.nodesCount = randomIntBetween(1, 10); for (int i = 0; i < nodesCount; i++) { - transportClientNodesService.addTransportAddresses(new LocalTransportAddress("node" + i)); + TransportAddress transportAddress = buildNewFakeTransportAddress(); + nodeAddresses.add(transportAddress); + transportClientNodesService.addTransportAddresses(transportAddress); } transport.endConnectMode(); } @@ -118,7 +126,7 @@ public class TransportClientNodesServiceTests extends ESTestCase { LivenessResponse livenessResponse = new LivenessResponse(clusterName, new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), "liveness-hostname" + node.getId(), "liveness-hostaddress" + node.getId(), - new LocalTransportAddress("liveness-address-" + node.getId()), node.getAttributes(), node.getRoles(), + livenessAddress, node.getAttributes(), node.getRoles(), node.getVersion())); handler.handleResponse((T)livenessResponse); } @@ -237,10 +245,8 @@ public class TransportClientNodesServiceTests extends ESTestCase { for (DiscoveryNode discoveryNode : iteration.transportClientNodesService.connectedNodes()) { assertThat(discoveryNode.getHostName(), startsWith("liveness-")); assertThat(discoveryNode.getHostAddress(), startsWith("liveness-")); - assertThat(discoveryNode.getAddress(), instanceOf(LocalTransportAddress.class)); - LocalTransportAddress localTransportAddress = (LocalTransportAddress) discoveryNode.getAddress(); - //the original listed transport address is kept rather than the one returned from the liveness api - assertThat(localTransportAddress.id(), startsWith("node")); + assertNotEquals(discoveryNode.getAddress(), iteration.livenessAddress); + assertThat(iteration.nodeAddresses, hasItem(discoveryNode.getAddress())); } } } diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java index 6326d96f317..939954c4560 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java @@ -30,7 +30,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.index.Index; import org.elasticsearch.test.ESTestCase; @@ -320,7 +319,7 @@ public class ClusterChangedEventTests extends ESTestCase { // Create a new DiscoveryNode private static DiscoveryNode newNode(final String nodeId, Set roles) { - return new DiscoveryNode(nodeId, nodeId, nodeId, "host", "host_address", new LocalTransportAddress("_test_" + nodeId), + return new DiscoveryNode(nodeId, nodeId, nodeId, "host", "host_address", buildNewFakeTransportAddress(), Collections.emptyMap(), roles, Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java index db59e785871..c77d7c10c96 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java @@ -46,7 +46,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.index.Index; @@ -74,9 +73,9 @@ import static org.hamcrest.Matchers.is; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0, numClientNodes = 0) public class ClusterStateDiffIT extends ESIntegTestCase { public void testClusterStateDiffSerialization() throws Exception { - DiscoveryNode masterNode = new DiscoveryNode("master", new LocalTransportAddress("master"), + DiscoveryNode masterNode = new DiscoveryNode("master", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); - DiscoveryNode otherNode = new DiscoveryNode("other", new LocalTransportAddress("other"), + DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(masterNode).add(otherNode).localNodeId(masterNode.getId()).build(); ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); @@ -193,14 +192,14 @@ public class ClusterStateDiffIT extends ESIntegTestCase { if (nodeId.startsWith("node-")) { nodes.remove(nodeId); if (randomBoolean()) { - nodes.add(new DiscoveryNode(nodeId, new LocalTransportAddress(randomAsciiOfLength(10)), emptyMap(), + nodes.add(new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), emptySet(), randomVersion(random()))); } } } int additionalNodeCount = randomIntBetween(1, 20); for (int i = 0; i < additionalNodeCount; i++) { - nodes.add(new DiscoveryNode("node-" + randomAsciiOfLength(10), new LocalTransportAddress(randomAsciiOfLength(10)), + nodes.add(new DiscoveryNode("node-" + randomAsciiOfLength(10), buildNewFakeTransportAddress(), emptyMap(), emptySet(), randomVersion(random()))); } return ClusterState.builder(clusterState).nodes(nodes); diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java index b0cba5bf1de..21c661230eb 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import static java.util.Collections.emptyMap; @@ -33,8 +32,8 @@ public class ClusterStateTests extends ESTestCase { public void testSupersedes() { final Version version = Version.CURRENT; - final DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), version); - final DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), version); + final DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); + final DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); final DiscoveryNodes nodes = DiscoveryNodes.builder().add(node1).add(node2).build(); ClusterName name = ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY); ClusterState noMaster1 = ClusterState.builder(name).version(randomInt(5)).nodes(nodes).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java b/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java index b9e07a9207f..3f72be32730 100644 --- a/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; @@ -199,11 +198,11 @@ public class DiskUsageTests extends ESTestCase { new FsInfo.Path("/most", "/dev/sda", 100, 90, 80), }; List nodeStats = Arrays.asList( - new NodeStats(new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null) ); InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages); @@ -240,11 +239,11 @@ public class DiskUsageTests extends ESTestCase { new FsInfo.Path("/least", "/dev/sda", 10, -8, 0), }; List nodeStats = Arrays.asList( - new NodeStats(new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null), - new NodeStats(new DiscoveryNode("node_3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), 0, + new NodeStats(new DiscoveryNode("node_3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0, null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null) ); InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvailableUsages, newMostAvailableUsages); diff --git a/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java b/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java index 2e86cb5b896..b8e95343932 100644 --- a/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.Discovery; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.ZenDiscovery; @@ -63,7 +64,6 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode @TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.discovery.zen:TRACE") public class MinimumMasterNodesIT extends ESIntegTestCase { @@ -74,6 +74,12 @@ public class MinimumMasterNodesIT extends ESIntegTestCase { return classes; } + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + public void testSimpleMinimumMasterNodes() throws Exception { Settings settings = Settings.builder() diff --git a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java index 84410a92c83..d730b859113 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.MasterNotDiscoveredException; import org.elasticsearch.discovery.zen.ZenDiscovery; @@ -49,8 +50,14 @@ import static org.hamcrest.Matchers.lessThan; /** */ @ClusterScope(scope = Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode public class NoMasterNodeIT extends ESIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + public void testNoMasterActions() throws Exception { // note, sometimes, we want to check with the fact that an index gets created, sometimes not... boolean autoCreateIndex = randomBoolean(); diff --git a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java index 5bf2bc38c3e..b91bd3f385c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/NodeConnectionsServiceTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.test.ESTestCase; @@ -64,7 +63,7 @@ public class NodeConnectionsServiceTests extends ESTestCase { List nodes = new ArrayList<>(); for (int i = randomIntBetween(20, 50); i > 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))); - nodes.add(new DiscoveryNode("node_" + i, "" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + nodes.add(new DiscoveryNode("node_" + i, "" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), roles, Version.CURRENT)); } return nodes; diff --git a/core/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java b/core/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java index 4773aafbf3f..4d62f9b664f 100644 --- a/core/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster; import org.apache.lucene.search.join.ScoreMode; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.MasterNotDiscoveredException; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.node.Node; @@ -38,8 +39,13 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode public class SpecificMasterNodesIT extends ESIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } protected final Settings.Builder settingsBuilder() { return Settings.builder().put("discovery.type", "zen"); } diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java index d74b450f5bf..28652074826 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.InvalidIndexNameException; @@ -181,7 +180,7 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { } private DiscoveryNode newNode(String nodeId) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), + return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA))), Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java index 59f058a95fb..e1b6d0e6ef8 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeFiltersTests.java @@ -21,8 +21,7 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; -import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -46,11 +45,11 @@ import static org.hamcrest.Matchers.equalTo; */ public class DiscoveryNodeFiltersTests extends ESTestCase { - private static InetSocketTransportAddress localAddress; + private static TransportAddress localAddress; @BeforeClass public static void createLocalAddress() throws UnknownHostException { - localAddress = new InetSocketTransportAddress(InetAddress.getByName("192.1.1.54"), 9999); + localAddress = new TransportAddress(InetAddress.getByName("192.1.1.54"), 9999); } @AfterClass @@ -64,11 +63,11 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + DiscoveryNode node = new DiscoveryNode("name1", "id1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -78,11 +77,11 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + DiscoveryNode node = new DiscoveryNode("name1", "id1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name2", "id2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -94,13 +93,13 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); final Version version = Version.CURRENT; - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), version); + DiscoveryNode node = new DiscoveryNode("name1", "id1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), version); + node = new DiscoveryNode("name2", "id2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); assertThat(filters.match(node), equalTo(true)); - node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), version); + node = new DiscoveryNode("name3", "id3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); assertThat(filters.match(node), equalTo(false)); } @@ -114,7 +113,7 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { Map attributes = new HashMap<>(); attributes.put("tag", "A"); attributes.put("group", "B"); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), + DiscoveryNode node = new DiscoveryNode("name1", "id1", buildNewFakeTransportAddress(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); @@ -122,7 +121,7 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { attributes.put("tag", "A"); attributes.put("group", "B"); attributes.put("name", "X"); - node = new DiscoveryNode("name2", "id2", LocalTransportAddress.buildUnique(), + node = new DiscoveryNode("name2", "id2", buildNewFakeTransportAddress(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); @@ -130,11 +129,11 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { attributes.put("tag", "A"); attributes.put("group", "F"); attributes.put("name", "X"); - node = new DiscoveryNode("name3", "id3", LocalTransportAddress.buildUnique(), + node = new DiscoveryNode("name3", "id3", buildNewFakeTransportAddress(), attributes, emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); - node = new DiscoveryNode("name4", "id4", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + node = new DiscoveryNode("name4", "id4", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(false)); } @@ -144,7 +143,7 @@ public class DiscoveryNodeFiltersTests extends ESTestCase { .build(); DiscoveryNodeFilters filters = DiscoveryNodeFilters.buildFromSettings(OR, "xxx.", settings); - DiscoveryNode node = new DiscoveryNode("name1", "id1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + DiscoveryNode node = new DiscoveryNode("name1", "id1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); assertThat(filters.match(node), equalTo(true)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index d6a83108d0f..a16520faeb1 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.cluster.node; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.elasticsearch.Version; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -194,7 +193,7 @@ public class DiscoveryNodesTests extends ESTestCase { } private static DiscoveryNode newNode(int nodeId, Map attributes, Set roles) { - return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, LocalTransportAddress.buildUnique(), attributes, roles, + return new DiscoveryNode("name_" + nodeId, "node_" + nodeId, buildNewFakeTransportAddress(), attributes, roles, Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 8bc9c29bb37..9f21d133264 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -27,10 +27,10 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.AllocateStalePrimaryAllocationCommand; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -38,7 +38,6 @@ import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect; import org.elasticsearch.test.disruption.NetworkDisruption.TwoPartitions; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; import java.util.Arrays; @@ -55,7 +54,6 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode public class PrimaryAllocationIT extends ESIntegTestCase { @Override @@ -64,6 +62,12 @@ public class PrimaryAllocationIT extends ESIntegTestCase { return Arrays.asList(MockTransportService.TestPlugin.class); } + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + private void createStaleReplicaScenario() throws Exception { logger.info("--> starting 3 nodes, 1 master, 2 data"); String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); @@ -115,12 +119,7 @@ public class PrimaryAllocationIT extends ESIntegTestCase { logger.info("--> check that old primary shard does not get promoted to primary again"); // kick reroute and wait for all shard states to be fetched client(master).admin().cluster().prepareReroute().get(); - assertBusy(new Runnable() { - @Override - public void run() { - assertThat(internalCluster().getInstance(GatewayAllocator.class, master).getNumberOfInFlightFetch(), equalTo(0)); - } - }); + assertBusy(() -> assertThat(internalCluster().getInstance(GatewayAllocator.class, master).getNumberOfInFlightFetch(), equalTo(0))); // kick reroute a second time and check that all shards are unassigned assertThat(client(master).admin().cluster().prepareReroute().get().getState().getRoutingNodes().unassigned().size(), equalTo(2)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index b472cc5e0d2..b1c64d872dc 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -47,7 +47,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryA import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.snapshots.Snapshot; @@ -291,11 +290,11 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { public void testRebalanceDoesNotAllocatePrimaryAndReplicasOnDifferentVersionNodes() { ShardId shard1 = new ShardId("test1", "_na_", 0); ShardId shard2 = new ShardId("test2", "_na_", 0); - final DiscoveryNode newNode = new DiscoveryNode("newNode", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode newNode = new DiscoveryNode("newNode", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT); - final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); - final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); AllocationId allocationId1P = AllocationId.newInitializing(); AllocationId allocationId1R = AllocationId.newInitializing(); @@ -334,11 +333,11 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { } public void testRestoreDoesNotAllocateSnapshotOnOlderNodes() { - final DiscoveryNode newNode = new DiscoveryNode("newNode", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode newNode = new DiscoveryNode("newNode", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT); - final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode oldNode1 = new DiscoveryNode("oldNode1", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); - final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, VersionUtils.getPreviousVersion()); int numberOfShards = randomIntBetween(1, 3); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java index 66fe40793d5..e69d01ef5f2 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import static java.util.Collections.emptyMap; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; @@ -63,9 +62,9 @@ public class SameShardRoutingTests extends ESAllocationTestCase { logger.info("--> adding two nodes with the same host"); clusterState = ClusterState.builder(clusterState).nodes( DiscoveryNodes.builder() - .add(new DiscoveryNode("node1", "node1", "node1", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), + .add(new DiscoveryNode("node1", "node1", "node1", "test1", "test1", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT)) - .add(new DiscoveryNode("node2", "node2", "node2", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(), + .add(new DiscoveryNode("node2", "node2", "node2", "test1", "test1", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT))).build(); clusterState = strategy.reroute(clusterState, "reroute"); @@ -79,7 +78,7 @@ public class SameShardRoutingTests extends ESAllocationTestCase { logger.info("--> add another node, with a different host, replicas will be allocating"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .add(new DiscoveryNode("node3", "node3", "node3", "test2", "test2", LocalTransportAddress.buildUnique(), emptyMap(), + .add(new DiscoveryNode("node3", "node3", "node3", "test2", "test2", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT))).build(); clusterState = strategy.reroute(clusterState, "reroute"); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 062a018a82d..c80cc9a26b9 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -49,7 +49,6 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationComman import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.gateway.TestGatewayAllocator; import java.util.Arrays; @@ -823,9 +822,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .addAsNew(metaData.index("foo")) .build(); - DiscoveryNode discoveryNode1 = new DiscoveryNode("node1", new LocalTransportAddress("1"), emptyMap(), + DiscoveryNode discoveryNode1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT); - DiscoveryNode discoveryNode2 = new DiscoveryNode("node2", new LocalTransportAddress("2"), emptyMap(), + DiscoveryNode discoveryNode2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, Version.CURRENT); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(discoveryNode1).add(discoveryNode2).build(); @@ -942,9 +941,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { .build(); logger.info("--> adding one master node, one data node"); - DiscoveryNode discoveryNode1 = new DiscoveryNode("", "node1", new LocalTransportAddress("1"), emptyMap(), + DiscoveryNode discoveryNode1 = new DiscoveryNode("", "node1", buildNewFakeTransportAddress(), emptyMap(), singleton(DiscoveryNode.Role.MASTER), Version.CURRENT); - DiscoveryNode discoveryNode2 = new DiscoveryNode("", "node2", new LocalTransportAddress("2"), emptyMap(), + DiscoveryNode discoveryNode2 = new DiscoveryNode("", "node2", buildNewFakeTransportAddress(), emptyMap(), singleton(DiscoveryNode.Role.DATA), Version.CURRENT); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(discoveryNode1).add(discoveryNode2).build(); @@ -1010,7 +1009,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { // Add another datanode, it should relocate. logger.info("--> adding node3"); - DiscoveryNode discoveryNode3 = new DiscoveryNode("", "node3", new LocalTransportAddress("3"), emptyMap(), + DiscoveryNode discoveryNode3 = new DiscoveryNode("", "node3", buildNewFakeTransportAddress(), emptyMap(), singleton(DiscoveryNode.Role.DATA), Version.CURRENT); ClusterState updateClusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .add(discoveryNode3)).build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 5e8f3415273..d3e9259994c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -43,7 +43,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -70,9 +69,9 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { final Index index = metaData.index("test").getIndex(); ShardRouting test_0 = ShardRouting.newUnassigned(new ShardId(index, 0), true, StoreRecoverySource.EMPTY_STORE_INSTANCE, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); - DiscoveryNode node_0 = new DiscoveryNode("node_0", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node_0 = new DiscoveryNode("node_0", buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); - DiscoveryNode node_1 = new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node_1 = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); RoutingTable routingTable = RoutingTable.builder() @@ -108,9 +107,9 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss); ImmutableOpenMap.Builder shardRoutingMap = ImmutableOpenMap.builder(); - DiscoveryNode node_0 = new DiscoveryNode("node_0", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node_0 = new DiscoveryNode("node_0", buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); - DiscoveryNode node_1 = new DiscoveryNode("node_1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node_1 = new DiscoveryNode("node_1", buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.CURRENT); MetaData metaData = MetaData.builder() @@ -224,7 +223,7 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { assertEquals(100L, DiskThresholdDecider.getExpectedShardSize(test_1, allocation, 0)); assertEquals(10L, DiskThresholdDecider.getExpectedShardSize(test_0, allocation, 0)); - RoutingNode node = new RoutingNode("node1", new DiscoveryNode("node1", new LocalTransportAddress("test"), + RoutingNode node = new RoutingNode("node1", new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), test_0, test_1.getTargetRelocatingShard(), test_2); assertEquals(100L, DiskThresholdDecider.sizeOfRelocatingShards(node, allocation, false, "/dev/null")); assertEquals(90L, DiskThresholdDecider.sizeOfRelocatingShards(node, allocation, true, "/dev/null")); @@ -242,7 +241,7 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { other_0 = ShardRoutingHelper.moveToStarted(other_0); other_0 = ShardRoutingHelper.relocate(other_0, "node1"); - node = new RoutingNode("node1", new DiscoveryNode("node1", new LocalTransportAddress("test"), + node = new RoutingNode("node1", new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), test_0, test_1.getTargetRelocatingShard(), test_2, other_0.getTargetRelocatingShard()); if (other_0.primary()) { assertEquals(10100L, DiskThresholdDecider.sizeOfRelocatingShards(node, allocation, false, "/dev/null")); diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index 3259598f694..3f1e8f032ca 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -34,10 +34,8 @@ import org.elasticsearch.cluster.ESAllocationTestCase; import static org.hamcrest.Matchers.equalTo; -/** - * - */ public class ClusterSerializationTests extends ESAllocationTestCase { + public void testClusterStateSerialization() throws Exception { MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(10).numberOfReplicas(1)) diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java index 7c11e2b8c23..0dff5dc3998 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java @@ -29,7 +29,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.cluster.ESAllocationTestCase; import static java.util.Collections.emptyMap; @@ -50,7 +49,7 @@ public class ClusterStateToStringTests extends ESAllocationTestCase { .addAsNew(metaData.index("test_idx")) .build(); - DiscoveryNodes nodes = DiscoveryNodes.builder().add(new DiscoveryNode("node_foo", LocalTransportAddress.buildUnique(), + DiscoveryNodes nodes = DiscoveryNodes.builder().add(new DiscoveryNode("node_foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)).localNodeId("node_foo").masterNodeId("node_foo").build(); ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(nodes) diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java index b8527872d70..a25e3abe65d 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceIT.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Singleton; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -60,7 +61,6 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode public class ClusterServiceIT extends ESIntegTestCase { @Override @@ -68,6 +68,12 @@ public class ClusterServiceIT extends ESIntegTestCase { return Arrays.asList(TestPlugin.class); } + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + public void testAckedUpdateTask() throws Exception { Settings settings = Settings.builder() .put("discovery.type", "local") diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java index a0d6bc8e407..c12b54e71ef 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLogAppender; @@ -116,7 +115,7 @@ public class ClusterServiceTests extends ESTestCase { TimedClusterService timedClusterService = new TimedClusterService(Settings.builder().put("cluster.name", "ClusterServiceTests").build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool); - timedClusterService.setLocalNode(new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), emptyMap(), + timedClusterService.setLocalNode(new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); timedClusterService.setNodeConnectionsService(new NodeConnectionsService(Settings.EMPTY, null, null) { @Override diff --git a/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java b/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java index d86773d2d91..3e29068e672 100644 --- a/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java +++ b/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java @@ -36,7 +36,6 @@ import org.elasticsearch.plugins.NetworkPlugin; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.cat.AbstractCatAction; -import org.elasticsearch.test.transport.AssertingLocalTransport; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportInterceptor; @@ -51,12 +50,6 @@ import java.util.function.Supplier; public class NetworkModuleTests extends ModuleTestCase { - static class FakeTransport extends AssertingLocalTransport { - public FakeTransport() { - super(null, null, null, null); - } - } - static class FakeHttpTransport extends AbstractLifecycleComponent implements HttpServerTransport { public FakeHttpTransport() { super(null); @@ -110,7 +103,7 @@ public class NetworkModuleTests extends ModuleTestCase { Settings settings = Settings.builder().put(NetworkModule.TRANSPORT_TYPE_KEY, "custom") .put(NetworkModule.HTTP_ENABLED.getKey(), false) .build(); - Supplier custom = FakeTransport::new; + Supplier custom = () -> null; // content doesn't matter we check reference equality NetworkPlugin plugin = new NetworkPlugin() { @Override public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, @@ -166,7 +159,7 @@ public class NetworkModuleTests extends ModuleTestCase { .put(NetworkModule.HTTP_DEFAULT_TYPE_SETTING.getKey(), "default_custom") .put(NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING.getKey(), "local") .put(NetworkModule.TRANSPORT_TYPE_KEY, "default_custom").build(); - Supplier customTransport = FakeTransport::new; + Supplier customTransport = () -> null; // content doesn't matter we check reference equality Supplier custom = FakeHttpTransport::new; Supplier def = FakeHttpTransport::new; NetworkModule module = newNetworkModule(settings, false, new NetworkPlugin() { @@ -200,7 +193,7 @@ public class NetworkModuleTests extends ModuleTestCase { .put(NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING.getKey(), "default_custom").build(); Supplier custom = FakeHttpTransport::new; Supplier def = FakeHttpTransport::new; - Supplier customTransport = FakeTransport::new; + Supplier customTransport = () -> null; NetworkModule module = newNetworkModule(settings, false, new NetworkPlugin() { @Override public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, @@ -236,7 +229,7 @@ public class NetworkModuleTests extends ModuleTestCase { }; NetworkModule module = newNetworkModule(settings, false, new NetworkPlugin() { @Override - public List getTransportInterceptors() { + public List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry) { return Collections.singletonList(interceptor); } }); @@ -249,7 +242,7 @@ public class NetworkModuleTests extends ModuleTestCase { NullPointerException nullPointerException = expectThrows(NullPointerException.class, () -> { newNetworkModule(settings, false, new NetworkPlugin() { @Override - public List getTransportInterceptors() { + public List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry) { return Collections.singletonList(null); } }); diff --git a/core/src/test/java/org/elasticsearch/common/transport/BoundTransportAddressTests.java b/core/src/test/java/org/elasticsearch/common/transport/BoundTransportAddressTests.java index 1a3fa4db137..a3a1178473d 100644 --- a/core/src/test/java/org/elasticsearch/common/transport/BoundTransportAddressTests.java +++ b/core/src/test/java/org/elasticsearch/common/transport/BoundTransportAddressTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.common.transport; -import org.elasticsearch.common.io.stream.ByteBufferStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; @@ -40,12 +39,12 @@ public class BoundTransportAddressTests extends ESTestCase { public void testSerialization() throws Exception { InetAddress[] inetAddresses = InetAddress.getAllByName("0.0.0.0"); - List transportAddressList = new ArrayList<>(); + List transportAddressList = new ArrayList<>(); for (InetAddress address : inetAddresses) { - transportAddressList.add(new InetSocketTransportAddress(address, randomIntBetween(9200, 9299))); + transportAddressList.add(new TransportAddress(address, randomIntBetween(9200, 9299))); } final BoundTransportAddress transportAddress = - new BoundTransportAddress(transportAddressList.toArray(new InetSocketTransportAddress[0]), transportAddressList.get(0)); + new BoundTransportAddress(transportAddressList.toArray(new TransportAddress[0]), transportAddressList.get(0)); assertThat(transportAddress.boundAddresses().length, equalTo(transportAddressList.size())); // serialize @@ -75,7 +74,7 @@ public class BoundTransportAddressTests extends ESTestCase { public void testBadBoundAddressArray() { try { TransportAddress[] badArray = randomBoolean() ? null : new TransportAddress[0]; - new BoundTransportAddress(badArray, new InetSocketTransportAddress(InetAddress.getLoopbackAddress(), 80)); + new BoundTransportAddress(badArray, new TransportAddress(InetAddress.getLoopbackAddress(), 80)); fail("expected an exception to be thrown due to no bound address"); } catch (IllegalArgumentException e) { //expected diff --git a/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java b/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java index 481d15020fc..264e6ed0289 100644 --- a/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/BlockingClusterStatePublishResponseHandlerTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.discovery; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.test.ESTestCase; @@ -77,7 +76,7 @@ public class BlockingClusterStatePublishResponseHandlerTests extends ESTestCase int nodeCount = scaledRandomIntBetween(10, 20); DiscoveryNode[] allNodes = new DiscoveryNode[nodeCount]; for (int i = 0; i < nodeCount; i++) { - DiscoveryNode node = new DiscoveryNode("node_" + i, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("node_" + i, buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); allNodes[i] = node; } diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index 764e363d4d6..58a67769730 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -121,7 +121,6 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0) -@ESIntegTestCase.SuppressLocalMode @TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE") public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java index d51447c9298..2803448aa1c 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java @@ -29,9 +29,11 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.discovery.zen.fd.FaultDetection; import org.elasticsearch.discovery.zen.fd.MasterFaultDetection; import org.elasticsearch.discovery.zen.fd.NodesFaultDetection; @@ -41,10 +43,10 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.MockTcpTransport; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.local.LocalTransport; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.junit.After; @@ -141,12 +143,8 @@ public class ZenFaultDetectionTests extends ESTestCase { // trace zenfd actions but keep the default otherwise .put(TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), singleton(TransportLivenessAction.NAME)) .build(), - new LocalTransport(settings, threadPool, namedWriteableRegistry, circuitBreakerService) { - @Override - protected Version getVersion() { - return version; - } - }, + new MockTcpTransport(settings, threadPool, BigArrays.NON_RECYCLING_INSTANCE, circuitBreakerService, + namedWriteableRegistry, new NetworkService(settings, Collections.emptyList()), version), threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java b/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java index 95ad650cd16..14561f255cf 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java @@ -35,7 +35,6 @@ import java.util.concurrent.ExecutionException; import static org.hamcrest.Matchers.equalTo; @ClusterScope(scope = Scope.TEST, numDataNodes = 0) -@ESIntegTestCase.SuppressLocalMode public class ZenUnicastDiscoveryIT extends ESIntegTestCase { private ClusterDiscoveryConfiguration discoveryConfig; diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java index 737607df6be..b836d2cff9a 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java @@ -22,7 +22,6 @@ package org.elasticsearch.discovery.zen; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.ElectMasterService.MasterCandidate; import org.elasticsearch.test.ESTestCase; @@ -50,7 +49,7 @@ public class ElectMasterServiceTests extends ESTestCase { if (randomBoolean()) { roles.add(DiscoveryNode.Role.MASTER); } - DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), roles, Version.CURRENT); nodes.add(node); } @@ -65,7 +64,7 @@ public class ElectMasterServiceTests extends ESTestCase { for (int i = 0; i < count; i++) { Set roles = new HashSet<>(); roles.add(DiscoveryNode.Role.MASTER); - DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), roles, Version.CURRENT); candidates.add(new MasterCandidate(node, randomBoolean() ? MasterCandidate.UNRECOVERED_CLUSTER_VERSION : randomPositiveLong())); } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index 907d3786992..60842e624db 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.BaseFuture; @@ -427,7 +426,7 @@ public class NodeJoinControllerTests extends ESTestCase { public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException { ClusterState state = clusterService.state(); final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); - final DiscoveryNode other_node = new DiscoveryNode("other_node", LocalTransportAddress.buildUnique(), + final DiscoveryNode other_node = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); nodesBuilder.add(other_node); setState(clusterService, ClusterState.builder(state).nodes(nodesBuilder)); @@ -557,7 +556,7 @@ public class NodeJoinControllerTests extends ESTestCase { final DiscoveryNode other_node = new DiscoveryNode( randomBoolean() ? existing.getName() : "other_name", existing.getId(), - randomBoolean() ? existing.getAddress() : LocalTransportAddress.buildUnique(), + randomBoolean() ? existing.getAddress() : buildNewFakeTransportAddress(), randomBoolean() ? existing.getAttributes() : Collections.singletonMap("attr", "other"), randomBoolean() ? existing.getRoles() : new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), randomBoolean() ? existing.getVersion() : VersionUtils.randomVersion(random())); @@ -585,7 +584,7 @@ public class NodeJoinControllerTests extends ESTestCase { */ public void testElectionBasedOnConflictingNodes() throws InterruptedException, ExecutionException { final DiscoveryNode masterNode = clusterService.localNode(); - final DiscoveryNode otherNode = new DiscoveryNode("other_node", LocalTransportAddress.buildUnique(), emptyMap(), + final DiscoveryNode otherNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); // simulate master going down with stale nodes in it's cluster state (for example when min master nodes is set to 2) // also add some shards to that node @@ -629,7 +628,7 @@ public class NodeJoinControllerTests extends ESTestCase { setState(clusterService, stateBuilder.build()); final DiscoveryNode restartedNode = new DiscoveryNode(otherNode.getId(), - randomBoolean() ? otherNode.getAddress() : LocalTransportAddress.buildUnique(), otherNode.getAttributes(), + randomBoolean() ? otherNode.getAddress() : buildNewFakeTransportAddress(), otherNode.getAttributes(), otherNode.getRoles(), Version.CURRENT); nodeJoinController.startElectionContext(); @@ -669,7 +668,7 @@ public class NodeJoinControllerTests extends ESTestCase { ClusterState state = clusterService.state(); final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); for (int i = 0;i< count;i++) { - final DiscoveryNode node = new DiscoveryNode("node_" + state.nodes().getSize() + i, LocalTransportAddress.buildUnique(), + final DiscoveryNode node = new DiscoveryNode("node_" + state.nodes().getSize() + i, buildNewFakeTransportAddress(), emptyMap(), new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), Version.CURRENT); nodesBuilder.add(node); } @@ -752,6 +751,6 @@ public class NodeJoinControllerTests extends ESTestCase { roles.add(DiscoveryNode.Role.MASTER); } final String prefix = master ? "master_" : "data_"; - return new DiscoveryNode(prefix + i, i + "", new LocalTransportAddress("test_" + i), emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(prefix + i, i + "", buildNewFakeTransportAddress(), emptyMap(), roles, Version.CURRENT); } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeRemovalClusterStateTaskExecutorTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeRemovalClusterStateTaskExecutorTests.java index 4492cdb52e3..d63ddfdaee6 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeRemovalClusterStateTaskExecutorTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeRemovalClusterStateTaskExecutorTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -177,7 +176,7 @@ public class NodeRemovalClusterStateTaskExecutorTests extends ESTestCase { } private DiscoveryNode node(final int id) { - return new DiscoveryNode(Integer.toString(id), LocalTransportAddress.buildUnique(), Version.CURRENT); + return new DiscoveryNode(Integer.toString(id), buildNewFakeTransportAddress(), Version.CURRENT); } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java index d9a8c9be7f4..d336534a158 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryIT.java @@ -34,11 +34,11 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.discovery.Discovery; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoveryStats; import org.elasticsearch.discovery.zen.fd.FaultDetection; import org.elasticsearch.discovery.zen.membership.MembershipAction; @@ -75,12 +75,17 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0) -@ESIntegTestCase.SuppressLocalMode @TestLogging("_root:DEBUG") public class ZenDiscoveryIT extends ESIntegTestCase { private Version previousMajorVersion; + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + @Before public void computePrevMajorVersion() { Version previousMajor; @@ -204,7 +209,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase { assert node != null; DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(state.nodes()) - .add(new DiscoveryNode("abc", new LocalTransportAddress("abc"), emptyMap(), + .add(new DiscoveryNode("abc", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)).masterNodeId("abc"); ClusterState.Builder builder = ClusterState.builder(state); builder.nodes(nodes); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index a7291dc3736..ee5fe8a1aa8 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.zen.ping.ZenPing; import org.elasticsearch.discovery.zen.ping.ZenPingService; @@ -67,9 +66,9 @@ public class ZenDiscoveryUnitTests extends ESTestCase { ClusterName clusterName = new ClusterName("abc"); DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("a").add(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("a").add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder(); - newNodes.masterNodeId("a").add(new DiscoveryNode("a", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + newNodes.masterNodeId("a").add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); ClusterState.Builder currentState = ClusterState.builder(clusterName); currentState.nodes(currentNodes); @@ -87,7 +86,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase { assertFalse("should not ignore, because new state's version is higher to current state's version", shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("b").add(new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT)); + currentNodes.masterNodeId("b").add(new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); ; // version isn't taken into account, so randomize it to ensure this. if (randomBoolean()) { @@ -125,7 +124,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase { ArrayList allNodes = new ArrayList<>(); for (int i = randomIntBetween(10, 20); i >= 0; i--) { Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(Role.values()))); - DiscoveryNode node = new DiscoveryNode("node_" + i, "id_" + i, LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode node = new DiscoveryNode("node_" + i, "id_" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), roles, Version.CURRENT); responses.add(new ZenPing.PingResponse(node, randomBoolean() ? null : node, new ClusterName("test"), randomLong())); allNodes.add(node); @@ -155,7 +154,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase { try { Set expectedFDNodes = null; - final MockTransportService masterTransport = MockTransportService.local(settings, Version.CURRENT, threadPool); + final MockTransportService masterTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool); masterTransport.start(); DiscoveryNode masterNode = new DiscoveryNode("master", masterTransport.boundAddress().publishAddress(), Version.CURRENT); toClose.add(masterTransport); @@ -171,7 +170,7 @@ public class ZenDiscoveryUnitTests extends ESTestCase { toClose.add(masterZen); masterTransport.acceptIncomingRequests(); - final MockTransportService otherTransport = MockTransportService.local(settings, Version.CURRENT, threadPool); + final MockTransportService otherTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool); otherTransport.start(); toClose.add(otherTransport); DiscoveryNode otherNode = new DiscoveryNode("other", otherTransport.boundAddress().publishAddress(), Version.CURRENT); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java index 2275756e8ee..cad640b4dfa 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.ping.ZenPing; import org.elasticsearch.test.ESTestCase; @@ -43,7 +42,7 @@ public class ZenPingTests extends ESTestCase { long clusterStateVersionPerNode[] = new long[nodes.length]; ArrayList pings = new ArrayList<>(); for (int i = 0; i < nodes.length; i++) { - nodes[i] = new DiscoveryNode("" + i, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + nodes[i] = new DiscoveryNode("" + i, buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); } for (int pingCount = scaledRandomIntBetween(10, nodes.length * 10); pingCount > 0; pingCount--) { diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingTests.java index bdffb5f99d6..537139c8103 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingTests.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; @@ -229,16 +228,16 @@ public class UnicastZenPingTests extends ESTestCase { final DiscoveryNode node = new DiscoveryNode(nodeId, transportService.boundAddress().publishAddress(), emptyMap(), emptySet(), version); transportService.setLocalNode(node); - return new NetworkHandle((InetSocketTransportAddress)transport.boundAddress().publishAddress(), transportService, node, counters); + return new NetworkHandle((TransportAddress)transport.boundAddress().publishAddress(), transportService, node, counters); } private static class NetworkHandle { - public final InetSocketTransportAddress address; + public final TransportAddress address; public final TransportService transportService; public final DiscoveryNode node; public final ConcurrentMap counters; - public NetworkHandle(InetSocketTransportAddress address, TransportService transportService, DiscoveryNode discoveryNode, + public NetworkHandle(TransportAddress address, TransportService transportService, DiscoveryNode discoveryNode, ConcurrentMap counters) { this.address = address; this.transportService = transportService; diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java index 9bb8bf801f1..9ff09137bf9 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PendingClusterStatesQueueTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.discovery.zen.publish.PendingClusterStatesQueue.ClusterStateContext; import org.elasticsearch.test.ESTestCase; @@ -237,7 +236,7 @@ public class PendingClusterStatesQueueTests extends ESTestCase { ClusterState state = lastClusterStatePerMaster[masterIndex]; if (state == null) { state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(DiscoveryNodes.builder() - .add(new DiscoveryNode(masters[masterIndex], LocalTransportAddress.buildUnique(), + .add(new DiscoveryNode(masters[masterIndex], buildNewFakeTransportAddress(), emptyMap(), emptySet(),Version.CURRENT)).masterNodeId(masters[masterIndex]).build() ).build(); } else { diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java index 50ec06694fe..fee4030e1e1 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java @@ -229,7 +229,7 @@ public class PublishClusterStateActionTests extends ESTestCase { } private static MockTransportService buildTransportService(Settings settings, ThreadPool threadPool) { - MockTransportService transportService = MockTransportService.local(settings, Version.CURRENT, threadPool); + MockTransportService transportService = MockTransportService.createNewService(settings, Version.CURRENT, threadPool); transportService.start(); transportService.acceptIncomingRequests(); return transportService; diff --git a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java index 043eaa2708f..1f2dbd608bb 100644 --- a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.support.nodes.BaseNodeResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -45,11 +44,11 @@ import static org.hamcrest.Matchers.sameInstance; /** */ public class AsyncShardFetchTests extends ESTestCase { - private final DiscoveryNode node1 = new DiscoveryNode("node1", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response1 = new Response(node1); private final Throwable failure1 = new Throwable("simulated failure 1"); - private final DiscoveryNode node2 = new DiscoveryNode("node2", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + private final DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); private final Response response2 = new Response(node2); private final Throwable failure2 = new Throwable("simulate failure 2"); diff --git a/core/src/test/java/org/elasticsearch/http/HttpServerTests.java b/core/src/test/java/org/elasticsearch/http/HttpServerTests.java index 87167cdb733..eb22e5bac47 100644 --- a/core/src/test/java/org/elasticsearch/http/HttpServerTests.java +++ b/core/src/test/java/org/elasticsearch/http/HttpServerTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -142,7 +141,7 @@ public class HttpServerTests extends ESTestCase { @Override public BoundTransportAddress boundAddress() { - LocalTransportAddress transportAddress = new LocalTransportAddress("1"); + TransportAddress transportAddress = buildNewFakeTransportAddress(); return new BoundTransportAddress(new TransportAddress[] {transportAddress} ,transportAddress); } diff --git a/core/src/test/java/org/elasticsearch/index/TransportIndexFailuresIT.java b/core/src/test/java/org/elasticsearch/index/TransportIndexFailuresIT.java index 1ead4554a11..91a18f9c053 100644 --- a/core/src/test/java/org/elasticsearch/index/TransportIndexFailuresIT.java +++ b/core/src/test/java/org/elasticsearch/index/TransportIndexFailuresIT.java @@ -49,7 +49,6 @@ import static org.hamcrest.Matchers.equalTo; * Test failure when index replication actions fail mid-flight */ @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, transportClientRatio = 0) -@ESIntegTestCase.SuppressLocalMode public class TransportIndexFailuresIT extends ESIntegTestCase { private static final Settings nodeSettings = Settings.builder() diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java index d634d8cd4fe..092e93f908a 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java @@ -29,16 +29,19 @@ import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.MockTcpTransport; +import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.local.LocalTransport; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -54,7 +57,7 @@ public class DynamicMappingDisabledTests extends ESSingleNodeTestCase { private static ThreadPool THREAD_POOL; private ClusterService clusterService; - private LocalTransport transport; + private Transport transport; private TransportService transportService; private IndicesService indicesService; private ShardStateAction shardStateAction; @@ -75,8 +78,9 @@ public class DynamicMappingDisabledTests extends ESSingleNodeTestCase { .put(MapperService.INDEX_MAPPER_DYNAMIC_SETTING.getKey(), false) .build(); clusterService = createClusterService(THREAD_POOL); - transport = new LocalTransport(settings, THREAD_POOL, new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()); + transport = new MockTcpTransport(settings, THREAD_POOL, BigArrays.NON_RECYCLING_INSTANCE, + new NoneCircuitBreakerService(), new NamedWriteableRegistry(Collections.emptyList()), + new NetworkService(settings, Collections.emptyList())); transportService = new TransportService(clusterService.getSettings(), transport, THREAD_POOL, TransportService.NOOP_TRANSPORT_INTERCEPTOR); indicesService = getInstanceFromNode(IndicesService.class); diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 6e200c4756a..9ccfd7243a5 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -39,7 +39,6 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.Uid; @@ -90,7 +89,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } protected DiscoveryNode getDiscoveryNode(String id) { - return new DiscoveryNode(id, id, new LocalTransportAddress(id), Collections.emptyMap(), + return new DiscoveryNode(id, id, buildNewFakeTransportAddress(), Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index fc943bcebe9..2248ff156ac 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -42,7 +42,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.env.Environment; @@ -448,7 +447,7 @@ public class IndexShardIT extends ESSingleNodeTestCase { public static final IndexShard recoverShard(IndexShard newShard) throws IOException { - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); assertTrue(newShard.recoverFromStore()); newShard.updateRoutingEntry(newShard.routingEntry().moveToStarted()); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b5d3d69705a..335cffba493 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -58,7 +58,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -423,7 +422,7 @@ public class IndexShardTests extends IndexShardTestCase { flushShard(shard); final IndexShard newShard = reinitShard(shard); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); Store.MetadataSnapshot snapshot = newShard.snapshotStoreMetadata(); assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_2")); @@ -863,7 +862,7 @@ public class IndexShardTests extends IndexShardTestCase { translogOps = 0; } IndexShard newShard = reinitShard(shard); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); assertTrue(newShard.recoverFromStore()); assertEquals(translogOps, newShard.recoveryState().getTranslog().recoveredOperations()); @@ -886,7 +885,7 @@ public class IndexShardTests extends IndexShardTestCase { ShardRoutingHelper.initWithSameId(shardRouting, RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE) ); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); assertTrue(newShard.recoverFromStore()); assertEquals(0, newShard.recoveryState().getTranslog().recoveredOperations()); @@ -911,7 +910,7 @@ public class IndexShardTests extends IndexShardTestCase { cleanLuceneIndex(store.directory()); store.decRef(); IndexShard newShard = reinitShard(shard); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); ShardRouting routing = newShard.routingEntry(); newShard.markAsRecovering("store", new RecoveryState(routing, localNode, null)); try { @@ -985,7 +984,7 @@ public class IndexShardTests extends IndexShardTestCase { Store sourceStore = source.store(); Store targetStore = target.store(); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); target.markAsRecovering("store", new RecoveryState(routing, localNode, null)); assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") { @Override @@ -1224,7 +1223,7 @@ public class IndexShardTests extends IndexShardTestCase { IndexShard shard = newStartedShard(true); indexDoc(shard, "type", "0"); shard = reinitShard(shard); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); shard.markAsRecovering("for testing", new RecoveryState(shard.routingEntry(), localNode, null)); // Shard is still inactive since we haven't started recovering yet assertFalse(shard.isActive()); @@ -1251,7 +1250,7 @@ public class IndexShardTests extends IndexShardTestCase { indexDoc(primary, "test", "0", "{\"foo\" : \"bar\"}"); IndexShard replica = newShard(primary.shardId(), false, "n2", metaData, null); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode)); // Shard is still inactive since we haven't started recovering yet assertFalse(replica.isActive()); @@ -1299,7 +1298,7 @@ public class IndexShardTests extends IndexShardTestCase { ShardRoutingState.INITIALIZING, RecoverySource.LocalShardsRecoverySource.INSTANCE); final IndexShard targetShard; - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); Map requestedMappingUpdates = ConcurrentCollections.newConcurrentMap(); { targetShard = newShard(targetRouting); diff --git a/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java b/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java index 2d0e4a3aeb9..e9a717f6636 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndexingMemoryControllerTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.IndexService; @@ -447,7 +446,7 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase { try { assertEquals(0, imc.availableShards().size()); ShardRouting routing = newShard.routingEntry(); - DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(routing, localNode, null)); assertEquals(1, imc.availableShards().size()); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 66d6c16b4e0..c6ab63a5eed 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.NodeServicesProvider; @@ -106,7 +105,7 @@ public class IndicesLifecycleListenerSingleNodeTests extends ESSingleNodeTestCas newRouting = ShardRoutingHelper.initialize(newRouting, nodeId); IndexShard shard = index.createShard(newRouting); shard.updateRoutingEntry(newRouting); - final DiscoveryNode localNode = new DiscoveryNode("foo", LocalTransportAddress.buildUnique(), + final DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); shard.markAsRecovering("store", new RecoveryState(newRouting, localNode, null)); shard.recoverFromStore(); diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index 48a105874fb..85912552d7a 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -43,7 +43,6 @@ import org.elasticsearch.cluster.routing.allocation.FailedShard; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; @@ -353,7 +352,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice roles.add(mustHaveRole); } final String id = String.format(Locale.ROOT, "node_%03d", nodeIdGenerator.incrementAndGet()); - return new DiscoveryNode(id, id, LocalTransportAddress.buildUnique(), Collections.emptyMap(), roles, + return new DiscoveryNode(id, id, buildNewFakeTransportAddress(), Collections.emptyMap(), roles, Version.CURRENT); } diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 0678ebef99f..1dce300ef0c 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -35,13 +35,11 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.store.IndexOutputOutputStream; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRelocatedException; @@ -82,8 +80,8 @@ public class RecoverySourceHandlerTests extends ESTestCase { put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), null, randomBoolean(), randomLong()); Store store = newStore(createTempDir()); RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, () -> 0L, e -> () -> {}, @@ -134,8 +132,8 @@ public class RecoverySourceHandlerTests extends ESTestCase { put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), null, randomBoolean(), randomLong()); Path tempDir = createTempDir(); Store store = newStore(tempDir, false); @@ -198,8 +196,8 @@ public class RecoverySourceHandlerTests extends ESTestCase { put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), null, randomBoolean(), randomLong()); Path tempDir = createTempDir(); Store store = newStore(tempDir, false); @@ -257,8 +255,8 @@ public class RecoverySourceHandlerTests extends ESTestCase { public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Completed() throws IOException { final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), null, false, randomLong()); IndexShard shard = mock(IndexShard.class); Translog.View translogView = mock(Translog.View.class); @@ -287,8 +285,8 @@ public class RecoverySourceHandlerTests extends ESTestCase { public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, InterruptedException { final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), null, true, randomLong()); AtomicBoolean phase1Called = new AtomicBoolean(); AtomicBoolean phase2Called = new AtomicBoolean(); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryStatusTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryStatusTests.java index 55359a935f6..2c00c59c343 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryStatusTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryStatusTests.java @@ -22,7 +22,6 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.store.IndexOutput; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.shard.IndexShard; @@ -42,7 +41,7 @@ public class RecoveryStatusTests extends ESSingleNodeTestCase { IndexService service = createIndex("foo"); IndexShard indexShard = service.getShardOrNull(0); - DiscoveryNode node = new DiscoveryNode("foo", new LocalTransportAddress("bar"), emptyMap(), emptySet(), Version.CURRENT); + DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); RecoveryTarget status = new RecoveryTarget(indexShard, node, new PeerRecoveryTargetService.RecoveryListener() { @Override public void onRecoveryDone(RecoveryState state) { diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java index 6d37ae7d0d7..1245725e055 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTargetTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.recovery.RecoveryState.File; import org.elasticsearch.indices.recovery.RecoveryState.Index; @@ -342,7 +341,7 @@ public class RecoveryTargetTests extends ESTestCase { } public void testStageSequenceEnforcement() { - final DiscoveryNode discoveryNode = new DiscoveryNode("1", LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), + final DiscoveryNode discoveryNode = new DiscoveryNode("1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); Stage[] stages = Stage.values(); int i = randomIntBetween(0, stages.length - 1); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java index 7065ffa5dfc..8fa6e3d03a6 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.test.ESTestCase; @@ -43,8 +42,8 @@ public class StartRecoveryRequestTests extends ESTestCase { Version targetNodeVersion = randomVersion(random()); StartRecoveryRequest outRequest = new StartRecoveryRequest( new ShardId("test", "_na_", 0), - new DiscoveryNode("a", new LocalTransportAddress("1"), emptyMap(), emptySet(), targetNodeVersion), - new DiscoveryNode("b", new LocalTransportAddress("1"), emptyMap(), emptySet(), targetNodeVersion), + new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), targetNodeVersion), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), targetNodeVersion), Store.MetadataSnapshot.EMPTY, randomBoolean(), 1L diff --git a/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java b/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java index 2ca8947cbf8..df73e11b731 100644 --- a/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java +++ b/core/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java @@ -41,7 +41,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; @@ -76,9 +75,15 @@ import static org.hamcrest.Matchers.instanceOf; /** */ @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0, transportClientRatio = 0) -@ESIntegTestCase.SuppressLocalMode @TestLogging("_root:DEBUG") public class RareClusterStateIT extends ESIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + @Override protected int numberOfShards() { return 1; @@ -126,7 +131,7 @@ public class RareClusterStateIT extends ESIntegTestCase { // inject a node ClusterState.Builder builder = ClusterState.builder(currentState); builder.nodes(DiscoveryNodes.builder(currentState.nodes()).add(new DiscoveryNode("_non_existent", - LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT))); + buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT))); // open index final IndexMetaData indexMetaData = IndexMetaData.builder(currentState.metaData().index(index)).state(IndexMetaData.State.OPEN).build(); diff --git a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java index 62b5bc30a68..641971896b3 100644 --- a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java +++ b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -86,7 +85,7 @@ public class IndicesStoreTests extends ESTestCase { @Before public void before() { - localNode = new DiscoveryNode("abc", new LocalTransportAddress("abc"), emptyMap(), emptySet(), Version.CURRENT); + localNode = new DiscoveryNode("abc", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); clusterService = createClusterService(threadPool); indicesStore = new IndicesStore(Settings.EMPTY, null, clusterService, new TransportService(clusterService.getSettings(), null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR), null); } @@ -142,7 +141,7 @@ public class IndicesStoreTests extends ESTestCase { ClusterState.Builder clusterState = ClusterState.builder(new ClusterName("test")); clusterState.metaData(MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numShards).numberOfReplicas(numReplicas))); clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.getId()).add(localNode).add(new DiscoveryNode("xyz", - new LocalTransportAddress("xyz"), emptyMap(), emptySet(), Version.CURRENT))); + buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT))); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", "_na_", 1)); int localShardId = randomInt(numShards - 1); for (int i = 0; i < numShards; i++) { @@ -187,7 +186,7 @@ public class IndicesStoreTests extends ESTestCase { ClusterState.Builder clusterState = ClusterState.builder(new ClusterName("test")); clusterState.metaData(MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numShards).numberOfReplicas(numReplicas))); clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.getId()).add(localNode).add(new DiscoveryNode("xyz", - new LocalTransportAddress("xyz"), emptyMap(), emptySet(), nodeVersion))); + buildNewFakeTransportAddress(), emptyMap(), emptySet(), nodeVersion))); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", "_na_", 1)); for (int i = 0; i < numShards; i++) { routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, true, ShardRoutingState.STARTED)); @@ -210,8 +209,8 @@ public class IndicesStoreTests extends ESTestCase { clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.getId()) .add(localNode) - .add(new DiscoveryNode("xyz", new LocalTransportAddress("xyz"), emptyMap(), emptySet(), Version.CURRENT)) - .add(new DiscoveryNode("def", new LocalTransportAddress("def"), emptyMap(), emptySet(), nodeVersion) // <-- only set relocating, since we're testing that in this test + .add(new DiscoveryNode("xyz", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)) + .add(new DiscoveryNode("def", buildNewFakeTransportAddress(), emptyMap(), emptySet(), nodeVersion) // <-- only set relocating, since we're testing that in this test )); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", "_na_", 1)); for (int i = 0; i < numShards; i++) { diff --git a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java index cdbe1e11570..4e90f5346d2 100644 --- a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java +++ b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java @@ -30,7 +30,6 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -264,9 +263,9 @@ public class PipelineStoreTests extends ESTestCase { PutPipelineRequest putRequest = new PutPipelineRequest("_id", new BytesArray( "{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}},{\"remove\" : {\"field\": \"_field\"}}]}")); - DiscoveryNode node1 = new DiscoveryNode("_node_id1", new LocalTransportAddress("_id"), + DiscoveryNode node1 = new DiscoveryNode("_node_id1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); - DiscoveryNode node2 = new DiscoveryNode("_node_id2", new LocalTransportAddress("_id"), + DiscoveryNode node2 = new DiscoveryNode("_node_id2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); Map ingestInfos = new HashMap<>(); ingestInfos.put(node1, new IngestInfo(Arrays.asList(new ProcessorInfo("set"), new ProcessorInfo("remove")))); @@ -293,7 +292,7 @@ public class PipelineStoreTests extends ESTestCase { assertThat(e.getMessage(), equalTo("Ingest info is empty")); } - DiscoveryNode discoveryNode = new DiscoveryNode("_node_id", new LocalTransportAddress("_id"), + DiscoveryNode discoveryNode = new DiscoveryNode("_node_id", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); IngestInfo ingestInfo = new IngestInfo(Collections.singletonList(new ProcessorInfo("set"))); store.validatePipeline(Collections.singletonMap(discoveryNode, ingestInfo), putRequest); diff --git a/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java b/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java index d9134ba5cf3..59077367904 100644 --- a/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java +++ b/core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ToXContent; @@ -106,7 +105,7 @@ public class NodeInfoStreamingTests extends ESTestCase { private static NodeInfo createNodeInfo() { Build build = Build.CURRENT; - DiscoveryNode node = new DiscoveryNode("test_node", LocalTransportAddress.buildUnique(), + DiscoveryNode node = new DiscoveryNode("test_node", buildNewFakeTransportAddress(), emptyMap(), emptySet(), VersionUtils.randomVersion(random())); Settings settings = randomBoolean() ? null : Settings.builder().put("test", "setting").build(); OsInfo osInfo = null; @@ -133,7 +132,7 @@ public class NodeInfoStreamingTests extends ESTestCase { } Map profileAddresses = new HashMap<>(); BoundTransportAddress dummyBoundTransportAddress = new BoundTransportAddress( - new TransportAddress[]{LocalTransportAddress.buildUnique()}, LocalTransportAddress.buildUnique()); + new TransportAddress[]{buildNewFakeTransportAddress()}, buildNewFakeTransportAddress()); profileAddresses.put("test_address", dummyBoundTransportAddress); TransportInfo transport = randomBoolean() ? null : new TransportInfo(dummyBoundTransportAddress, profileAddresses); HttpInfo httpInfo = randomBoolean() ? null : new HttpInfo(dummyBoundTransportAddress, randomLong()); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java index 9be3a83bbc9..dcc80f69903 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java @@ -21,19 +21,15 @@ package org.elasticsearch.search.aggregations.bucket; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.transport.AssertingLocalTransport; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.junit.After; import org.junit.Before; import java.io.IOException; -import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutionException; @@ -58,11 +54,6 @@ public class DateHistogramOffsetIT extends ESIntegTestCase { return DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parser().parseDateTime(date); } - @Override - protected Collection> nodePlugins() { - return Collections.singleton(AssertingLocalTransport.TestPlugin.class); - } - @Before public void beforeEachTest() throws IOException { prepareCreate("idx2").addMapping("type", "date", "type=date").execute().actionGet(); diff --git a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index 3a045c80ac8..a987aa7f9cc 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -44,6 +44,7 @@ import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.index.store.IndexStore; @@ -92,8 +93,15 @@ import static org.hamcrest.Matchers.nullValue; /** */ @ClusterScope(scope = Scope.TEST, numDataNodes = 0, transportClientRatio = 0) -@ESIntegTestCase.SuppressLocalMode // TODO only restorePersistentSettingsTest needs this maybe factor out? public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + // TODO only restorePersistentSettingsTest needs this maybe factor out? + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "zen").build(); + } + @Override protected Collection> nodePlugins() { return Arrays.asList(MockRepository.Plugin.class); diff --git a/core/src/test/java/org/elasticsearch/transport/local/SimpleLocalTransportTests.java b/core/src/test/java/org/elasticsearch/transport/local/SimpleLocalTransportTests.java deleted file mode 100644 index c4458b7883d..00000000000 --- a/core/src/test/java/org/elasticsearch/transport/local/SimpleLocalTransportTests.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.transport.local; - -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.AbstractSimpleTransportTestCase; - -public class SimpleLocalTransportTests extends AbstractSimpleTransportTestCase { - - @Override - protected MockTransportService build(Settings settings, Version version) { - MockTransportService transportService = MockTransportService.local(settings, version, threadPool); - transportService.start(); - return transportService; - } -} diff --git a/core/src/test/java/org/elasticsearch/tribe/TribeIT.java b/core/src/test/java/org/elasticsearch/tribe/TribeIT.java index 440859dce44..3d8689d5ca9 100644 --- a/core/src/test/java/org/elasticsearch/tribe/TribeIT.java +++ b/core/src/test/java/org/elasticsearch/tribe/TribeIT.java @@ -44,6 +44,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.NodeConfigurationSource; +import org.elasticsearch.transport.MockTcpTransportPlugin; import org.elasticsearch.transport.Transport; import org.junit.After; import org.junit.AfterClass; @@ -193,15 +194,15 @@ public class TribeIT extends ESIntegTestCase { settings.put(Node.NODE_DATA_SETTING.getKey(), false); settings.put(Node.NODE_MASTER_SETTING.getKey(), true); settings.put(NetworkModule.HTTP_ENABLED.getKey(), false); - settings.put(NetworkModule.TRANSPORT_TYPE_SETTING.getKey(), NetworkModule.LOCAL_TRANSPORT); - settings.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), NetworkModule.LOCAL_TRANSPORT); + settings.put(NetworkModule.TRANSPORT_TYPE_SETTING.getKey(), MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME); + settings.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local"); doWithAllClusters(filter, c -> { String tribeSetting = "tribe." + c.getClusterName() + "."; settings.put(tribeSetting + ClusterName.CLUSTER_NAME_SETTING.getKey(), c.getClusterName()); settings.put(tribeSetting + DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "100ms"); - settings.put(tribeSetting + NetworkModule.TRANSPORT_TYPE_SETTING.getKey(), NetworkModule.LOCAL_TRANSPORT); - settings.put(tribeSetting + DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), NetworkModule.LOCAL_TRANSPORT); + settings.put(tribeSetting + NetworkModule.TRANSPORT_TYPE_SETTING.getKey(), MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME); + settings.put(tribeSetting + DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local"); Set hosts = new HashSet<>(); for (Transport transport : c.getInstances(Transport.class)) { diff --git a/docs/reference/modules/transport.asciidoc b/docs/reference/modules/transport.asciidoc index bf9b6e8f9d4..231666d6bed 100644 --- a/docs/reference/modules/transport.asciidoc +++ b/docs/reference/modules/transport.asciidoc @@ -80,13 +80,6 @@ The following parameters can be configured like that * `tcp_send_buffer_size`: Configures the send buffer size of the socket * `tcp_receive_buffer_size`: Configures the receive buffer size of the socket -[float] -=== Local Transport - -This is a handy transport to use when running integration tests within -the JVM. It is automatically enabled when using -`NodeBuilder#local(true)`. - [float] === Transport Tracer diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java index f5fb927cda9..edd7d78a28f 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java @@ -56,7 +56,6 @@ import org.elasticsearch.client.ParentTaskAssigningClient; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.Text; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; @@ -141,7 +140,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { expectedHeaders.put(randomSimpleString(random()), randomSimpleString(random())); threadPool.getThreadContext().newStoredContext(); threadPool.getThreadContext().putHeader(expectedHeaders); - localNode = new DiscoveryNode("thenode", new LocalTransportAddress("dead.end:666"), emptyMap(), emptySet(), Version.CURRENT); + localNode = new DiscoveryNode("thenode", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); taskId = new TaskId(localNode.getId(), testTask.getId()); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java index 2e41550134e..2ea8fa6514d 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWhitelistTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.index.reindex.remote.RemoteInfo; import org.elasticsearch.test.ESTestCase; @@ -43,7 +42,7 @@ public class ReindexFromRemoteWhitelistTests extends ESTestCase { @Before public void setupLocalhost() throws UnknownHostException { - localhost = new InetSocketTransportAddress(InetAddress.getByAddress(new byte[] { 0x7f, 0x00, 0x00, 0x01 }), 9200); + localhost = new TransportAddress(InetAddress.getByAddress(new byte[] { 0x7f, 0x00, 0x00, 0x01 }), 9200); } public void testLocalRequestWithoutWhitelist() { @@ -66,7 +65,7 @@ public class ReindexFromRemoteWhitelistTests extends ESTestCase { public void testMyselfInWhitelistRemote() throws UnknownHostException { Set whitelist = randomWhitelist(); whitelist.add("myself"); - TransportAddress publishAddress = new InetSocketTransportAddress(InetAddress.getByAddress(new byte[] {0x7f,0x00,0x00,0x01}), 9200); + TransportAddress publishAddress = new TransportAddress(InetAddress.getByAddress(new byte[] {0x7f,0x00,0x00,0x01}), 9200); checkRemoteWhitelist(whitelist, new RemoteInfo(randomAsciiOfLength(5), "127.0.0.1", 9200, new BytesArray("test"), null, null, emptyMap()), publishAddress); } diff --git a/modules/transport-netty3/src/main/java/org/elasticsearch/http/netty3/Netty3HttpServerTransport.java b/modules/transport-netty3/src/main/java/org/elasticsearch/http/netty3/Netty3HttpServerTransport.java index c60f47ee3da..e961f497bcd 100644 --- a/modules/transport-netty3/src/main/java/org/elasticsearch/http/netty3/Netty3HttpServerTransport.java +++ b/modules/transport-netty3/src/main/java/org/elasticsearch/http/netty3/Netty3HttpServerTransport.java @@ -31,10 +31,9 @@ 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.TransportAddress; import org.elasticsearch.common.transport.NetworkExceptionHelper; import org.elasticsearch.common.transport.PortsRange; -import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; @@ -321,7 +320,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem throw new BindHttpException("Failed to resolve host [" + Arrays.toString(bindHosts) + "]", e); } - List boundAddresses = new ArrayList<>(hostAddresses.length); + List boundAddresses = new ArrayList<>(hostAddresses.length); for (InetAddress address : hostAddresses) { boundAddresses.add(bindAddress(address)); } @@ -335,15 +334,15 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem final int publishPort = resolvePublishPort(settings, boundAddresses, publishInetAddress); final InetSocketAddress publishAddress = new InetSocketAddress(publishInetAddress, publishPort); - return new BoundTransportAddress(boundAddresses.toArray(new TransportAddress[0]), new InetSocketTransportAddress(publishAddress)); + return new BoundTransportAddress(boundAddresses.toArray(new TransportAddress[0]), new TransportAddress(publishAddress)); } // package private for tests - static int resolvePublishPort(Settings settings, List boundAddresses, InetAddress publishInetAddress) { + static int resolvePublishPort(Settings settings, List boundAddresses, InetAddress publishInetAddress) { int publishPort = SETTING_HTTP_PUBLISH_PORT.get(settings); if (publishPort < 0) { - for (InetSocketTransportAddress boundAddress : boundAddresses) { + for (TransportAddress boundAddress : boundAddresses) { InetAddress boundInetAddress = boundAddress.address().getAddress(); if (boundInetAddress.isAnyLocalAddress() || boundInetAddress.equals(publishInetAddress)) { publishPort = boundAddress.getPort(); @@ -355,7 +354,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem // 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 (InetSocketTransportAddress boundAddress : boundAddresses) { + for (TransportAddress boundAddress : boundAddresses) { ports.add(boundAddress.getPort()); } if (ports.size() == 1) { @@ -400,7 +399,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem .build(); } - private InetSocketTransportAddress bindAddress(final InetAddress hostAddress) { + private TransportAddress bindAddress(final InetAddress hostAddress) { final AtomicReference lastException = new AtomicReference<>(); final AtomicReference boundSocket = new AtomicReference<>(); boolean success = port.iterate(new PortsRange.PortCallback() { @@ -426,7 +425,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem if (logger.isDebugEnabled()) { logger.debug("Bound http to address {{}}", NetworkAddress.format(boundSocket.get())); } - return new InetSocketTransportAddress(boundSocket.get()); + return new TransportAddress(boundSocket.get()); } @Override diff --git a/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Transport.java b/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Transport.java index 8d1a6edd78b..4531811b983 100644 --- a/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Transport.java +++ b/modules/transport-netty3/src/main/java/org/elasticsearch/transport/netty3/Netty3Transport.java @@ -33,7 +33,7 @@ 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.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -332,7 +332,7 @@ public class Netty3Transport extends TcpTransport { } protected NodeChannels connectToChannelsLight(DiscoveryNode node) { - InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address(); + InetSocketAddress address = node.getAddress().address(); ChannelFuture connect = clientBootstrap.connect(address); connect.awaitUninterruptibly((long) (connectTimeout.millis() * 1.5)); if (!connect.isSuccess()) { @@ -352,7 +352,7 @@ public class Netty3Transport extends TcpTransport { int numConnections = connectionsPerNodeBulk + connectionsPerNodePing + connectionsPerNodeRecovery + connectionsPerNodeReg + connectionsPerNodeState; ArrayList connections = new ArrayList<>(); - InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address(); + InetSocketAddress address = node.getAddress().address(); for (int i = 0; i < numConnections; i++) { connections.add(clientBootstrap.connect(address)); } diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/ESNetty3IntegTestCase.java b/modules/transport-netty3/src/test/java/org/elasticsearch/ESNetty3IntegTestCase.java index 20570536ae2..09a8641253f 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/ESNetty3IntegTestCase.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/ESNetty3IntegTestCase.java @@ -28,7 +28,6 @@ import org.elasticsearch.transport.netty3.Netty3Transport; import java.util.Arrays; import java.util.Collection; -@ESIntegTestCase.SuppressLocalMode public abstract class ESNetty3IntegTestCase extends ESIntegTestCase { @Override diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpPublishPortTests.java b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpPublishPortTests.java index 05c7ee36a24..889cb50e5fe 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpPublishPortTests.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpPublishPortTests.java @@ -21,7 +21,7 @@ package org.elasticsearch.http.netty3; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.BindHttpException; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.test.ESTestCase; @@ -73,16 +73,16 @@ public class Netty3HttpPublishPortTests extends ESTestCase { } } - private InetSocketTransportAddress address(String host, int port) throws UnknownHostException { - return new InetSocketTransportAddress(getByName(host), port); + private TransportAddress address(String host, int port) throws UnknownHostException { + return new TransportAddress(getByName(host), port); } - private InetSocketTransportAddress randomAddress() throws UnknownHostException { + private TransportAddress randomAddress() throws UnknownHostException { return address("127.0.0." + randomIntBetween(1, 100), randomIntBetween(9200, 9300)); } - private List randomAddresses() throws UnknownHostException { - List addresses = new ArrayList<>(); + private List randomAddresses() throws UnknownHostException { + List addresses = new ArrayList<>(); for (int i = 0; i < randomIntBetween(1, 5); i++) { addresses.add(randomAddress()); } diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpRequestSizeLimitIT.java b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpRequestSizeLimitIT.java index 66d9f2c88d1..d6da2cab9d5 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpRequestSizeLimitIT.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpRequestSizeLimitIT.java @@ -22,7 +22,7 @@ import org.elasticsearch.ESNetty3IntegTestCase; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.http.HttpServerTransport; @@ -78,15 +78,15 @@ public class Netty3HttpRequestSizeLimitIT extends ESNetty3IntegTestCase { } HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress ().boundAddresses()); try (Netty3HttpClient nettyHttpClient = new Netty3HttpClient()) { - Collection singleResponse = nettyHttpClient.post(inetSocketTransportAddress.address(), requests[0]); + Collection singleResponse = nettyHttpClient.post(transportAddress.address(), requests[0]); assertThat(singleResponse, hasSize(1)); assertAtLeastOnceExpectedStatus(singleResponse, HttpResponseStatus.OK); - Collection multipleResponses = nettyHttpClient.post(inetSocketTransportAddress.address(), requests); + Collection multipleResponses = nettyHttpClient.post(transportAddress.address(), requests); assertThat(multipleResponses, hasSize(requests.length)); assertAtLeastOnceExpectedStatus(multipleResponses, HttpResponseStatus.SERVICE_UNAVAILABLE); } @@ -103,11 +103,11 @@ public class Netty3HttpRequestSizeLimitIT extends ESNetty3IntegTestCase { } HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress ().boundAddresses()); try (Netty3HttpClient nettyHttpClient = new Netty3HttpClient()) { - Collection responses = nettyHttpClient.put(inetSocketTransportAddress.address(), requestUris); + Collection responses = nettyHttpClient.put(transportAddress.address(), requestUris); assertThat(responses, hasSize(requestUris.length)); assertAllInExpectedStatus(responses, HttpResponseStatus.OK); } diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpServerPipeliningTests.java b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpServerPipeliningTests.java index b432708de15..29fc543c4d2 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpServerPipeliningTests.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3HttpServerPipeliningTests.java @@ -20,7 +20,7 @@ package org.elasticsearch.http.netty3; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.http.netty3.Netty3HttpServerTransport.HttpChannelPipelineFactory; @@ -95,7 +95,7 @@ public class Netty3HttpServerPipeliningTests extends ESTestCase { .build(); httpServerTransport = new CustomNetty3HttpServerTransport(settings); httpServerTransport.start(); - InetSocketTransportAddress transportAddress = (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress() + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress() .boundAddresses()); List requests = Arrays.asList("/firstfast", "/slow?sleep=500", "/secondfast", "/slow?sleep=1000", "/thirdfast"); @@ -113,7 +113,7 @@ public class Netty3HttpServerPipeliningTests extends ESTestCase { .build(); httpServerTransport = new CustomNetty3HttpServerTransport(settings); httpServerTransport.start(); - InetSocketTransportAddress transportAddress = (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress() + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress() .boundAddresses()); List requests = Arrays.asList("/slow?sleep=1000", "/firstfast", "/secondfast", "/thirdfast", "/slow?sleep=500"); diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3PipeliningDisabledIT.java b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3PipeliningDisabledIT.java index 09325e2ed9b..ab30e5bc902 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3PipeliningDisabledIT.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/http/netty3/Netty3PipeliningDisabledIT.java @@ -21,7 +21,6 @@ package org.elasticsearch.http.netty3; import org.elasticsearch.ESNetty3IntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -57,10 +56,10 @@ public class Netty3PipeliningDisabledIT extends ESNetty3IntegTestCase { HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); try (Netty3HttpClient nettyHttpClient = new Netty3HttpClient()) { - Collection responses = nettyHttpClient.get(inetSocketTransportAddress.address(), requests); + Collection responses = nettyHttpClient.get(transportAddress.address(), requests); assertThat(responses, hasSize(requests.length)); List opaqueIds = new ArrayList<>(returnOpaqueIds(responses)); diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/Netty3SizeHeaderFrameDecoderTests.java b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/Netty3SizeHeaderFrameDecoderTests.java index ba72ade58e7..6b5163d7dfe 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/Netty3SizeHeaderFrameDecoderTests.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/Netty3SizeHeaderFrameDecoderTests.java @@ -22,7 +22,6 @@ package org.elasticsearch.transport; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; @@ -72,7 +71,7 @@ public class Netty3SizeHeaderFrameDecoderTests extends ESTestCase { nettyTransport.transportServiceAdapter(transportService.createAdapter()); TransportAddress[] boundAddresses = nettyTransport.boundAddress().boundAddresses(); - InetSocketTransportAddress transportAddress = (InetSocketTransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); port = transportAddress.address().getPort(); host = transportAddress.address().getAddress(); diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportMultiPortIntegrationIT.java b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportMultiPortIntegrationIT.java index d25951e254c..b5f0fc9f85b 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportMultiPortIntegrationIT.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportMultiPortIntegrationIT.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -77,7 +76,7 @@ public class Netty3TransportMultiPortIntegrationIT extends ESNetty3IntegTestCase .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); try (TransportClient transportClient = new MockTransportClient(settings, Netty3Plugin.class)) { - transportClient.addTransportAddress(new InetSocketTransportAddress(InetAddress.getByName("127.0.0.1"), randomPort)); + transportClient.addTransportAddress(new TransportAddress(InetAddress.getByName("127.0.0.1"), randomPort)); ClusterHealthResponse response = transportClient.admin().cluster().prepareHealth().get(); assertThat(response.getStatus(), is(ClusterHealthStatus.GREEN)); } @@ -91,19 +90,19 @@ public class Netty3TransportMultiPortIntegrationIT extends ESNetty3IntegTestCase assertThat(nodeInfo.getTransport().getProfileAddresses(), hasKey("client1")); BoundTransportAddress boundTransportAddress = nodeInfo.getTransport().getProfileAddresses().get("client1"); for (TransportAddress transportAddress : boundTransportAddress.boundAddresses()) { - assertThat(transportAddress, instanceOf(InetSocketTransportAddress.class)); + assertThat(transportAddress, instanceOf(TransportAddress.class)); } // bound addresses for (TransportAddress transportAddress : boundTransportAddress.boundAddresses()) { - assertThat(transportAddress, instanceOf(InetSocketTransportAddress.class)); - assertThat(((InetSocketTransportAddress) transportAddress).address().getPort(), + assertThat(transportAddress, instanceOf(TransportAddress.class)); + assertThat(transportAddress.address().getPort(), is(allOf(greaterThanOrEqualTo(randomPort), lessThanOrEqualTo(randomPort + 10)))); } // publish address - assertThat(boundTransportAddress.publishAddress(), instanceOf(InetSocketTransportAddress.class)); - InetSocketTransportAddress publishAddress = (InetSocketTransportAddress) boundTransportAddress.publishAddress(); + assertThat(boundTransportAddress.publishAddress(), instanceOf(TransportAddress.class)); + TransportAddress publishAddress = boundTransportAddress.publishAddress(); assertThat(NetworkAddress.format(publishAddress.address().getAddress()), is("127.0.0.7")); assertThat(publishAddress.address().getPort(), is(4321)); } diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportPublishAddressIT.java b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportPublishAddressIT.java index a936ad7d191..e12ac27b2ba 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportPublishAddressIT.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/Netty3TransportPublishAddressIT.java @@ -25,7 +25,6 @@ import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESIntegTestCase; @@ -67,8 +66,8 @@ public class Netty3TransportPublishAddressIT extends ESNetty3IntegTestCase { } else { assertThat(boundTransportAddress.boundAddresses().length, greaterThan(1)); for (TransportAddress boundAddress : boundTransportAddress.boundAddresses()) { - assertThat(boundAddress, instanceOf(InetSocketTransportAddress.class)); - InetSocketTransportAddress inetBoundAddress = (InetSocketTransportAddress) boundAddress; + assertThat(boundAddress, instanceOf(TransportAddress.class)); + TransportAddress inetBoundAddress = (TransportAddress) boundAddress; if (inetBoundAddress.address().getAddress() instanceof Inet4Address) { // IPv4 address is preferred publish address for _local_ assertThat(inetBoundAddress.getPort(), equalTo(boundTransportAddress.publishAddress().getPort())); diff --git a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/SimpleNetty3TransportTests.java b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/SimpleNetty3TransportTests.java index b90b788f904..d5ebef85e1b 100644 --- a/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/SimpleNetty3TransportTests.java +++ b/modules/transport-netty3/src/test/java/org/elasticsearch/transport/netty3/SimpleNetty3TransportTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.transport.MockTransportService; @@ -69,7 +69,7 @@ public class SimpleNetty3TransportTests extends AbstractSimpleTransportTestCase public void testConnectException() throws UnknownHostException { try { - serviceA.connectToNode(new DiscoveryNode("C", new InetSocketTransportAddress(InetAddress.getByName("localhost"), 9876), + serviceA.connectToNode(new DiscoveryNode("C", new TransportAddress(InetAddress.getByName("localhost"), 9876), emptyMap(), emptySet(),Version.CURRENT)); fail("Expected ConnectTransportException"); } catch (ConnectTransportException e) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java index cd0a208d2e6..98d4eeca17b 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java @@ -53,10 +53,9 @@ 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.TransportAddress; import org.elasticsearch.common.transport.NetworkExceptionHelper; import org.elasticsearch.common.transport.PortsRange; -import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; @@ -331,7 +330,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem throw new BindHttpException("Failed to resolve host [" + Arrays.toString(bindHosts) + "]", e); } - List boundAddresses = new ArrayList<>(hostAddresses.length); + List boundAddresses = new ArrayList<>(hostAddresses.length); for (InetAddress address : hostAddresses) { boundAddresses.add(bindAddress(address)); } @@ -345,15 +344,15 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem final int publishPort = resolvePublishPort(settings, boundAddresses, publishInetAddress); final InetSocketAddress publishAddress = new InetSocketAddress(publishInetAddress, publishPort); - return new BoundTransportAddress(boundAddresses.toArray(new TransportAddress[0]), new InetSocketTransportAddress(publishAddress)); + return new BoundTransportAddress(boundAddresses.toArray(new TransportAddress[0]), new TransportAddress(publishAddress)); } // package private for tests - static int resolvePublishPort(Settings settings, List boundAddresses, InetAddress publishInetAddress) { + static int resolvePublishPort(Settings settings, List boundAddresses, InetAddress publishInetAddress) { int publishPort = SETTING_HTTP_PUBLISH_PORT.get(settings); if (publishPort < 0) { - for (InetSocketTransportAddress boundAddress : boundAddresses) { + for (TransportAddress boundAddress : boundAddresses) { InetAddress boundInetAddress = boundAddress.address().getAddress(); if (boundInetAddress.isAnyLocalAddress() || boundInetAddress.equals(publishInetAddress)) { publishPort = boundAddress.getPort(); @@ -365,7 +364,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem // 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 (InetSocketTransportAddress boundAddress : boundAddresses) { + for (TransportAddress boundAddress : boundAddresses) { ports.add(boundAddress.getPort()); } if (ports.size() == 1) { @@ -415,7 +414,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem .build(); } - private InetSocketTransportAddress bindAddress(final InetAddress hostAddress) { + private TransportAddress bindAddress(final InetAddress hostAddress) { final AtomicReference lastException = new AtomicReference<>(); final AtomicReference boundSocket = new AtomicReference<>(); boolean success = port.iterate(new PortsRange.PortCallback() { @@ -441,7 +440,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem if (logger.isDebugEnabled()) { logger.debug("Bound http to address {{}}", NetworkAddress.format(boundSocket.get())); } - return new InetSocketTransportAddress(boundSocket.get()); + return new TransportAddress(boundSocket.get()); } @Override diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java index 808592c58f0..ee43a790d7f 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Transport.java @@ -54,7 +54,7 @@ import org.elasticsearch.common.network.NetworkService.TcpSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -333,7 +333,7 @@ public class Netty4Transport extends TcpTransport { } protected NodeChannels connectToChannelsLight(DiscoveryNode node) { - InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address(); + InetSocketAddress address = node.getAddress().address(); ChannelFuture connect = bootstrap.connect(address); connect.awaitUninterruptibly((long) (connectTimeout.millis() * 1.5)); if (!connect.isSuccess()) { @@ -364,7 +364,7 @@ public class Netty4Transport extends TcpTransport { connectionsPerNodeState + connectionsPerNodeRecovery; final ArrayList connections = new ArrayList<>(numConnections); - final InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address(); + final InetSocketAddress address = node.getAddress().address(); for (int i = 0; i < numConnections; i++) { connections.add(bootstrap.connect(address)); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/ESNetty4IntegTestCase.java b/modules/transport-netty4/src/test/java/org/elasticsearch/ESNetty4IntegTestCase.java index 54fc6b66539..b38cda76c69 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/ESNetty4IntegTestCase.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/ESNetty4IntegTestCase.java @@ -28,7 +28,6 @@ import org.elasticsearch.transport.netty4.Netty4Transport; import java.util.Arrays; import java.util.Collection; -@ESIntegTestCase.SuppressLocalMode public abstract class ESNetty4IntegTestCase extends ESIntegTestCase { @Override diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpPublishPortTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpPublishPortTests.java index 03e09f28d25..afa513275ea 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpPublishPortTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpPublishPortTests.java @@ -21,7 +21,7 @@ package org.elasticsearch.http.netty4; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.BindHttpException; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.test.ESTestCase; @@ -73,16 +73,16 @@ public class Netty4HttpPublishPortTests extends ESTestCase { } } - private InetSocketTransportAddress address(String host, int port) throws UnknownHostException { - return new InetSocketTransportAddress(getByName(host), port); + private TransportAddress address(String host, int port) throws UnknownHostException { + return new TransportAddress(getByName(host), port); } - private InetSocketTransportAddress randomAddress() throws UnknownHostException { + private TransportAddress randomAddress() throws UnknownHostException { return address("127.0.0." + randomIntBetween(1, 100), randomIntBetween(9200, 9300)); } - private List randomAddresses() throws UnknownHostException { - List addresses = new ArrayList<>(); + private List randomAddresses() throws UnknownHostException { + List addresses = new ArrayList<>(); for (int i = 0; i < randomIntBetween(1, 5); i++) { addresses.add(randomAddress()); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java index f1e2e922cbd..0b8b347e30f 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpRequestSizeLimitIT.java @@ -24,15 +24,13 @@ import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; -import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; -import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; import java.util.Collection; @@ -82,15 +80,15 @@ public class Netty4HttpRequestSizeLimitIT extends ESNetty4IntegTestCase { } HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress ().boundAddresses()); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { - Collection singleResponse = nettyHttpClient.post(inetSocketTransportAddress.address(), requests[0]); + Collection singleResponse = nettyHttpClient.post(transportAddress.address(), requests[0]); assertThat(singleResponse, hasSize(1)); assertAtLeastOnceExpectedStatus(singleResponse, HttpResponseStatus.OK); - Collection multipleResponses = nettyHttpClient.post(inetSocketTransportAddress.address(), requests); + Collection multipleResponses = nettyHttpClient.post(transportAddress.address(), requests); assertThat(multipleResponses, hasSize(requests.length)); assertAtLeastOnceExpectedStatus(multipleResponses, HttpResponseStatus.SERVICE_UNAVAILABLE); } @@ -107,11 +105,11 @@ public class Netty4HttpRequestSizeLimitIT extends ESNetty4IntegTestCase { } HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress + TransportAddress transportAddress = (TransportAddress) randomFrom(httpServerTransport.boundAddress ().boundAddresses()); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { - Collection responses = nettyHttpClient.put(inetSocketTransportAddress.address(), requestUris); + Collection responses = nettyHttpClient.put(transportAddress.address(), requestUris); assertThat(responses, hasSize(requestUris.length)); assertAllInExpectedStatus(responses, HttpResponseStatus.OK); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerPipeliningTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerPipeliningTests.java index 63e35a786c2..50c65a6195e 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerPipeliningTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerPipeliningTests.java @@ -33,7 +33,7 @@ import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.http.HttpServerTransport; @@ -89,8 +89,8 @@ public class Netty4HttpServerPipeliningTests extends ESTestCase { .build(); try (final HttpServerTransport httpServerTransport = new CustomNettyHttpServerTransport(settings)) { httpServerTransport.start(); - final InetSocketTransportAddress transportAddress = - (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress().boundAddresses()); + final TransportAddress transportAddress = + (TransportAddress) randomFrom(httpServerTransport.boundAddress().boundAddresses()); final int numberOfRequests = randomIntBetween(4, 16); final List requests = new ArrayList<>(numberOfRequests); @@ -117,8 +117,8 @@ public class Netty4HttpServerPipeliningTests extends ESTestCase { .build(); try (final HttpServerTransport httpServerTransport = new CustomNettyHttpServerTransport(settings)) { httpServerTransport.start(); - final InetSocketTransportAddress transportAddress = - (InetSocketTransportAddress) randomFrom(httpServerTransport.boundAddress().boundAddresses()); + final TransportAddress transportAddress = + (TransportAddress) randomFrom(httpServerTransport.boundAddress().boundAddresses()); final int numberOfRequests = randomIntBetween(4, 16); final Set slowIds = new HashSet<>(); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java index faea5c8ec18..ceed9d9a503 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java @@ -32,7 +32,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; @@ -123,7 +123,7 @@ public class Netty4HttpServerTransportTests extends ESTestCase { transport.httpServerAdapter((request, channel, context) -> channel.sendResponse(new BytesRestResponse(OK, BytesRestResponse.TEXT_CONTENT_TYPE, new BytesArray("done")))); transport.start(); - InetSocketTransportAddress remoteAddress = (InetSocketTransportAddress) randomFrom(transport.boundAddress().boundAddresses()); + TransportAddress remoteAddress = (TransportAddress) randomFrom(transport.boundAddress().boundAddresses()); try (Netty4HttpClient client = new Netty4HttpClient()) { FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "/"); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningDisabledIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningDisabledIT.java index 4ea46c651fc..9f117d4ee21 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningDisabledIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningDisabledIT.java @@ -22,7 +22,6 @@ import io.netty.handler.codec.http.FullHttpResponse; import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -54,10 +53,10 @@ public class Netty4PipeliningDisabledIT extends ESNetty4IntegTestCase { HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { - Collection responses = nettyHttpClient.get(inetSocketTransportAddress.address(), requests); + Collection responses = nettyHttpClient.get(transportAddress.address(), requests); assertThat(responses, hasSize(requests.length)); List opaqueIds = new ArrayList<>(Netty4HttpClient.returnOpaqueIds(responses)); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningEnabledIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningEnabledIT.java index c2222562d72..cc3f22be453 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningEnabledIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4PipeliningEnabledIT.java @@ -23,10 +23,8 @@ import io.netty.handler.codec.http.FullHttpResponse; import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; @@ -53,10 +51,10 @@ public class Netty4PipeliningEnabledIT extends ESNetty4IntegTestCase { HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class); TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses(); - InetSocketTransportAddress inetSocketTransportAddress = (InetSocketTransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); try (Netty4HttpClient nettyHttpClient = new Netty4HttpClient()) { - Collection responses = nettyHttpClient.get(inetSocketTransportAddress.address(), requests); + Collection responses = nettyHttpClient.get(transportAddress.address(), requests); assertThat(responses, hasSize(5)); Collection opaqueIds = Netty4HttpClient.returnOpaqueIds(responses); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java index 165b74431e1..0e9ebe5f7f4 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4SizeHeaderFrameDecoderTests.java @@ -22,7 +22,6 @@ package org.elasticsearch.transport.netty4; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; @@ -69,7 +68,7 @@ public class Netty4SizeHeaderFrameDecoderTests extends ESTestCase { nettyTransport.start(); TransportAddress[] boundAddresses = nettyTransport.boundAddress().boundAddresses(); - InetSocketTransportAddress transportAddress = (InetSocketTransportAddress) randomFrom(boundAddresses); + TransportAddress transportAddress = (TransportAddress) randomFrom(boundAddresses); port = transportAddress.address().getPort(); host = transportAddress.address().getAddress(); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java index 32268d1f5d8..bfd03ff75c6 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -77,7 +76,7 @@ public class Netty4TransportMultiPortIntegrationIT extends ESNetty4IntegTestCase .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .build(); try (TransportClient transportClient = new MockTransportClient(settings, Netty4Plugin.class)) { - transportClient.addTransportAddress(new InetSocketTransportAddress(InetAddress.getByName("127.0.0.1"), randomPort)); + transportClient.addTransportAddress(new TransportAddress(InetAddress.getByName("127.0.0.1"), randomPort)); ClusterHealthResponse response = transportClient.admin().cluster().prepareHealth().get(); assertThat(response.getStatus(), is(ClusterHealthStatus.GREEN)); } @@ -91,19 +90,19 @@ public class Netty4TransportMultiPortIntegrationIT extends ESNetty4IntegTestCase assertThat(nodeInfo.getTransport().getProfileAddresses(), hasKey("client1")); BoundTransportAddress boundTransportAddress = nodeInfo.getTransport().getProfileAddresses().get("client1"); for (TransportAddress transportAddress : boundTransportAddress.boundAddresses()) { - assertThat(transportAddress, instanceOf(InetSocketTransportAddress.class)); + assertThat(transportAddress, instanceOf(TransportAddress.class)); } // bound addresses for (TransportAddress transportAddress : boundTransportAddress.boundAddresses()) { - assertThat(transportAddress, instanceOf(InetSocketTransportAddress.class)); - assertThat(((InetSocketTransportAddress) transportAddress).address().getPort(), + assertThat(transportAddress, instanceOf(TransportAddress.class)); + assertThat(transportAddress.address().getPort(), is(allOf(greaterThanOrEqualTo(randomPort), lessThanOrEqualTo(randomPort + 10)))); } // publish address - assertThat(boundTransportAddress.publishAddress(), instanceOf(InetSocketTransportAddress.class)); - InetSocketTransportAddress publishAddress = (InetSocketTransportAddress) boundTransportAddress.publishAddress(); + assertThat(boundTransportAddress.publishAddress(), instanceOf(TransportAddress.class)); + TransportAddress publishAddress = boundTransportAddress.publishAddress(); assertThat(NetworkAddress.format(publishAddress.address().getAddress()), is("127.0.0.7")); assertThat(publishAddress.address().getPort(), is(4321)); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportPublishAddressIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportPublishAddressIT.java index 6a6f7bee80e..922031d3c3d 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportPublishAddressIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportPublishAddressIT.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.network.NetworkUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.transport.Netty4Plugin; @@ -76,8 +75,8 @@ public class Netty4TransportPublishAddressIT extends ESNetty4IntegTestCase { } else { assertThat(boundTransportAddress.boundAddresses().length, greaterThan(1)); for (TransportAddress boundAddress : boundTransportAddress.boundAddresses()) { - assertThat(boundAddress, instanceOf(InetSocketTransportAddress.class)); - InetSocketTransportAddress inetBoundAddress = (InetSocketTransportAddress) boundAddress; + assertThat(boundAddress, instanceOf(TransportAddress.class)); + TransportAddress inetBoundAddress = (TransportAddress) boundAddress; if (inetBoundAddress.address().getAddress() instanceof Inet4Address) { // IPv4 address is preferred publish address for _local_ assertThat(inetBoundAddress.getPort(), equalTo(boundTransportAddress.publishAddress().getPort())); diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java index 3a3a4587cac..8a97462356c 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/SimpleNetty4TransportTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.transport.MockTransportService; @@ -69,7 +69,7 @@ public class SimpleNetty4TransportTests extends AbstractSimpleTransportTestCase public void testConnectException() throws UnknownHostException { try { - serviceA.connectToNode(new DiscoveryNode("C", new InetSocketTransportAddress(InetAddress.getByName("localhost"), 9876), + serviceA.connectToNode(new DiscoveryNode("C", new TransportAddress(InetAddress.getByName("localhost"), 9876), emptyMap(), emptySet(),Version.CURRENT)); fail("Expected ConnectTransportException"); } catch (ConnectTransportException e) { diff --git a/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java index b3f1afb35a1..9e17ca21868 100644 --- a/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java +++ b/plugins/discovery-azure-classic/src/test/java/org/elasticsearch/discovery/azure/classic/AzureDiscoveryClusterFormationTests.java @@ -66,7 +66,6 @@ import java.util.concurrent.ExecutionException; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; -@ESIntegTestCase.SuppressLocalMode @ESIntegTestCase.ClusterScope(numDataNodes = 2, numClientNodes = 0) @SuppressForbidden(reason = "use http server") // TODO this should be a IT but currently all ITs in this project run against a real cluster diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryClusterFormationTests.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryClusterFormationTests.java index bd69953eb1e..693e765ac2d 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryClusterFormationTests.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryClusterFormationTests.java @@ -57,7 +57,6 @@ import java.util.concurrent.ExecutionException; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.hamcrest.Matchers.equalTo; -@ESIntegTestCase.SuppressLocalMode @ESIntegTestCase.ClusterScope(supportsDedicatedMasters = false, numDataNodes = 2, numClientNodes = 0) @SuppressForbidden(reason = "use http server") // TODO this should be a IT but currently all ITs in this project run against a real cluster diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java index edb062d9e30..67da32df8ff 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java @@ -23,22 +23,30 @@ import com.amazonaws.services.ec2.model.Tag; import org.elasticsearch.Version; import org.elasticsearch.cloud.aws.AwsEc2Service; import org.elasticsearch.cloud.aws.AwsEc2Service.DISCOVERY_EC2; -import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.collect.CopyOnWriteHashMap; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.MockTcpTransport; +import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasSize; @@ -51,6 +59,7 @@ public class Ec2DiscoveryTests extends ESTestCase { protected static ThreadPool threadPool; protected MockTransportService transportService; + private Map poorMansDNS = new ConcurrentHashMap<>(); @BeforeClass public static void createThreadPool() { @@ -67,7 +76,17 @@ public class Ec2DiscoveryTests extends ESTestCase { @Before public void createTransportService() { - transportService = MockTransportService.local(Settings.EMPTY, Version.CURRENT, threadPool); + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); + final Transport transport = new MockTcpTransport(Settings.EMPTY, threadPool, BigArrays.NON_RECYCLING_INSTANCE, + new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(Settings.EMPTY, Collections.emptyList()), + Version.CURRENT) { + @Override + public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws Exception { + // we just need to ensure we don't resolve DNS here + return new TransportAddress[] {poorMansDNS.getOrDefault(address, buildNewFakeTransportAddress())}; + } + }; + transportService = new MockTransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR); } protected List buildDynamicNodes(Settings nodeSettings, int nodes) { @@ -92,6 +111,9 @@ public class Ec2DiscoveryTests extends ESTestCase { public void testPrivateIp() throws InterruptedException { int nodes = randomInt(10); + for (int i = 0; i < nodes; i++) { + poorMansDNS.put(AmazonEC2Mock.PREFIX_PRIVATE_IP + (i+1), buildNewFakeTransportAddress()); + } Settings nodeSettings = Settings.builder() .put(DISCOVERY_EC2.HOST_TYPE_SETTING.getKey(), "private_ip") .build(); @@ -101,13 +123,16 @@ public class Ec2DiscoveryTests extends ESTestCase { int node = 1; for (DiscoveryNode discoveryNode : discoveryNodes) { TransportAddress address = discoveryNode.getAddress(); - TransportAddress expected = new LocalTransportAddress(AmazonEC2Mock.PREFIX_PRIVATE_IP + node++); - assertThat(address.sameHost(expected), is(true)); + TransportAddress expected = poorMansDNS.get(AmazonEC2Mock.PREFIX_PRIVATE_IP + node++); + assertEquals(address, expected); } } public void testPublicIp() throws InterruptedException { int nodes = randomInt(10); + for (int i = 0; i < nodes; i++) { + poorMansDNS.put(AmazonEC2Mock.PREFIX_PUBLIC_IP + (i+1), buildNewFakeTransportAddress()); + } Settings nodeSettings = Settings.builder() .put(DISCOVERY_EC2.HOST_TYPE_SETTING.getKey(), "public_ip") .build(); @@ -117,13 +142,18 @@ public class Ec2DiscoveryTests extends ESTestCase { int node = 1; for (DiscoveryNode discoveryNode : discoveryNodes) { TransportAddress address = discoveryNode.getAddress(); - TransportAddress expected = new LocalTransportAddress(AmazonEC2Mock.PREFIX_PUBLIC_IP + node++); - assertThat(address.sameHost(expected), is(true)); + TransportAddress expected = poorMansDNS.get(AmazonEC2Mock.PREFIX_PUBLIC_IP + node++); + assertEquals(address, expected); } } public void testPrivateDns() throws InterruptedException { int nodes = randomInt(10); + for (int i = 0; i < nodes; i++) { + String instanceId = "node" + (i+1); + poorMansDNS.put(AmazonEC2Mock.PREFIX_PRIVATE_DNS + instanceId + + AmazonEC2Mock.SUFFIX_PRIVATE_DNS, buildNewFakeTransportAddress()); + } Settings nodeSettings = Settings.builder() .put(DISCOVERY_EC2.HOST_TYPE_SETTING.getKey(), "private_dns") .build(); @@ -134,14 +164,19 @@ public class Ec2DiscoveryTests extends ESTestCase { for (DiscoveryNode discoveryNode : discoveryNodes) { String instanceId = "node" + node++; TransportAddress address = discoveryNode.getAddress(); - TransportAddress expected = new LocalTransportAddress( + TransportAddress expected = poorMansDNS.get( AmazonEC2Mock.PREFIX_PRIVATE_DNS + instanceId + AmazonEC2Mock.SUFFIX_PRIVATE_DNS); - assertThat(address.sameHost(expected), is(true)); + assertEquals(address, expected); } } public void testPublicDns() throws InterruptedException { int nodes = randomInt(10); + for (int i = 0; i < nodes; i++) { + String instanceId = "node" + (i+1); + poorMansDNS.put(AmazonEC2Mock.PREFIX_PUBLIC_DNS + instanceId + + AmazonEC2Mock.SUFFIX_PUBLIC_DNS, buildNewFakeTransportAddress()); + } Settings nodeSettings = Settings.builder() .put(DISCOVERY_EC2.HOST_TYPE_SETTING.getKey(), "public_dns") .build(); @@ -152,9 +187,9 @@ public class Ec2DiscoveryTests extends ESTestCase { for (DiscoveryNode discoveryNode : discoveryNodes) { String instanceId = "node" + node++; TransportAddress address = discoveryNode.getAddress(); - TransportAddress expected = new LocalTransportAddress( + TransportAddress expected = poorMansDNS.get( AmazonEC2Mock.PREFIX_PUBLIC_DNS + instanceId + AmazonEC2Mock.SUFFIX_PUBLIC_DNS); - assertThat(address.sameHost(expected), is(true)); + assertEquals(address, expected); } } diff --git a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoverTests.java b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoverTests.java index 98f6fd0dc1b..1512da2429f 100644 --- a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoverTests.java +++ b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoverTests.java @@ -59,7 +59,6 @@ import java.util.concurrent.ExecutionException; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; -@ESIntegTestCase.SuppressLocalMode @ESIntegTestCase.ClusterScope(supportsDedicatedMasters = false, numDataNodes = 2, numClientNodes = 0) @SuppressForbidden(reason = "use http server") // TODO this should be a IT but currently all ITs in this project run against a real cluster diff --git a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java index 08e7ee963d3..cd7a5dbb274 100644 --- a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java +++ b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java @@ -94,7 +94,7 @@ public class GceDiscoveryTests extends ESTestCase { @Before public void createTransportService() { - transportService = MockTransportService.local(Settings.EMPTY, Version.CURRENT, threadPool); + transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool); } @After diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index 0171dfb99d1..7ca0aa57d7a 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -29,11 +29,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeValidationException; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.transport.MockTcpTransportPlugin; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -60,7 +62,7 @@ public class TribeUnitTests extends ESTestCase { public static void createTribes() throws NodeValidationException { Settings baseSettings = Settings.builder() .put(NetworkModule.HTTP_ENABLED.getKey(), false) - .put("transport.type", "local") + .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local") .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2) @@ -72,14 +74,14 @@ public class TribeUnitTests extends ESTestCase { .put("cluster.name", "tribe1") .put("node.name", "tribe1_node") .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) - .build(), Collections.emptyList()).start(); + .build(), Collections.singleton(MockTcpTransportPlugin.class)).start(); tribe2 = new TribeClientNode( Settings.builder() .put(baseSettings) .put("cluster.name", "tribe2") .put("node.name", "tribe2_node") .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong()) - .build(), Collections.emptyList()).start(); + .build(), Collections.singleton(MockTcpTransportPlugin.class)).start(); } @AfterClass @@ -101,13 +103,14 @@ public class TribeUnitTests extends ESTestCase { private static void assertTribeNodeSuccessfullyCreated(Settings extraSettings) throws Exception { //The tribe clients do need it to make sure they can find their corresponding tribes using the proper transport Settings settings = Settings.builder().put(NetworkModule.HTTP_ENABLED.getKey(), false).put("node.name", "tribe_node") - .put("transport.type", "local").put("discovery.type", "local") - .put("tribe.t1.transport.type", "local").put("tribe.t2.transport.type", "local") + .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).put("discovery.type", "local") + .put("tribe.t1.transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) + .put("tribe.t2.transport.type",MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put("tribe.t1.discovery.type", "local").put("tribe.t2.discovery.type", "local") .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) .put(extraSettings).build(); - try (Node node = new Node(settings).start()) { + try (Node node = new MockNode(settings, Collections.singleton(MockTcpTransportPlugin.class)).start()) { try (Client client = node.client()) { assertBusy(() -> { ClusterState state = client.admin().cluster().prepareState().clear().setNodes(true).get().getState(); diff --git a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java b/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java index d36c2aa04d2..bc1c95424d8 100644 --- a/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java +++ b/qa/smoke-test-client/src/test/java/org/elasticsearch/smoketest/ESSmokeClientTestCase.java @@ -27,13 +27,11 @@ import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.transport.MockTcpTransportPlugin; import org.elasticsearch.transport.Netty3Plugin; -import org.elasticsearch.transport.Netty4Plugin; import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.After; import org.junit.AfterClass; @@ -125,7 +123,7 @@ public abstract class ESSmokeClientTestCase extends LuceneTestCase { for (String stringAddress : stringAddresses) { URL url = new URL("http://" + stringAddress); InetAddress inetAddress = InetAddress.getByName(url.getHost()); - transportAddresses[i++] = new InetSocketTransportAddress(new InetSocketAddress(inetAddress, url.getPort())); + transportAddresses[i++] = new TransportAddress(new InetSocketAddress(inetAddress, url.getPort())); } return startClient(createTempDir(), transportAddresses); } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index 89fafc74c86..af774c8f29f 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -36,7 +36,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.gateway.TestGatewayAllocator; @@ -105,19 +104,19 @@ public abstract class ESAllocationTestCase extends ESTestCase { } protected static DiscoveryNode newNode(String nodeName, String nodeId, Map attributes) { - return new DiscoveryNode(nodeName, nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeName, nodeId, buildNewFakeTransportAddress(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Map attributes) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), attributes, MASTER_DATA_ROLES, Version.CURRENT); + return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), attributes, MASTER_DATA_ROLES, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Set roles) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), roles, Version.CURRENT); + return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), roles, Version.CURRENT); } protected static DiscoveryNode newNode(String nodeId, Version version) { - return new DiscoveryNode(nodeId, LocalTransportAddress.buildUnique(), emptyMap(), MASTER_DATA_ROLES, version); + return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), MASTER_DATA_ROLES, version); } protected static ClusterState startRandomInitializingShard(ClusterState clusterState, AllocationService strategy) { diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java index 45edbd8bcb2..576b290ed40 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java @@ -32,10 +32,10 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; @@ -67,7 +67,7 @@ public class MockInternalClusterInfoService extends InternalClusterInfoService { usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes()); paths[0] = path; FsInfo fsInfo = new FsInfo(System.currentTimeMillis(), null, paths); - return new NodeStats(new DiscoveryNode(nodeName, LocalTransportAddress.buildUnique(), emptyMap(), emptySet(), Version.CURRENT), + return new NodeStats(new DiscoveryNode(nodeName, ESTestCase.buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), System.currentTimeMillis(), null, null, null, null, null, fsInfo, diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 23aed676af8..fbb87d9f8d1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -41,7 +41,6 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.env.NodeEnvironment; @@ -353,7 +352,7 @@ public abstract class IndexShardTestCase extends ESTestCase { } private DiscoveryNode getFakeDiscoNode(String id) { - return new DiscoveryNode(id, new LocalTransportAddress("_fake_" + id), Version.CURRENT); + return new DiscoveryNode(id, buildNewFakeTransportAddress(), Version.CURRENT); } /** recovers a replica from the given primary **/ diff --git a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java index 38682239b78..f247c56636c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java @@ -29,7 +29,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; @@ -42,7 +41,7 @@ import static junit.framework.TestCase.fail; public class ClusterServiceUtils { public static ClusterService createClusterService(ThreadPool threadPool) { - DiscoveryNode discoveryNode = new DiscoveryNode("node", LocalTransportAddress.buildUnique(), Collections.emptyMap(), + DiscoveryNode discoveryNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())),Version.CURRENT); return createClusterService(threadPool, discoveryNode); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 645801b316d..26059ba60ca 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -34,6 +34,7 @@ import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.transport.AssertingTransportInterceptor; import org.elasticsearch.transport.MockTcpTransportPlugin; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ShardOperationFailedException; @@ -86,7 +87,6 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; @@ -124,7 +124,6 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.client.RandomizingClient; import org.elasticsearch.test.disruption.ServiceDisruptionScheme; import org.elasticsearch.test.store.MockFSIndexStore; -import org.elasticsearch.test.transport.AssertingLocalTransport; import org.elasticsearch.test.transport.MockTransportService; import org.hamcrest.Matchers; import org.junit.After; @@ -1679,7 +1678,7 @@ public abstract class ESIntegTestCase extends ESTestCase { for (String stringAddress : stringAddresses) { URL url = new URL("http://" + stringAddress); InetAddress inetAddress = InetAddress.getByName(url.getHost()); - transportAddresses[i++] = new InetSocketTransportAddress(new InetSocketAddress(inetAddress, url.getPort())); + transportAddresses[i++] = new TransportAddress(new InetSocketAddress(inetAddress, url.getPort())); } return new ExternalTestCluster(createTempDir(), externalClusterClientSettings(), transportClientPlugins(), transportAddresses); } @@ -1730,9 +1729,6 @@ public abstract class ESIntegTestCase extends ESTestCase { ArrayList> mocks = new ArrayList<>(mockPlugins); // add both mock plugins - local and tcp if they are not there // we do this in case somebody overrides getMockPlugins and misses to call super - if (mockPlugins.contains(AssertingLocalTransport.TestPlugin.class) == false) { - mocks.add(AssertingLocalTransport.TestPlugin.class); - } if (mockPlugins.contains(MockTcpTransportPlugin.class) == false) { mocks.add(MockTcpTransportPlugin.class); } @@ -1744,24 +1740,9 @@ public abstract class ESIntegTestCase extends ESTestCase { } protected NodeConfigurationSource getNodeConfigSource() { - SuppressLocalMode noLocal = getAnnotation(this.getClass(), SuppressLocalMode.class); - SuppressNetworkMode noNetwork = getAnnotation(this.getClass(), SuppressNetworkMode.class); Settings.Builder networkSettings = Settings.builder(); - final boolean isNetwork; - if (noLocal != null && noNetwork != null) { - throw new IllegalStateException("Can't suppress both network and local mode"); - } else if (noLocal != null) { - if (addMockTransportService()) { - networkSettings.put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME); - } - isNetwork = true; - } else { - if (addMockTransportService()) { - networkSettings.put(NetworkModule.TRANSPORT_TYPE_KEY, AssertingLocalTransport.ASSERTING_TRANSPORT_NAME); - } else { - networkSettings.put(NetworkModule.TRANSPORT_TYPE_KEY, "local"); - } - isNetwork = false; + if (addMockTransportService()) { + networkSettings.put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME); } NodeConfigurationSource nodeConfigurationSource = new NodeConfigurationSource() { @@ -1769,8 +1750,7 @@ public abstract class ESIntegTestCase extends ESTestCase { public Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(NetworkModule.HTTP_ENABLED.getKey(), false) - .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), - isNetwork ? DiscoveryModule.DISCOVERY_TYPE_SETTING.getDefault(Settings.EMPTY) : "local") + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local") .put(networkSettings.build()). put(ESIntegTestCase.this.nodeSettings(nodeOrdinal)).build(); } @@ -1789,12 +1769,9 @@ public abstract class ESIntegTestCase extends ESTestCase { @Override public Collection> transportClientPlugins() { Collection> plugins = ESIntegTestCase.this.transportClientPlugins(); - if (isNetwork && plugins.contains(MockTcpTransportPlugin.class) == false) { + if (plugins.contains(MockTcpTransportPlugin.class) == false) { plugins = new ArrayList<>(plugins); plugins.add(MockTcpTransportPlugin.class); - } else if (isNetwork == false && plugins.contains(AssertingLocalTransport.class) == false) { - plugins = new ArrayList<>(plugins); - plugins.add(AssertingLocalTransport.TestPlugin.class); } return Collections.unmodifiableCollection(plugins); } @@ -1838,10 +1815,12 @@ public abstract class ESIntegTestCase extends ESTestCase { if (randomBoolean()) { mocks.add(MockSearchService.TestPlugin.class); } + if (randomBoolean()) { + mocks.add(AssertingTransportInterceptor.TestPlugin.class); + } } if (addMockTransportService()) { - mocks.add(AssertingLocalTransport.TestPlugin.class); mocks.add(MockTcpTransportPlugin.class); } mocks.add(TestSeedPlugin.class); @@ -2106,8 +2085,7 @@ public abstract class ESIntegTestCase extends ESTestCase { for (NodeInfo node : nodes) { if (node.getHttp() != null) { TransportAddress publishAddress = node.getHttp().address().publishAddress(); - assertEquals(1, publishAddress.uniqueAddressTypeId()); - InetSocketAddress address = ((InetSocketTransportAddress) publishAddress).address(); + InetSocketAddress address = publishAddress.address(); hosts.add(new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), protocol)); } } @@ -2143,29 +2121,10 @@ public abstract class ESIntegTestCase extends ESTestCase { public @interface SuiteScopeTestCase { } - /** - * If used the test will never run in local mode. - */ - @Retention(RetentionPolicy.RUNTIME) - @Inherited - @Target(ElementType.TYPE) - public @interface SuppressLocalMode { - } - - /** - * If used the test will never run in network mode - */ - @Retention(RetentionPolicy.RUNTIME) - @Inherited - @Target(ElementType.TYPE) - public @interface SuppressNetworkMode { - } - public static Index resolveIndex(String index) { GetIndexResponse getIndexResponse = client().admin().indices().prepareGetIndex().setIndices(index).get(); assertTrue("index " + index + " not found", getIndexResponse.getSettings().containsKey(index)); String uuid = getIndexResponse.getSettings().get(index).get(IndexMetaData.SETTING_INDEX_UUID); return new Index(index, uuid); } - } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index e440af4e9b9..2752e0f2eca 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -46,6 +46,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.MockTcpTransportPlugin; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -53,6 +54,7 @@ import org.junit.BeforeClass; import java.io.IOException; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -180,11 +182,16 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { .put(EsExecutors.PROCESSORS_SETTING.getKey(), 1) // limit the number of threads created .put(NetworkModule.HTTP_ENABLED.getKey(), false) .put("discovery.type", "local") - .put("transport.type", "local") + .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put(Node.NODE_DATA_SETTING.getKey(), true) .put(nodeSettings()) // allow test cases to provide their own settings or override these .build(); - Node build = new MockNode(settings, getPlugins()); + Collection> plugins = getPlugins(); + if (plugins.contains(MockTcpTransportPlugin.class) == false) { + plugins = new ArrayList<>(plugins); + plugins.add(MockTcpTransportPlugin.class); + } + Node build = new MockNode(settings, plugins); try { build.start(); } catch (NodeValidationException e) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 799fdbf8943..ade720dd0bc 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -45,6 +45,7 @@ import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.io.PathUtilsForTesting; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.MockPageCacheRecycler; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -100,6 +101,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Predicate; @@ -131,6 +133,14 @@ import static org.hamcrest.Matchers.equalTo; @LuceneTestCase.SuppressReproduceLine public abstract class ESTestCase extends LuceneTestCase { + private static final AtomicInteger portGenerator = new AtomicInteger(); + + @AfterClass + public static void resetPortCounter() { + portGenerator.set(0); + } + + static { System.setProperty("log4j.shutdownHookEnabled", "false"); // we can not shutdown logging when tests are running or the next test that runs within the @@ -173,6 +183,14 @@ public abstract class ESTestCase extends LuceneTestCase { } }); + /** + * Generates a new transport address using {@link TransportAddress#META_ADDRESS} with an incrementing port number. + * The port number starts at 0 and is reset after each test suite run. + */ + public static TransportAddress buildNewFakeTransportAddress() { + return new TransportAddress(TransportAddress.META_ADDRESS, portGenerator.incrementAndGet()); + } + /** * Called when a test fails, supplying the errors it generated. Not called when the test fails because assumptions are violated. */ diff --git a/test/framework/src/main/java/org/elasticsearch/test/ExternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/ExternalTestCluster.java index adab3b70455..8e24a99b895 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ExternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ExternalTestCluster.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.Plugin; @@ -98,7 +97,7 @@ public final class ExternalTestCluster extends TestCluster { int masterAndDataNodes = 0; for (int i = 0; i < nodeInfos.getNodes().size(); i++) { NodeInfo nodeInfo = nodeInfos.getNodes().get(i); - httpAddresses[i] = ((InetSocketTransportAddress) nodeInfo.getHttp().address().publishAddress()).address(); + httpAddresses[i] = nodeInfo.getHttp().address().publishAddress().address(); if (DiscoveryNode.isDataNode(nodeInfo.getSettings())) { dataNodes++; masterAndDataNodes++; 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 6a5493ff1eb..ef86c45b59f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -56,7 +56,6 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings.Builder; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; @@ -1191,7 +1190,7 @@ public final class InternalTestCluster extends TestCluster { public InetSocketAddress[] httpAddresses() { List addresses = new ArrayList<>(); for (HttpServerTransport httpServerTransport : getInstances(HttpServerTransport.class)) { - addresses.add(((InetSocketTransportAddress) httpServerTransport.boundAddress().publishAddress()).address()); + addresses.add(httpServerTransport.boundAddress().publishAddress().address()); } return addresses.toArray(new InetSocketAddress[addresses.size()]); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/AssertingLocalTransport.java b/test/framework/src/main/java/org/elasticsearch/test/transport/AssertingLocalTransport.java deleted file mode 100644 index b44e180b453..00000000000 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/AssertingLocalTransport.java +++ /dev/null @@ -1,110 +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.test.transport; - -import org.elasticsearch.Version; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.plugins.NetworkPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.VersionUtils; -import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.local.LocalTransport; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.function.Supplier; - -public class AssertingLocalTransport extends LocalTransport { - - public static final String ASSERTING_TRANSPORT_NAME = "asserting_local"; - - public static class TestPlugin extends Plugin implements NetworkPlugin { - - @Override - public List> getSettings() { - return Arrays.asList(ASSERTING_TRANSPORT_MIN_VERSION_KEY, ASSERTING_TRANSPORT_MAX_VERSION_KEY); - } - - @Override - public Map> getTransports(Settings settings, ThreadPool threadPool, BigArrays bigArrays, - CircuitBreakerService circuitBreakerService, - NamedWriteableRegistry namedWriteableRegistry, - NetworkService networkService) { - return Collections.singletonMap(ASSERTING_TRANSPORT_NAME, - () -> new AssertingLocalTransport(settings, circuitBreakerService, threadPool, namedWriteableRegistry)); - } - } - - public static final Setting ASSERTING_TRANSPORT_MIN_VERSION_KEY = - new Setting<>("transport.asserting.version.min", Integer.toString(Version.CURRENT.minimumCompatibilityVersion().id), - (s) -> Version.fromId(Integer.parseInt(s)), Property.NodeScope); - public static final Setting ASSERTING_TRANSPORT_MAX_VERSION_KEY = - new Setting<>("transport.asserting.version.max", Integer.toString(Version.CURRENT.id), - (s) -> Version.fromId(Integer.parseInt(s)), Property.NodeScope); - private final Random random; - private final Version minVersion; - private final Version maxVersion; - - @Inject - public AssertingLocalTransport(Settings settings, CircuitBreakerService circuitBreakerService, ThreadPool threadPool, - NamedWriteableRegistry namedWriteableRegistry) { - super(settings, threadPool, namedWriteableRegistry, circuitBreakerService); - final long seed = ESIntegTestCase.INDEX_TEST_SEED_SETTING.get(settings); - random = new Random(seed); - minVersion = ASSERTING_TRANSPORT_MIN_VERSION_KEY.get(settings); - maxVersion = ASSERTING_TRANSPORT_MAX_VERSION_KEY.get(settings); - } - - @Override - protected void handleParsedResponse(final TransportResponse response, final TransportResponseHandler handler) { - ElasticsearchAssertions.assertVersionSerializable(VersionUtils.randomVersionBetween(random, minVersion, maxVersion), response, - namedWriteableRegistry); - super.handleParsedResponse(response, handler); - } - - @Override - public void sendRequest(final DiscoveryNode node, final long requestId, final String action, final TransportRequest request, - TransportRequestOptions options) throws IOException, TransportException { - ElasticsearchAssertions.assertVersionSerializable(VersionUtils.randomVersionBetween(random, minVersion, maxVersion), request, - namedWriteableRegistry); - super.sendRequest(node, requestId, action, request, options); - } -} diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index f9b7e1d3a89..d3c28fd88e3 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -20,7 +20,12 @@ package org.elasticsearch.test.transport; import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.util.BigArray; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.transport.MockTcpTransport; import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportService; @@ -29,7 +34,6 @@ import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.component.LifecycleListener; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; @@ -37,7 +41,6 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.tasks.MockTaskManager; @@ -49,7 +52,6 @@ import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportServiceAdapter; -import org.elasticsearch.transport.local.LocalTransport; import java.io.IOException; import java.util.Arrays; @@ -74,6 +76,7 @@ import java.util.concurrent.CopyOnWriteArrayList; */ public final class MockTransportService extends TransportService { + public static class TestPlugin extends Plugin { @Override public List> getSettings() { @@ -81,14 +84,10 @@ public final class MockTransportService extends TransportService { } } - public static MockTransportService local(Settings settings, Version version, ThreadPool threadPool) { + public static MockTransportService createNewService(Settings settings, Version version, ThreadPool threadPool) { NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); - Transport transport = new LocalTransport(settings, threadPool, namedWriteableRegistry, new NoneCircuitBreakerService()) { - @Override - protected Version getVersion() { - return version; - } - }; + final Transport transport = new MockTcpTransport(settings, threadPool, BigArrays.NON_RECYCLING_INSTANCE, + new NoneCircuitBreakerService(), namedWriteableRegistry, new NetworkService(settings, Collections.emptyList()), version); return new MockTransportService(settings, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR); } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index ba831dde092..c05afdc69e3 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -69,12 +69,13 @@ import static org.hamcrest.Matchers.notNullValue; public abstract class AbstractSimpleTransportTestCase extends ESTestCase { protected ThreadPool threadPool; - - protected static final Version version0 = Version.CURRENT.minimumCompatibilityVersion(); + // we use always a non-alpha or beta version here otherwise minimumCompatibilityVersion will be different for the two used versions + private static final Version CURRENT_VERSION = Version.fromString(String.valueOf(Version.CURRENT.major) + ".0.0"); + protected static final Version version0 = CURRENT_VERSION.minimumCompatibilityVersion(); protected volatile DiscoveryNode nodeA; protected volatile MockTransportService serviceA; - protected static final Version version1 = Version.fromId(Version.CURRENT.id + 1); + protected static final Version version1 = Version.fromId(CURRENT_VERSION.id + 1); protected volatile DiscoveryNode nodeB; protected volatile MockTransportService serviceB; diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AssertingTransportInterceptor.java b/test/framework/src/main/java/org/elasticsearch/transport/AssertingTransportInterceptor.java new file mode 100644 index 00000000000..ee29864ba03 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/transport/AssertingTransportInterceptor.java @@ -0,0 +1,126 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.transport; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.NetworkPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; + +import java.util.Collections; +import java.util.List; +import java.util.Random; + +/** + * A transport interceptor that applies {@link ElasticsearchAssertions#assertVersionSerializable(Streamable)} + * to all requests and response objects send across the wire + */ +public final class AssertingTransportInterceptor implements TransportInterceptor { + + private final Random random; + private final NamedWriteableRegistry namedWriteableRegistry; + + public static final class TestPlugin extends Plugin implements NetworkPlugin { + + private final Settings settings; + + public TestPlugin(Settings settings) { + this.settings = settings; + } + + @Override + public List getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry) { + return Collections.singletonList(new AssertingTransportInterceptor(settings, namedWriteableRegistry)); + } + } + + public AssertingTransportInterceptor(Settings settings, NamedWriteableRegistry namedWriteableRegistry) { + final long seed = ESIntegTestCase.INDEX_TEST_SEED_SETTING.get(settings); + random = new Random(seed); + this.namedWriteableRegistry = namedWriteableRegistry; + } + + @Override + public TransportRequestHandler interceptHandler(String action, + TransportRequestHandler actualHandler) { + return new TransportRequestHandler() { + + @Override + public void messageReceived(T request, TransportChannel channel, Task task) throws Exception { + assertVersionSerializable(request); + actualHandler.messageReceived(request, channel, task); + } + + @Override + public void messageReceived(T request, TransportChannel channel) throws Exception { + assertVersionSerializable(request); + actualHandler.messageReceived(request, channel); + } + }; + } + + private void assertVersionSerializable(Streamable streamable) { + Version version = VersionUtils.randomVersionBetween(random, Version.CURRENT.minimumCompatibilityVersion(), Version.CURRENT); + ElasticsearchAssertions.assertVersionSerializable(version, streamable, namedWriteableRegistry); + + } + + @Override + public AsyncSender interceptSender(final AsyncSender sender) { + return new AsyncSender() { + @Override + public void sendRequest(DiscoveryNode node, String action, TransportRequest request, + TransportRequestOptions options, + final TransportResponseHandler handler) { + assertVersionSerializable(request); + sender.sendRequest(node, action, request, options, new TransportResponseHandler() { + @Override + public T newInstance() { + return handler.newInstance(); + } + + @Override + public void handleResponse(T response) { + assertVersionSerializable(response); + handler.handleResponse(response); + } + + @Override + public void handleException(TransportException exp) { + handler.handleException(exp); + } + + @Override + public String executor() { + return handler.executor(); + } + }); + } + }; + } + + +} diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java index a0c81cb63d7..96145fc6fca 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java @@ -29,7 +29,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.CancellableThreads; @@ -184,7 +184,7 @@ public class MockTcpTransport extends TcpTransport } } }; - InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address(); + final InetSocketAddress address = node.getAddress().address(); // we just use a single connections configureSocket(socket); socket.connect(address, (int) TCP_CONNECT_TIMEOUT.get(settings).millis()); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTransportClient.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTransportClient.java index a198ef77956..8338d5e5cfc 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/MockTransportClient.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTransportClient.java @@ -22,19 +22,30 @@ import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @SuppressWarnings({"unchecked","varargs"}) public class MockTransportClient extends TransportClient { - private static final Settings DEFAULT_SETTINGS = Settings.builder().put("transport.type.default", "local").build(); + private static final Settings DEFAULT_SETTINGS = Settings.builder().put("transport.type.default", + MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).build(); public MockTransportClient(Settings settings, Class... plugins) { - super(settings, DEFAULT_SETTINGS, Arrays.asList(plugins)); + this(settings, Arrays.asList(plugins)); } public MockTransportClient(Settings settings, Collection> plugins) { - super(settings, DEFAULT_SETTINGS, plugins); + super(settings, DEFAULT_SETTINGS, addMockTransportIfMissing(plugins)); + } + + private static Collection> addMockTransportIfMissing(Collection> plugins) { + if (plugins.contains(MockTcpTransportPlugin.class)) { + return plugins; + } + plugins = new ArrayList<>(plugins); + plugins.add(MockTcpTransportPlugin.class); + return plugins; } } diff --git a/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java b/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java index 51a90282b4f..d5cd304659a 100644 --- a/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java +++ b/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.MockSearchService; import org.elasticsearch.search.SearchService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.MockTcpTransportPlugin; import java.io.IOException; import java.util.ArrayList; @@ -40,10 +41,11 @@ public class MockNodeTests extends ESTestCase { public void testComponentsMockedByMarkerPlugins() throws IOException { Settings settings = Settings.builder() // All these are required or MockNode will fail to build. .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) - .put("transport.type", "local") + .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put("http.enabled", false) .build(); List> plugins = new ArrayList<>(); + plugins.add(MockTcpTransportPlugin.class); boolean useMockBigArrays = randomBoolean(); boolean useMockSearchService = randomBoolean(); if (useMockBigArrays) { diff --git a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java index f0b7454fe9d..e5b704b2f2c 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java @@ -33,6 +33,8 @@ import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.NodeConfigurationSource; +import org.elasticsearch.transport.MockTcpTransport; +import org.elasticsearch.transport.MockTcpTransportPlugin; import org.elasticsearch.transport.TransportSettings; import java.io.IOException; @@ -140,13 +142,13 @@ public class InternalTestClusterTests extends ESTestCase { 2 * ((masterNodes ? InternalTestCluster.DEFAULT_HIGH_NUM_MASTER_NODES : 0) + maxNumDataNodes + numClientNodes)) .put(NetworkModule.HTTP_ENABLED.getKey(), false) .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local") - .put(NetworkModule.TRANSPORT_TYPE_KEY, "local").build(); + .put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).build(); } @Override public Settings transportClientSettings() { return Settings.builder() - .put(NetworkModule.TRANSPORT_TYPE_KEY, "local").build(); + .put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).build(); } }; @@ -156,10 +158,10 @@ public class InternalTestClusterTests extends ESTestCase { Path baseDir = createTempDir(); InternalTestCluster cluster0 = new InternalTestCluster(clusterSeed, baseDir, masterNodes, minNumDataNodes, maxNumDataNodes, clusterName1, nodeConfigurationSource, numClientNodes, - enableHttpPipelining, nodePrefix, Collections.emptyList(), Function.identity()); + enableHttpPipelining, nodePrefix, Collections.singleton(MockTcpTransportPlugin.class), Function.identity()); InternalTestCluster cluster1 = new InternalTestCluster(clusterSeed, baseDir, masterNodes, minNumDataNodes, maxNumDataNodes, clusterName2, nodeConfigurationSource, numClientNodes, - enableHttpPipelining, nodePrefix, Collections.emptyList(), Function.identity()); + enableHttpPipelining, nodePrefix, Collections.singleton(MockTcpTransportPlugin.class), Function.identity()); assertClusters(cluster0, cluster1, false); long seed = randomLong(); @@ -202,14 +204,14 @@ public class InternalTestClusterTests extends ESTestCase { .put( NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2 + (masterNodes ? InternalTestCluster.DEFAULT_HIGH_NUM_MASTER_NODES : 0) + maxNumDataNodes + numClientNodes) - .put(NetworkModule.TRANSPORT_TYPE_KEY, "local") + .put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local") .build(); } @Override public Settings transportClientSettings() { return Settings.builder() - .put(NetworkModule.TRANSPORT_TYPE_KEY, "local").build(); + .put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).build(); } }; boolean enableHttpPipelining = randomBoolean(); @@ -217,7 +219,7 @@ public class InternalTestClusterTests extends ESTestCase { Path baseDir = createTempDir(); InternalTestCluster cluster = new InternalTestCluster(clusterSeed, baseDir, masterNodes, minNumDataNodes, maxNumDataNodes, clusterName1, nodeConfigurationSource, numClientNodes, - enableHttpPipelining, nodePrefix, Collections.emptyList(), Function.identity()); + enableHttpPipelining, nodePrefix, Collections.singleton(MockTcpTransportPlugin.class), Function.identity()); try { cluster.beforeTest(random(), 0.0); final Map shardNodePaths = new HashMap<>(); @@ -285,7 +287,7 @@ public class InternalTestClusterTests extends ESTestCase { return Settings.builder() .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), numNodes) .put(NetworkModule.HTTP_ENABLED.getKey(), false) - .put(NetworkModule.TRANSPORT_TYPE_KEY, "local") + .put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local") .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0).build(); } @@ -293,9 +295,9 @@ public class InternalTestClusterTests extends ESTestCase { @Override public Settings transportClientSettings() { return Settings.builder() - .put(NetworkModule.TRANSPORT_TYPE_KEY, "local").build(); + .put(NetworkModule.TRANSPORT_TYPE_KEY, MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).build(); } - }, 0, randomBoolean(), "", Collections.emptyList(), Function.identity()); + }, 0, randomBoolean(), "", Collections.singleton(MockTcpTransportPlugin.class), Function.identity()); cluster.beforeTest(random(), 0.0); try { Map> pathsPerRole = new HashMap<>();