From 52bebec51f60f23bb6fada00723a3a9ffb021704 Mon Sep 17 00:00:00 2001 From: William Brafford Date: Mon, 13 Apr 2020 17:18:39 -0400 Subject: [PATCH] NodeInfo response should use a collection rather than fields (#54460) (#55132) This is a first cut at giving NodeInfo the ability to carry a flexible list of heterogeneous info responses. The trick is to be able to serialize and deserialize an arbitrary list of blocks of information. It is convenient to be able to deserialize into usable Java objects so that we can aggregate nodes stats for the cluster stats endpoint. In order to provide a little bit of clarity about which objects can and can't be used as info blocks, I've introduced a new interface called "ReportingService." I have removed the hard-coded getters (e.g., getOs()) in favor of a flexible method that can return heterogeneous kinds of info blocks (e.g., getInfo(OsInfo.class)). Taking a class as an argument removes the need to cast in the client code. --- .../ReindexFromRemoteWithAuthTests.java | 3 +- .../index/reindex/RetryTests.java | 3 +- ...Netty4TransportMultiPortIntegrationIT.java | 7 +- .../Netty4TransportPublishAddressIT.java | 3 +- .../admin/cluster/node/info/NodeInfo.java | 145 +++++++----------- .../cluster/node/info/NodesInfoResponse.java | 39 +++-- .../cluster/node/info/PluginsAndModules.java | 6 +- .../cluster/stats/ClusterStatsNodes.java | 21 +-- .../ingest/PutPipelineTransportAction.java | 2 +- .../java/org/elasticsearch/http/HttpInfo.java | 5 +- .../http/HttpServerTransport.java | 4 +- .../org/elasticsearch/ingest/IngestInfo.java | 5 +- .../elasticsearch/ingest/IngestService.java | 4 +- .../elasticsearch/monitor/jvm/JvmInfo.java | 4 +- .../elasticsearch/monitor/jvm/JvmService.java | 4 +- .../org/elasticsearch/monitor/os/OsInfo.java | 5 +- .../elasticsearch/monitor/os/OsService.java | 4 +- .../monitor/process/ProcessInfo.java | 6 +- .../monitor/process/ProcessService.java | 4 +- .../elasticsearch/node/ReportingService.java | 31 ++++ .../elasticsearch/plugins/PluginsService.java | 4 +- .../rest/action/cat/RestNodeAttrsAction.java | 3 +- .../rest/action/cat/RestNodesAction.java | 7 +- .../rest/action/cat/RestPluginsAction.java | 2 +- .../rest/action/cat/RestThreadPoolAction.java | 6 +- .../elasticsearch/threadpool/ThreadPool.java | 4 +- .../threadpool/ThreadPoolInfo.java | 6 +- .../transport/TransportInfo.java | 5 +- .../transport/TransportService.java | 5 +- .../cluster/node/info/NodeInfoTests.java | 68 ++++++++ .../SettingsBasedSeedHostsProviderIT.java | 4 +- .../nodesinfo/NodeInfoStreamingTests.java | 16 +- .../nodesinfo/SimpleNodesInfoIT.java | 9 +- .../elasticsearch/test/ESIntegTestCase.java | 5 +- .../test/ExternalTestCluster.java | 3 +- .../ClusterStatsMonitoringDocTests.java | 8 +- .../elasticsearch/license/LicensingTests.java | 3 +- .../test/SecurityIntegTestCase.java | 6 +- .../test/SecuritySingleNodeTestCase.java | 13 +- .../xpack/security/authc/RunAsIntegTests.java | 3 +- .../transport/ssl/EllipticCurveSSLTests.java | 3 +- .../qa/SecurityTransportClientIT.java | 3 +- .../example/realm/CustomRealmIT.java | 5 +- .../SmokeTestMonitoringWithSecurityIT.java | 3 +- 44 files changed, 300 insertions(+), 199 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/node/ReportingService.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfoTests.java diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWithAuthTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWithAuthTests.java index 4e4e3218ad2..f1d363da5cd 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWithAuthTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFromRemoteWithAuthTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.http.HttpInfo; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestHeaderDefinition; @@ -99,7 +100,7 @@ public class ReindexFromRemoteWithAuthTests extends ESSingleNodeTestCase { @Before public void fetchTransportAddress() { NodeInfo nodeInfo = client().admin().cluster().prepareNodesInfo().get().getNodes().get(0); - address = nodeInfo.getHttp().getAddress().publishAddress(); + address = nodeInfo.getInfo(HttpInfo.class).getAddress().publishAddress(); } /** diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java index 916c18e38f7..0c341833ae7 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.http.HttpInfo; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -123,7 +124,7 @@ public class RetryTests extends ESIntegTestCase { } assertNotNull(masterNode); - TransportAddress address = masterNode.getHttp().getAddress().publishAddress(); + TransportAddress address = masterNode.getInfo(HttpInfo.class).getAddress().publishAddress(); RemoteInfo remote = new RemoteInfo("http", address.getAddress(), address.getPort(), null, new BytesArray("{\"match_all\":{}}"), null, null, emptyMap(), 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 ef899a5a3fe..e690ebead86 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 @@ -35,6 +35,7 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.junit.annotations.Network; import org.elasticsearch.transport.MockTransportClient; import org.elasticsearch.transport.Netty4Plugin; +import org.elasticsearch.transport.TransportInfo; import java.net.InetAddress; import java.util.Arrays; @@ -91,9 +92,9 @@ public class Netty4TransportMultiPortIntegrationIT extends ESNetty4IntegTestCase public void testThatInfosAreExposed() throws Exception { NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().clear().addMetric(TRANSPORT.metricName()).get(); for (NodeInfo nodeInfo : response.getNodes()) { - assertThat(nodeInfo.getTransport().getProfileAddresses().keySet(), hasSize(1)); - assertThat(nodeInfo.getTransport().getProfileAddresses(), hasKey("client1")); - BoundTransportAddress boundTransportAddress = nodeInfo.getTransport().getProfileAddresses().get("client1"); + assertThat(nodeInfo.getInfo(TransportInfo.class).getProfileAddresses().keySet(), hasSize(1)); + assertThat(nodeInfo.getInfo(TransportInfo.class).getProfileAddresses(), hasKey("client1")); + BoundTransportAddress boundTransportAddress = nodeInfo.getInfo(TransportInfo.class).getProfileAddresses().get("client1"); for (TransportAddress transportAddress : boundTransportAddress.boundAddresses()) { assertThat(transportAddress, instanceOf(TransportAddress.class)); } 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 28258299f9c..5f5041a6a12 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 @@ -29,6 +29,7 @@ import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.transport.Netty4Plugin; +import org.elasticsearch.transport.TransportInfo; import java.net.Inet4Address; @@ -68,7 +69,7 @@ public class Netty4TransportPublishAddressIT extends ESNetty4IntegTestCase { logger.info("--> checking if boundAddress matching publishAddress has same port"); NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); for (NodeInfo nodeInfo : nodesInfoResponse.getNodes()) { - BoundTransportAddress boundTransportAddress = nodeInfo.getTransport().getAddress(); + BoundTransportAddress boundTransportAddress = nodeInfo.getInfo(TransportInfo.class).getAddress(); if (nodeInfo.getNode().getName().equals(ipv4OnlyNode)) { assertThat(boundTransportAddress.boundAddresses().length, equalTo(1)); assertThat(boundTransportAddress.boundAddresses()[0].getPort(), equalTo(boundTransportAddress.publishAddress().getPort())); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java index 1a1bd4e0466..13dfdc0896b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java @@ -33,10 +33,13 @@ import org.elasticsearch.ingest.IngestInfo; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.monitor.os.OsInfo; import org.elasticsearch.monitor.process.ProcessInfo; +import org.elasticsearch.node.ReportingService; import org.elasticsearch.threadpool.ThreadPoolInfo; import org.elasticsearch.transport.TransportInfo; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; /** * Node information (static, does not change over time). @@ -49,29 +52,12 @@ public class NodeInfo extends BaseNodeResponse { @Nullable private Settings settings; - @Nullable - private OsInfo os; - - @Nullable - private ProcessInfo process; - - @Nullable - private JvmInfo jvm; - - @Nullable - private ThreadPoolInfo threadPool; - - @Nullable - private TransportInfo transport; - - @Nullable - private HttpInfo http; - - @Nullable - private PluginsAndModules plugins; - - @Nullable - private IngestInfo ingest; + /** + * Do not expose this map to other classes. For type safety, use {@link #getInfo(Class)} + * to retrieve items from this map and {@link #addInfoIfNonNull(Class, ReportingService.Info)} + * to retrieve items from it. + */ + private Map, ReportingService.Info> infoMap = new HashMap<>(); @Nullable private ByteSizeValue totalIndexingBuffer; @@ -88,14 +74,14 @@ public class NodeInfo extends BaseNodeResponse { if (in.readBoolean()) { settings = Settings.readSettingsFromStream(in); } - os = in.readOptionalWriteable(OsInfo::new); - process = in.readOptionalWriteable(ProcessInfo::new); - jvm = in.readOptionalWriteable(JvmInfo::new); - threadPool = in.readOptionalWriteable(ThreadPoolInfo::new); - transport = in.readOptionalWriteable(TransportInfo::new); - http = in.readOptionalWriteable(HttpInfo::new); - plugins = in.readOptionalWriteable(PluginsAndModules::new); - ingest = in.readOptionalWriteable(IngestInfo::new); + addInfoIfNonNull(OsInfo.class, in.readOptionalWriteable(OsInfo::new)); + addInfoIfNonNull(ProcessInfo.class, in.readOptionalWriteable(ProcessInfo::new)); + addInfoIfNonNull(JvmInfo.class, in.readOptionalWriteable(JvmInfo::new)); + addInfoIfNonNull(ThreadPoolInfo.class, in.readOptionalWriteable(ThreadPoolInfo::new)); + addInfoIfNonNull(TransportInfo.class, in.readOptionalWriteable(TransportInfo::new)); + addInfoIfNonNull(HttpInfo.class, in.readOptionalWriteable(HttpInfo::new)); + addInfoIfNonNull(PluginsAndModules.class, in.readOptionalWriteable(PluginsAndModules::new)); + addInfoIfNonNull(IngestInfo.class, in.readOptionalWriteable(IngestInfo::new)); } public NodeInfo(Version version, Build build, DiscoveryNode node, @Nullable Settings settings, @@ -106,14 +92,14 @@ public class NodeInfo extends BaseNodeResponse { this.version = version; this.build = build; this.settings = settings; - this.os = os; - this.process = process; - this.jvm = jvm; - this.threadPool = threadPool; - this.transport = transport; - this.http = http; - this.plugins = plugins; - this.ingest = ingest; + addInfoIfNonNull(OsInfo.class, os); + addInfoIfNonNull(ProcessInfo.class, process); + addInfoIfNonNull(JvmInfo.class, jvm); + addInfoIfNonNull(ThreadPoolInfo.class, threadPool); + addInfoIfNonNull(TransportInfo.class, transport); + addInfoIfNonNull(HttpInfo.class, http); + addInfoIfNonNull(PluginsAndModules.class, plugins); + addInfoIfNonNull(IngestInfo.class, ingest); this.totalIndexingBuffer = totalIndexingBuffer; } @@ -148,52 +134,17 @@ public class NodeInfo extends BaseNodeResponse { } /** - * Operating System level information. + * Get a particular info object, e.g. {@link JvmInfo} or {@link OsInfo}. This + * generic method handles all casting in order to spare client classes the + * work of explicit casts. This {@link NodeInfo} class guarantees type + * safety for these stored info blocks. + * + * @param clazz Class for retrieval. + * @param Specific subtype of ReportingService.Info to retrieve. + * @return An object of type T. */ - @Nullable - public OsInfo getOs() { - return this.os; - } - - /** - * Process level information. - */ - @Nullable - public ProcessInfo getProcess() { - return process; - } - - /** - * JVM level information. - */ - @Nullable - public JvmInfo getJvm() { - return jvm; - } - - @Nullable - public ThreadPoolInfo getThreadPool() { - return this.threadPool; - } - - @Nullable - public TransportInfo getTransport() { - return transport; - } - - @Nullable - public HttpInfo getHttp() { - return http; - } - - @Nullable - public PluginsAndModules getPlugins() { - return this.plugins; - } - - @Nullable - public IngestInfo getIngest() { - return ingest; + public T getInfo(Class clazz) { + return clazz.cast(infoMap.get(clazz)); } @Nullable @@ -201,6 +152,16 @@ public class NodeInfo extends BaseNodeResponse { return totalIndexingBuffer; } + /** + * Add a value to the map of information blocks. This method guarantees the + * type safety of the storage of heterogeneous types of reporting service information. + */ + private void addInfoIfNonNull(Class clazz, T info) { + if (info != null) { + infoMap.put(clazz, info); + } + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -218,13 +179,13 @@ public class NodeInfo extends BaseNodeResponse { out.writeBoolean(true); Settings.writeSettingsToStream(settings, out); } - out.writeOptionalWriteable(os); - out.writeOptionalWriteable(process); - out.writeOptionalWriteable(jvm); - out.writeOptionalWriteable(threadPool); - out.writeOptionalWriteable(transport); - out.writeOptionalWriteable(http); - out.writeOptionalWriteable(plugins); - out.writeOptionalWriteable(ingest); + out.writeOptionalWriteable(getInfo(OsInfo.class)); + out.writeOptionalWriteable(getInfo(ProcessInfo.class)); + out.writeOptionalWriteable(getInfo(JvmInfo.class)); + out.writeOptionalWriteable(getInfo(ThreadPoolInfo.class)); + out.writeOptionalWriteable(getInfo(TransportInfo.class)); + out.writeOptionalWriteable(getInfo(HttpInfo.class)); + out.writeOptionalWriteable(getInfo(PluginsAndModules.class)); + out.writeOptionalWriteable(getInfo(IngestInfo.class)); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java index 181fe4155b2..8251d49676c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java @@ -30,6 +30,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.http.HttpInfo; +import org.elasticsearch.ingest.IngestInfo; +import org.elasticsearch.monitor.jvm.JvmInfo; +import org.elasticsearch.monitor.os.OsInfo; +import org.elasticsearch.monitor.process.ProcessInfo; +import org.elasticsearch.threadpool.ThreadPoolInfo; +import org.elasticsearch.transport.TransportInfo; import java.io.IOException; import java.util.List; @@ -95,29 +102,29 @@ public class NodesInfoResponse extends BaseNodesResponse implements To builder.endObject(); } - if (nodeInfo.getOs() != null) { - nodeInfo.getOs().toXContent(builder, params); + if (nodeInfo.getInfo(OsInfo.class) != null) { + nodeInfo.getInfo(OsInfo.class).toXContent(builder, params); } - if (nodeInfo.getProcess() != null) { - nodeInfo.getProcess().toXContent(builder, params); + if (nodeInfo.getInfo(ProcessInfo.class) != null) { + nodeInfo.getInfo(ProcessInfo.class).toXContent(builder, params); } - if (nodeInfo.getJvm() != null) { - nodeInfo.getJvm().toXContent(builder, params); + if (nodeInfo.getInfo(JvmInfo.class) != null) { + nodeInfo.getInfo(JvmInfo.class).toXContent(builder, params); } - if (nodeInfo.getThreadPool() != null) { - nodeInfo.getThreadPool().toXContent(builder, params); + if (nodeInfo.getInfo(ThreadPoolInfo.class) != null) { + nodeInfo.getInfo(ThreadPoolInfo.class).toXContent(builder, params); } - if (nodeInfo.getTransport() != null) { - nodeInfo.getTransport().toXContent(builder, params); + if (nodeInfo.getInfo(TransportInfo.class) != null) { + nodeInfo.getInfo(TransportInfo.class).toXContent(builder, params); } - if (nodeInfo.getHttp() != null) { - nodeInfo.getHttp().toXContent(builder, params); + if (nodeInfo.getInfo(HttpInfo.class) != null) { + nodeInfo.getInfo(HttpInfo.class).toXContent(builder, params); } - if (nodeInfo.getPlugins() != null) { - nodeInfo.getPlugins().toXContent(builder, params); + if (nodeInfo.getInfo(PluginsAndModules.class) != null) { + nodeInfo.getInfo(PluginsAndModules.class).toXContent(builder, params); } - if (nodeInfo.getIngest() != null) { - nodeInfo.getIngest().toXContent(builder, params); + if (nodeInfo.getInfo(IngestInfo.class) != null) { + nodeInfo.getInfo(IngestInfo.class).toXContent(builder, params); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/PluginsAndModules.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/PluginsAndModules.java index 67ee5756789..01d293a4389 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/PluginsAndModules.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/PluginsAndModules.java @@ -21,10 +21,8 @@ package org.elasticsearch.action.admin.cluster.node.info; 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.xcontent.ToXContent.Params; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import org.elasticsearch.plugins.PluginInfo; import java.io.IOException; @@ -36,7 +34,7 @@ import java.util.List; /** * Information about plugins and modules */ -public class PluginsAndModules implements Writeable, ToXContentFragment { +public class PluginsAndModules implements ReportingService.Info { private final List plugins; private final List modules; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java index 62dbacd9397..02f85e327ab 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java @@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectIntHashMap; import com.carrotsearch.hppc.cursors.ObjectIntCursor; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; +import org.elasticsearch.action.admin.cluster.node.info.PluginsAndModules; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; @@ -38,7 +39,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.jvm.JvmInfo; +import org.elasticsearch.monitor.os.OsInfo; import org.elasticsearch.plugins.PluginInfo; +import org.elasticsearch.transport.TransportInfo; import java.io.IOException; import java.net.InetAddress; @@ -80,11 +83,11 @@ public class ClusterStatsNodes implements ToXContentFragment { nodeInfos.add(nodeResponse.nodeInfo()); nodeStats.add(nodeResponse.nodeStats()); this.versions.add(nodeResponse.nodeInfo().getVersion()); - this.plugins.addAll(nodeResponse.nodeInfo().getPlugins().getPluginInfos()); + this.plugins.addAll(nodeResponse.nodeInfo().getInfo(PluginsAndModules.class).getPluginInfos()); // now do the stats that should be deduped by hardware (implemented by ip deduping) TransportAddress publishAddress = - nodeResponse.nodeInfo().getTransport().address().publishAddress(); + nodeResponse.nodeInfo().getInfo(TransportInfo.class).address().publishAddress(); final InetAddress inetAddress = publishAddress.address().getAddress(); if (!seenAddresses.add(inetAddress)) { continue; @@ -256,14 +259,14 @@ public class ClusterStatsNodes implements ToXContentFragment { int availableProcessors = 0; int allocatedProcessors = 0; for (NodeInfo nodeInfo : nodeInfos) { - availableProcessors += nodeInfo.getOs().getAvailableProcessors(); - allocatedProcessors += nodeInfo.getOs().getAllocatedProcessors(); + availableProcessors += nodeInfo.getInfo(OsInfo.class).getAvailableProcessors(); + allocatedProcessors += nodeInfo.getInfo(OsInfo.class).getAllocatedProcessors(); - if (nodeInfo.getOs().getName() != null) { - names.addTo(nodeInfo.getOs().getName(), 1); + if (nodeInfo.getInfo(OsInfo.class).getName() != null) { + names.addTo(nodeInfo.getInfo(OsInfo.class).getName(), 1); } - if (nodeInfo.getOs().getPrettyName() != null) { - prettyNames.addTo(nodeInfo.getOs().getPrettyName(), 1); + if (nodeInfo.getInfo(OsInfo.class).getPrettyName() != null) { + prettyNames.addTo(nodeInfo.getInfo(OsInfo.class).getPrettyName(), 1); } } this.availableProcessors = availableProcessors; @@ -454,7 +457,7 @@ public class ClusterStatsNodes implements ToXContentFragment { long heapMax = 0; long heapUsed = 0; for (NodeInfo nodeInfo : nodeInfos) { - versions.addTo(new JvmVersion(nodeInfo.getJvm()), 1); + versions.addTo(new JvmVersion(nodeInfo.getInfo(JvmInfo.class)), 1); } for (NodeStats nodeStats : nodeStatsList) { diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java index 9b70b2e0194..2fa9e0e294e 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java @@ -78,7 +78,7 @@ public class PutPipelineTransportAction extends TransportMasterNodeAction { Map ingestInfos = new HashMap<>(); for (NodeInfo nodeInfo : nodeInfos.getNodes()) { - ingestInfos.put(nodeInfo.getNode(), nodeInfo.getIngest()); + ingestInfos.put(nodeInfo.getNode(), nodeInfo.getInfo(IngestInfo.class)); } ingestService.putPipeline(ingestInfos, request, listener); }, listener::onFailure)); diff --git a/server/src/main/java/org/elasticsearch/http/HttpInfo.java b/server/src/main/java/org/elasticsearch/http/HttpInfo.java index e792647f4a7..e2b7c60fd43 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpInfo.java +++ b/server/src/main/java/org/elasticsearch/http/HttpInfo.java @@ -22,18 +22,17 @@ package org.elasticsearch.http; import org.apache.logging.log4j.LogManager; 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.logging.DeprecationLogger; import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; -public class HttpInfo implements Writeable, ToXContentFragment { +public class HttpInfo implements ReportingService.Info { private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(HttpInfo.class)); diff --git a/server/src/main/java/org/elasticsearch/http/HttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/HttpServerTransport.java index e0216565850..0ae43babcda 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/HttpServerTransport.java @@ -22,15 +22,17 @@ package org.elasticsearch.http; import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.node.ReportingService; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; -public interface HttpServerTransport extends LifecycleComponent { +public interface HttpServerTransport extends LifecycleComponent, ReportingService { String HTTP_SERVER_WORKER_THREAD_NAME_PREFIX = "http_server_worker"; BoundTransportAddress boundAddress(); + @Override HttpInfo info(); HttpStats stats(); diff --git a/server/src/main/java/org/elasticsearch/ingest/IngestInfo.java b/server/src/main/java/org/elasticsearch/ingest/IngestInfo.java index ad7b59368e9..64f34a6b16d 100644 --- a/server/src/main/java/org/elasticsearch/ingest/IngestInfo.java +++ b/server/src/main/java/org/elasticsearch/ingest/IngestInfo.java @@ -21,9 +21,8 @@ package org.elasticsearch.ingest; 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.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; import java.util.List; @@ -31,7 +30,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeSet; -public class IngestInfo implements Writeable, ToXContentFragment { +public class IngestInfo implements ReportingService.Info { private final Set processors; diff --git a/server/src/main/java/org/elasticsearch/ingest/IngestService.java b/server/src/main/java/org/elasticsearch/ingest/IngestService.java index cca3e907741..9994f5328fc 100644 --- a/server/src/main/java/org/elasticsearch/ingest/IngestService.java +++ b/server/src/main/java/org/elasticsearch/ingest/IngestService.java @@ -48,6 +48,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.node.ReportingService; import org.elasticsearch.plugins.IngestPlugin; import org.elasticsearch.script.ScriptService; import org.elasticsearch.threadpool.ThreadPool; @@ -72,7 +73,7 @@ import java.util.function.IntConsumer; /** * Holder class for several ingest related services. */ -public class IngestService implements ClusterStateApplier { +public class IngestService implements ClusterStateApplier, ReportingService { public static final String NOOP_PIPELINE_NAME = "_none"; @@ -255,6 +256,7 @@ public class IngestService implements ClusterStateApplier { return processorFactories; } + @Override public IngestInfo info() { Map processorFactories = getProcessorFactories(); List processorInfoList = new ArrayList<>(processorFactories.size()); diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java index b8eda330337..534534c799a 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java @@ -28,8 +28,8 @@ 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.unit.ByteSizeValue; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; import java.lang.management.GarbageCollectorMXBean; @@ -44,7 +44,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -public class JvmInfo implements Writeable, ToXContentFragment { +public class JvmInfo implements ReportingService.Info { private static JvmInfo INSTANCE; diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmService.java b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmService.java index 29b879ffe93..79bd6be2fe9 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmService.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmService.java @@ -25,8 +25,9 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.node.ReportingService; -public class JvmService { +public class JvmService implements ReportingService { private static final Logger logger = LogManager.getLogger(JvmService.class); @@ -49,6 +50,7 @@ public class JvmService { logger.debug("using refresh_interval [{}]", refreshInterval); } + @Override public JvmInfo info() { return this.jvmInfo; } diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java b/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java index e7d02cb1a09..324c6bc0278 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java @@ -22,14 +22,13 @@ package org.elasticsearch.monitor.os; import org.elasticsearch.Version; 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.unit.TimeValue; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; -public class OsInfo implements Writeable, ToXContentFragment { +public class OsInfo implements ReportingService.Info { private final long refreshInterval; private final int availableProcessors; diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsService.java b/server/src/main/java/org/elasticsearch/monitor/os/OsService.java index d4ec4fda795..d8f1de192a0 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsService.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsService.java @@ -27,10 +27,11 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.SingleObjectCache; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.node.ReportingService; import java.io.IOException; -public class OsService { +public class OsService implements ReportingService { private static final Logger logger = LogManager.getLogger(OsService.class); @@ -50,6 +51,7 @@ public class OsService { logger.debug("using refresh_interval [{}]", refreshInterval); } + @Override public OsInfo info() { return this.info; } diff --git a/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java b/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java index 6bac78d2f14..3b17d22f119 100644 --- a/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java @@ -21,15 +21,13 @@ package org.elasticsearch.monitor.process; 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.unit.TimeValue; -import org.elasticsearch.common.xcontent.ToXContent.Params; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; -public class ProcessInfo implements Writeable, ToXContentFragment { +public class ProcessInfo implements ReportingService.Info { private final long refreshInterval; private final long id; diff --git a/server/src/main/java/org/elasticsearch/monitor/process/ProcessService.java b/server/src/main/java/org/elasticsearch/monitor/process/ProcessService.java index 963d865638c..a57fceef528 100644 --- a/server/src/main/java/org/elasticsearch/monitor/process/ProcessService.java +++ b/server/src/main/java/org/elasticsearch/monitor/process/ProcessService.java @@ -26,8 +26,9 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.SingleObjectCache; +import org.elasticsearch.node.ReportingService; -public final class ProcessService { +public final class ProcessService implements ReportingService { private static final Logger logger = LogManager.getLogger(ProcessService.class); @@ -47,6 +48,7 @@ public final class ProcessService { logger.debug("using refresh_interval [{}]", refreshInterval); } + @Override public ProcessInfo info() { return this.info; } diff --git a/server/src/main/java/org/elasticsearch/node/ReportingService.java b/server/src/main/java/org/elasticsearch/node/ReportingService.java new file mode 100644 index 00000000000..219d7c7f74b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/node/ReportingService.java @@ -0,0 +1,31 @@ +/* + * 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.node; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; + +public interface ReportingService { + I info(); + + interface Info extends Writeable, ToXContent { + + } +} diff --git a/server/src/main/java/org/elasticsearch/plugins/PluginsService.java b/server/src/main/java/org/elasticsearch/plugins/PluginsService.java index 240837077e1..4c3be448afc 100644 --- a/server/src/main/java/org/elasticsearch/plugins/PluginsService.java +++ b/server/src/main/java/org/elasticsearch/plugins/PluginsService.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexModule; +import org.elasticsearch.node.ReportingService; import org.elasticsearch.threadpool.ExecutorBuilder; import org.elasticsearch.transport.TransportSettings; @@ -69,7 +70,7 @@ import java.util.stream.Collectors; import static org.elasticsearch.common.io.FileSystemUtils.isAccessibleDirectory; -public class PluginsService { +public class PluginsService implements ReportingService { private static final Logger logger = LogManager.getLogger(PluginsService.class); @@ -271,6 +272,7 @@ public class PluginsService { /** * Get information about plugins and modules */ + @Override public PluginsAndModules info() { return info; } diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java index 5231baacb6f..3d14fa95cf8 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; +import org.elasticsearch.monitor.process.ProcessInfo; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.action.RestActionListener; @@ -108,7 +109,7 @@ public class RestNodeAttrsAction extends AbstractCatAction { table.startRow(); table.addCell(node.getName()); table.addCell(fullId ? node.getId() : Strings.substring(node.getId(), 0, 4)); - table.addCell(info == null ? null : info.getProcess().getId()); + table.addCell(info == null ? null : info.getInfo(ProcessInfo.class).getId()); table.addCell(node.getHostName()); table.addCell(node.getHostAddress()); table.addCell(node.getAddress().address().getPort()); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java index 8b0aac11464..e7d4ab51784 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestNodesAction.java @@ -54,6 +54,7 @@ import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.monitor.jvm.JvmStats; import org.elasticsearch.monitor.os.OsStats; +import org.elasticsearch.monitor.process.ProcessInfo; import org.elasticsearch.monitor.process.ProcessStats; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; @@ -275,7 +276,7 @@ public class RestNodesAction extends AbstractCatAction { NodeInfo info = nodesInfo.getNodesMap().get(node.getId()); NodeStats stats = nodesStats.getNodesMap().get(node.getId()); - JvmInfo jvmInfo = info == null ? null : info.getJvm(); + JvmInfo jvmInfo = info == null ? null : info.getInfo(JvmInfo.class); JvmStats jvmStats = stats == null ? null : stats.getJvm(); FsInfo fsInfo = stats == null ? null : stats.getFs(); OsStats osStats = stats == null ? null : stats.getOs(); @@ -285,10 +286,10 @@ public class RestNodesAction extends AbstractCatAction { table.startRow(); table.addCell(fullId ? node.getId() : Strings.substring(node.getId(), 0, 4)); - table.addCell(info == null ? null : info.getProcess().getId()); + table.addCell(info == null ? null : info.getInfo(ProcessInfo.class).getId()); table.addCell(node.getHostAddress()); table.addCell(node.getAddress().address().getPort()); - final HttpInfo httpInfo = info == null ? null : info.getHttp(); + final HttpInfo httpInfo = info == null ? null : info.getInfo(HttpInfo.class); if (httpInfo != null) { TransportAddress transportAddress = httpInfo.getAddress().publishAddress(); table.addCell(NetworkAddress.format(transportAddress.address())); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestPluginsAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestPluginsAction.java index f3c71321b9c..175f192b3f7 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestPluginsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestPluginsAction.java @@ -102,7 +102,7 @@ public class RestPluginsAction extends AbstractCatAction { if (info == null) { continue; } - PluginsAndModules plugins = info.getPlugins(); + PluginsAndModules plugins = info.getInfo(PluginsAndModules.class); if (plugins == null) { continue; } diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java index ea2182ddfc2..18074ee7656 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java @@ -32,11 +32,13 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Table; import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.monitor.process.ProcessInfo; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.rest.action.RestResponseListener; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.threadpool.ThreadPoolInfo; import org.elasticsearch.threadpool.ThreadPoolStats; import java.util.Collections; @@ -185,7 +187,7 @@ public class RestThreadPoolAction extends AbstractCatAction { poolThreadStats.put(threadPoolStat.getName(), threadPoolStat); } if (info != null) { - for (ThreadPool.Info threadPoolInfo : info.getThreadPool()) { + for (ThreadPool.Info threadPoolInfo : info.getInfo(ThreadPoolInfo.class)) { poolThreadInfo.put(threadPoolInfo.getName(), threadPoolInfo); } } @@ -199,7 +201,7 @@ public class RestThreadPoolAction extends AbstractCatAction { table.addCell(node.getName()); table.addCell(node.getId()); table.addCell(node.getEphemeralId()); - table.addCell(info == null ? null : info.getProcess().getId()); + table.addCell(info == null ? null : info.getInfo(ProcessInfo.class).getId()); table.addCell(node.getHostName()); table.addCell(node.getHostAddress()); table.addCell(node.getAddress().address().getPort()); diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 8316d457abe..8bd44128a2e 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.util.concurrent.XRejectedExecutionHandler; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.node.Node; +import org.elasticsearch.node.ReportingService; import java.io.IOException; import java.util.ArrayList; @@ -59,7 +60,7 @@ import java.util.stream.Collectors; import static java.util.Collections.unmodifiableMap; -public class ThreadPool implements Scheduler { +public class ThreadPool implements ReportingService, Scheduler { private static final Logger logger = LogManager.getLogger(ThreadPool.class); @@ -260,6 +261,7 @@ public class ThreadPool implements Scheduler { return cachedTimeThread.absoluteTimeInMillis(); } + @Override public ThreadPoolInfo info() { return threadPoolInfo; } diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPoolInfo.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPoolInfo.java index 3adb487b8b5..fcba058a820 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPoolInfo.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPoolInfo.java @@ -21,17 +21,15 @@ package org.elasticsearch.threadpool; 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.xcontent.ToXContent.Params; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; -public class ThreadPoolInfo implements Writeable, Iterable, ToXContentFragment { +public class ThreadPoolInfo implements ReportingService.Info, Iterable { private final List infos; diff --git a/server/src/main/java/org/elasticsearch/transport/TransportInfo.java b/server/src/main/java/org/elasticsearch/transport/TransportInfo.java index 9212fef6317..607321fc954 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportInfo.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportInfo.java @@ -23,13 +23,12 @@ import org.apache.logging.log4j.LogManager; import org.elasticsearch.common.Nullable; 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.logging.DeprecationLogger; import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.node.ReportingService; import java.io.IOException; import java.util.HashMap; @@ -37,7 +36,7 @@ import java.util.Map; import static org.elasticsearch.common.Booleans.parseBoolean; -public class TransportInfo implements Writeable, ToXContentFragment { +public class TransportInfo implements ReportingService.Info { private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(TransportInfo.class)); diff --git a/server/src/main/java/org/elasticsearch/transport/TransportService.java b/server/src/main/java/org/elasticsearch/transport/TransportService.java index 8da2c1380b6..5638bb5a1e8 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportService.java @@ -50,6 +50,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeClosedException; +import org.elasticsearch.node.ReportingService; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.threadpool.Scheduler; @@ -73,7 +74,8 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; -public class TransportService extends AbstractLifecycleComponent implements TransportMessageListener, TransportConnectionListener { +public class TransportService extends AbstractLifecycleComponent implements ReportingService, TransportMessageListener, + TransportConnectionListener { private static final Logger logger = LogManager.getLogger(TransportService.class); public static final String DIRECT_RESPONSE_PROFILE = ".direct"; @@ -299,6 +301,7 @@ public class TransportService extends AbstractLifecycleComponent implements Tran handleIncomingRequests.set(true); } + @Override public TransportInfo info() { BoundTransportAddress boundTransportAddress = boundAddress(); if (boundTransportAddress == null) { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfoTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfoTests.java new file mode 100644 index 00000000000..6a4aa92c81b --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfoTests.java @@ -0,0 +1,68 @@ +/* + * 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.action.admin.cluster.node.info; + +import org.elasticsearch.Build; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.monitor.jvm.JvmInfo; +import org.elasticsearch.monitor.os.OsInfo; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +/** + * Tests for {@link NodeInfo}. Serialization and deserialization tested in + * {@link org.elasticsearch.nodesinfo.NodeInfoStreamingTests}. + */ +public class NodeInfoTests extends ESTestCase { + + /** + * Check that the the {@link NodeInfo#getInfo(Class)} method returns null + * for absent info objects, and returns the right thing for present info + * objects. + */ + public void testGetInfo() { + NodeInfo nodeInfo = new NodeInfo( + Version.CURRENT, + Build.CURRENT, + new DiscoveryNode("test_node", buildNewFakeTransportAddress(), emptyMap(), emptySet(), VersionUtils.randomVersion(random())), + null, + null, + null, + JvmInfo.jvmInfo(), + null, + null, + null, + null, + null, + null); + + // OsInfo is absent + assertThat(nodeInfo.getInfo(OsInfo.class), nullValue()); + + // JvmInfo is present + assertThat(nodeInfo.getInfo(JvmInfo.class), notNullValue()); + } +} diff --git a/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderIT.java b/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderIT.java index 290d0fbdf58..8a0fd494357 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.TransportInfo; import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING; import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; @@ -51,7 +52,8 @@ public class SettingsBasedSeedHostsProviderIT extends ESIntegTestCase { final String seedNodeName = internalCluster().startNode(); final NodesInfoResponse nodesInfoResponse = client(seedNodeName).admin().cluster().nodesInfo(new NodesInfoRequest("_local")).actionGet(); - final String seedNodeAddress = nodesInfoResponse.getNodes().get(0).getTransport().getAddress().publishAddress().toString(); + final String seedNodeAddress = + nodesInfoResponse.getNodes().get(0).getInfo(TransportInfo.class).getAddress().publishAddress().toString(); logger.info("--> using seed node address {}", seedNodeAddress); int extraNodes = randomIntBetween(1, 5); diff --git a/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java b/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java index c41caf1523f..41632c51988 100644 --- a/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java +++ b/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java @@ -75,16 +75,16 @@ public class NodeInfoStreamingTests extends ESTestCase { assertThat(nodeInfo.getBuild().toString(), equalTo(readNodeInfo.getBuild().toString())); assertThat(nodeInfo.getHostname(), equalTo(readNodeInfo.getHostname())); assertThat(nodeInfo.getVersion(), equalTo(readNodeInfo.getVersion())); - compareJsonOutput(nodeInfo.getHttp(), readNodeInfo.getHttp()); - compareJsonOutput(nodeInfo.getJvm(), readNodeInfo.getJvm()); - compareJsonOutput(nodeInfo.getProcess(), readNodeInfo.getProcess()); + compareJsonOutput(nodeInfo.getInfo(HttpInfo.class), readNodeInfo.getInfo(HttpInfo.class)); + compareJsonOutput(nodeInfo.getInfo(JvmInfo.class), readNodeInfo.getInfo(JvmInfo.class)); + compareJsonOutput(nodeInfo.getInfo(ProcessInfo.class), readNodeInfo.getInfo(ProcessInfo.class)); compareJsonOutput(nodeInfo.getSettings(), readNodeInfo.getSettings()); - compareJsonOutput(nodeInfo.getThreadPool(), readNodeInfo.getThreadPool()); - compareJsonOutput(nodeInfo.getTransport(), readNodeInfo.getTransport()); + compareJsonOutput(nodeInfo.getInfo(ThreadPoolInfo.class), readNodeInfo.getInfo(ThreadPoolInfo.class)); + compareJsonOutput(nodeInfo.getInfo(TransportInfo.class), readNodeInfo.getInfo(TransportInfo.class)); compareJsonOutput(nodeInfo.getNode(), readNodeInfo.getNode()); - compareJsonOutput(nodeInfo.getOs(), readNodeInfo.getOs()); - compareJsonOutput(nodeInfo.getPlugins(), readNodeInfo.getPlugins()); - compareJsonOutput(nodeInfo.getIngest(), readNodeInfo.getIngest()); + compareJsonOutput(nodeInfo.getInfo(OsInfo.class), readNodeInfo.getInfo(OsInfo.class)); + compareJsonOutput(nodeInfo.getInfo(PluginsAndModules.class), readNodeInfo.getInfo(PluginsAndModules.class)); + compareJsonOutput(nodeInfo.getInfo(IngestInfo.class), readNodeInfo.getInfo(IngestInfo.class)); } private void compareJsonOutput(ToXContent param1, ToXContent param2) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/nodesinfo/SimpleNodesInfoIT.java b/server/src/test/java/org/elasticsearch/nodesinfo/SimpleNodesInfoIT.java index a0a6c1c7de1..2b2f1478ef3 100644 --- a/server/src/test/java/org/elasticsearch/nodesinfo/SimpleNodesInfoIT.java +++ b/server/src/test/java/org/elasticsearch/nodesinfo/SimpleNodesInfoIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.monitor.os.OsInfo; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; @@ -135,12 +136,12 @@ public class SimpleNodesInfoIT extends ESIntegTestCase { assertThat(response.getNodesMap().get(server1NodeId), notNullValue()); assertThat(response.getNodesMap().get(server2NodeId), notNullValue()); - assertThat(response.getNodesMap().get(server1NodeId).getOs().getAvailableProcessors(), + assertThat(response.getNodesMap().get(server1NodeId).getInfo(OsInfo.class).getAvailableProcessors(), equalTo(Runtime.getRuntime().availableProcessors())); - assertThat(response.getNodesMap().get(server2NodeId).getOs().getAvailableProcessors(), + assertThat(response.getNodesMap().get(server2NodeId).getInfo(OsInfo.class).getAvailableProcessors(), equalTo(Runtime.getRuntime().availableProcessors())); - assertThat(response.getNodesMap().get(server1NodeId).getOs().getAllocatedProcessors(), equalTo(3)); - assertThat(response.getNodesMap().get(server2NodeId).getOs().getAllocatedProcessors(), equalTo(6)); + assertThat(response.getNodesMap().get(server1NodeId).getInfo(OsInfo.class).getAllocatedProcessors(), equalTo(3)); + assertThat(response.getNodesMap().get(server2NodeId).getInfo(OsInfo.class).getAllocatedProcessors(), equalTo(6)); } } 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 e03f35c865d..cd88e210d6d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -112,6 +112,7 @@ import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.http.HttpInfo; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexSettings; @@ -2314,8 +2315,8 @@ public abstract class ESIntegTestCase extends ESTestCase { RestClientBuilder.HttpClientConfigCallback httpClientConfigCallback, String protocol) { List hosts = new ArrayList<>(); for (NodeInfo node : nodes) { - if (node.getHttp() != null) { - TransportAddress publishAddress = node.getHttp().address().publishAddress(); + if (node.getInfo(HttpInfo.class) != null) { + TransportAddress publishAddress = node.getInfo(HttpInfo.class).address().publishAddress(); InetSocketAddress address = publishAddress.address(); hosts.add(new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), protocol)); } 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 868ed7511c0..95d4b26fec6 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ExternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ExternalTestCluster.java @@ -33,6 +33,7 @@ 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.http.HttpInfo; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.transport.MockTransportClient; import org.elasticsearch.transport.TransportSettings; @@ -106,7 +107,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] = nodeInfo.getHttp().address().publishAddress().address(); + httpAddresses[i] = nodeInfo.getInfo(HttpInfo.class).address().publishAddress().address(); if (DiscoveryNode.isDataNode(nodeInfo.getSettings())) { dataNodes++; masterAndDataNodes++; diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java index f3f419cff94..6e21730093c 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java @@ -241,7 +241,7 @@ public class ClusterStatsMonitoringDocTests extends BaseMonitoringDocTestCase seedHosts = internalCluster().masterClient().admin().cluster().nodesInfo(new NodesInfoRequest()).get() - .getNodes().stream().map(n -> n.getTransport().getAddress().publishAddress().toString()).distinct() + .getNodes().stream().map(n -> n.getInfo(TransportInfo.class).getAddress().publishAddress().toString()).distinct() .collect(Collectors.toList()); Path home = createTempDir(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java index 1294c3480eb..4aed192d408 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java @@ -12,6 +12,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.elasticsearch.action.admin.cluster.node.info.PluginsAndModules; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.get.GetIndexRequest; @@ -36,6 +37,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.gateway.GatewayService; +import org.elasticsearch.http.HttpInfo; import org.elasticsearch.index.Index; import org.elasticsearch.license.LicenseService; import org.elasticsearch.plugins.Plugin; @@ -231,7 +233,7 @@ public abstract class SecurityIntegTestCase extends ESIntegTestCase { // TODO: disable this assertion for now, due to random runs with mock plugins. perhaps run without mock plugins? // assertThat(nodeInfo.getPlugins().getInfos(), hasSize(2)); Collection pluginNames = - nodeInfo.getPlugins().getPluginInfos().stream().map(p -> p.getClassname()).collect(Collectors.toList()); + nodeInfo.getInfo(PluginsAndModules.class).getPluginInfos().stream().map(p -> p.getClassname()).collect(Collectors.toList()); assertThat("plugin [" + LocalStateSecurity.class.getName() + "] not found in [" + pluginNames + "]", pluginNames, hasItem(LocalStateSecurity.class.getName())); } @@ -479,7 +481,7 @@ public abstract class SecurityIntegTestCase extends ESIntegTestCase { assertTrue("there is at least one node", nodes.size() > 0); NodeInfo ni = randomFrom(nodes); boolean useSSL = XPackSettings.HTTP_SSL_ENABLED.get(ni.getSettings()); - TransportAddress publishAddress = ni.getHttp().address().publishAddress(); + TransportAddress publishAddress = ni.getInfo(HttpInfo.class).address().publishAddress(); InetSocketAddress address = publishAddress.address(); return (useSSL ? "https://" : "http://") + NetworkAddress.format(address.getAddress()) + ":" + address.getPort(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySingleNodeTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySingleNodeTestCase.java index e369fd06dd4..c149c733cb2 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySingleNodeTestCase.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySingleNodeTestCase.java @@ -10,6 +10,7 @@ import io.netty.util.concurrent.GlobalEventExecutor; import org.apache.http.HttpHost; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.elasticsearch.action.admin.cluster.node.info.PluginsAndModules; import org.elasticsearch.client.Client; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; @@ -19,6 +20,7 @@ import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.http.HttpInfo; import org.elasticsearch.license.LicenseService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginInfo; @@ -153,9 +155,10 @@ public abstract class SecuritySingleNodeTestCase extends ESSingleNodeTestCase { NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().addMetric(PLUGINS.metricName()).get(); for (NodeInfo nodeInfo : nodeInfos.getNodes()) { // TODO: disable this assertion for now, due to random runs with mock plugins. perhaps run without mock plugins? - // assertThat(nodeInfo.getPlugins().getInfos(), hasSize(2)); - Collection pluginNames = - nodeInfo.getPlugins().getPluginInfos().stream().map(PluginInfo::getClassname).collect(Collectors.toList()); + // assertThat(nodeInfo.getInfo(PluginsAndModules.class).getInfos(), hasSize(2)); + Collection pluginNames = nodeInfo.getInfo(PluginsAndModules.class).getPluginInfos().stream() + .map(PluginInfo::getClassname) + .collect(Collectors.toList()); assertThat("plugin [" + LocalStateSecurity.class.getName() + "] not found in [" + pluginNames + "]", pluginNames, hasItem(LocalStateSecurity.class.getName())); } @@ -311,8 +314,8 @@ public abstract class SecuritySingleNodeTestCase extends ESSingleNodeTestCase { assertFalse(nodesInfoResponse.hasFailures()); assertEquals(nodesInfoResponse.getNodes().size(), 1); NodeInfo node = nodesInfoResponse.getNodes().get(0); - assertNotNull(node.getHttp()); - TransportAddress publishAddress = node.getHttp().address().publishAddress(); + assertNotNull(node.getInfo(HttpInfo.class)); + TransportAddress publishAddress = node.getInfo(HttpInfo.class).address().publishAddress(); InetSocketAddress address = publishAddress.address(); final HttpHost host = new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), protocol); RestClientBuilder builder = RestClient.builder(host); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/RunAsIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/RunAsIntegTests.java index c114f1ee818..9450ab5c4fb 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/RunAsIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/RunAsIntegTests.java @@ -19,6 +19,7 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.SecurityIntegTestCase; import org.elasticsearch.test.SecuritySettingsSource; import org.elasticsearch.test.SecuritySettingsSourceField; +import org.elasticsearch.transport.TransportInfo; import org.elasticsearch.xpack.core.TestXPackTransportClient; import org.elasticsearch.xpack.core.security.authc.AuthenticationServiceField; import org.elasticsearch.xpack.security.LocalStateSecurity; @@ -230,7 +231,7 @@ public class RunAsIntegTests extends SecurityIntegTestCase { NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().get(); List nodes = nodeInfos.getNodes(); assertTrue(nodes.isEmpty() == false); - TransportAddress publishAddress = randomFrom(nodes).getTransport().address().publishAddress(); + TransportAddress publishAddress = randomFrom(nodes).getInfo(TransportInfo.class).address().publishAddress(); String clusterName = nodeInfos.getClusterName().value(); Settings settings = Settings.builder() diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java index d4a5cf27401..068c999e383 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.SecurityIntegTestCase; +import org.elasticsearch.transport.TransportInfo; import org.elasticsearch.xpack.core.ssl.CertParsingUtils; import org.elasticsearch.xpack.core.ssl.PemUtils; import org.junit.BeforeClass; @@ -88,7 +89,7 @@ public class EllipticCurveSSLTests extends SecurityIntegTestCase { new SecureRandom()); SSLSocketFactory socketFactory = sslContext.getSocketFactory(); NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().addMetric(TRANSPORT.metricName()).get(); - TransportAddress address = randomFrom(response.getNodes()).getTransport().getAddress().publishAddress(); + TransportAddress address = randomFrom(response.getNodes()).getInfo(TransportInfo.class).getAddress().publishAddress(); final CountDownLatch latch = new CountDownLatch(1); try (SSLSocket sslSocket = AccessController.doPrivileged(new PrivilegedExceptionAction() { diff --git a/x-pack/qa/security-client-tests/src/test/java/org/elasticsearch/xpack/security/qa/SecurityTransportClientIT.java b/x-pack/qa/security-client-tests/src/test/java/org/elasticsearch/xpack/security/qa/SecurityTransportClientIT.java index 6892f640415..6af48b1514d 100644 --- a/x-pack/qa/security-client-tests/src/test/java/org/elasticsearch/xpack/security/qa/SecurityTransportClientIT.java +++ b/x-pack/qa/security-client-tests/src/test/java/org/elasticsearch/xpack/security/qa/SecurityTransportClientIT.java @@ -16,6 +16,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.TransportInfo; import org.elasticsearch.xpack.core.XPackClientPlugin; import org.elasticsearch.xpack.client.PreBuiltXPackTransportClient; import org.elasticsearch.xpack.core.security.SecurityField; @@ -100,7 +101,7 @@ public class SecurityTransportClientIT extends ESIntegTestCase { NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().get(); List nodes = nodeInfos.getNodes(); assertTrue(nodes.isEmpty() == false); - TransportAddress publishAddress = randomFrom(nodes).getTransport().address().publishAddress(); + TransportAddress publishAddress = randomFrom(nodes).getInfo(TransportInfo.class).address().publishAddress(); String clusterName = nodeInfos.getClusterName().value(); Settings settings = Settings.builder() diff --git a/x-pack/qa/security-example-spi-extension/src/test/java/org/elasticsearch/example/realm/CustomRealmIT.java b/x-pack/qa/security-example-spi-extension/src/test/java/org/elasticsearch/example/realm/CustomRealmIT.java index 2e242cf853d..f33ecf7144d 100644 --- a/x-pack/qa/security-example-spi-extension/src/test/java/org/elasticsearch/example/realm/CustomRealmIT.java +++ b/x-pack/qa/security-example-spi-extension/src/test/java/org/elasticsearch/example/realm/CustomRealmIT.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.TransportInfo; import org.elasticsearch.xpack.client.PreBuiltXPackTransportClient; import org.elasticsearch.xpack.core.XPackClientPlugin; @@ -76,7 +77,7 @@ public class CustomRealmIT extends ESIntegTestCase { NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().get(); List nodes = nodeInfos.getNodes(); assertTrue(nodes.isEmpty() == false); - TransportAddress publishAddress = randomFrom(nodes).getTransport().address().publishAddress(); + TransportAddress publishAddress = randomFrom(nodes).getInfo(TransportInfo.class).address().publishAddress(); String clusterName = nodeInfos.getClusterName().value(); Settings settings = Settings.builder() @@ -96,7 +97,7 @@ public class CustomRealmIT extends ESIntegTestCase { NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().get(); List nodes = nodeInfos.getNodes(); assertTrue(nodes.isEmpty() == false); - TransportAddress publishAddress = randomFrom(nodes).getTransport().address().publishAddress(); + TransportAddress publishAddress = randomFrom(nodes).getInfo(TransportInfo.class).address().publishAddress(); String clusterName = nodeInfos.getClusterName().value(); Settings settings = Settings.builder() diff --git a/x-pack/qa/smoke-test-plugins-ssl/src/test/java/org/elasticsearch/smoketest/SmokeTestMonitoringWithSecurityIT.java b/x-pack/qa/smoke-test-plugins-ssl/src/test/java/org/elasticsearch/smoketest/SmokeTestMonitoringWithSecurityIT.java index eb300e68157..b21f01f614c 100644 --- a/x-pack/qa/smoke-test-plugins-ssl/src/test/java/org/elasticsearch/smoketest/SmokeTestMonitoringWithSecurityIT.java +++ b/x-pack/qa/smoke-test-plugins-ssl/src/test/java/org/elasticsearch/smoketest/SmokeTestMonitoringWithSecurityIT.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResp import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.http.HttpInfo; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.xpack.core.XPackPlugin; @@ -176,7 +177,7 @@ public class SmokeTestMonitoringWithSecurityIT extends ESIntegTestCase { InetSocketAddress[] httpAddresses = new InetSocketAddress[nodes.size()]; for (int i = 0; i < nodes.size(); i++) { - httpAddresses[i] = nodes.get(i).getHttp().address().publishAddress().address(); + httpAddresses[i] = nodes.get(i).getInfo(HttpInfo.class).address().publishAddress().address(); } return NetworkAddress.format(randomFrom(httpAddresses)); }