Refactor nodes stats request builders to match requests (#54363) (#54604)

* Refactor nodes stats request builders to match requests (#54363)

* Remove hard-coded setters from NodesInfoRequestBuilder

* Remove hard-coded setters from NodesStatsRequest

* Use static imports to reduce clutter

* Remove uses of old info APIs
This commit is contained in:
William Brafford 2020-04-01 17:03:04 -04:00 committed by GitHub
parent fd729a6509
commit 958e9d1b78
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 115 additions and 219 deletions

View File

@ -39,6 +39,7 @@ import java.util.Map;
import java.util.function.Consumer;
import java.util.function.Function;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.INGEST;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -99,7 +100,9 @@ public class IngestRestartIT extends ESIntegTestCase {
);
assertTrue(e.getMessage().contains("this script always fails"));
NodesStatsResponse r = client().admin().cluster().prepareNodesStats(internalCluster().getNodeNames()).setIngest(true).get();
NodesStatsResponse r = client().admin().cluster().prepareNodesStats(internalCluster().getNodeNames())
.addMetric(INGEST.metricName())
.get();
int nodeCount = r.getNodes().size();
for (int k = 0; k < nodeCount; k++) {
List<IngestStats.ProcessorStat> stats = r.getNodes().get(k).getIngestStats().getProcessorStats().get(pipelineId);

View File

@ -40,6 +40,7 @@ import java.net.InetAddress;
import java.util.Arrays;
import java.util.Locale;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.TRANSPORT;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.hasKey;
@ -88,7 +89,7 @@ public class Netty4TransportMultiPortIntegrationIT extends ESNetty4IntegTestCase
@Network
public void testThatInfosAreExposed() throws Exception {
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().clear().setTransport(true).get();
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"));

View File

@ -22,7 +22,6 @@ package org.elasticsearch.action.admin.cluster.node.info;
import org.elasticsearch.action.support.nodes.NodesOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
// TODO: This class's interface should match that of NodesInfoRequest
public class NodesInfoRequestBuilder extends NodesOperationRequestBuilder<NodesInfoRequest, NodesInfoResponse, NodesInfoRequestBuilder> {
public NodesInfoRequestBuilder(ElasticsearchClient client, NodesInfoAction action) {
@ -46,90 +45,24 @@ public class NodesInfoRequestBuilder extends NodesOperationRequestBuilder<NodesI
}
/**
* Should the node settings be returned.
* Add a single metric to the request.
*
* @param metric Name of metric as a string.
* @return This, for request chaining.
*/
public NodesInfoRequestBuilder setSettings(boolean settings) {
addOrRemoveMetric(settings, NodesInfoRequest.Metric.SETTINGS);
public NodesInfoRequestBuilder addMetric(String metric) {
request.addMetric(metric);
return this;
}
/**
* Should the node OS info be returned.
* Add an array of metrics to the request.
*
* @param metrics Metric names as strings.
* @return This, for request chaining.
*/
public NodesInfoRequestBuilder setOs(boolean os) {
addOrRemoveMetric(os, NodesInfoRequest.Metric.OS);
public NodesInfoRequestBuilder addMetrics(String... metrics) {
request.addMetrics(metrics);
return this;
}
/**
* Should the node OS process be returned.
*/
public NodesInfoRequestBuilder setProcess(boolean process) {
addOrRemoveMetric(process, NodesInfoRequest.Metric.PROCESS);
return this;
}
/**
* Should the node JVM info be returned.
*/
public NodesInfoRequestBuilder setJvm(boolean jvm) {
addOrRemoveMetric(jvm, NodesInfoRequest.Metric.JVM);
return this;
}
/**
* Should the node thread pool info be returned.
*/
public NodesInfoRequestBuilder setThreadPool(boolean threadPool) {
addOrRemoveMetric(threadPool, NodesInfoRequest.Metric.THREAD_POOL);
return this;
}
/**
* Should the node Transport info be returned.
*/
public NodesInfoRequestBuilder setTransport(boolean transport) {
addOrRemoveMetric(transport, NodesInfoRequest.Metric.TRANSPORT);
return this;
}
/**
* Should the node HTTP info be returned.
*/
public NodesInfoRequestBuilder setHttp(boolean http) {
addOrRemoveMetric(http, NodesInfoRequest.Metric.HTTP);
return this;
}
/**
* Should the node plugins info be returned.
*/
public NodesInfoRequestBuilder setPlugins(boolean plugins) {
addOrRemoveMetric(plugins, NodesInfoRequest.Metric.PLUGINS);
return this;
}
/**
* Should the node ingest info be returned.
*/
public NodesInfoRequestBuilder setIngest(boolean ingest) {
addOrRemoveMetric(ingest, NodesInfoRequest.Metric.INGEST);
return this;
}
/**
* Should the node indices info be returned.
*/
public NodesInfoRequestBuilder setIndices(boolean indices) {
addOrRemoveMetric(indices, NodesInfoRequest.Metric.INDICES);
return this;
}
private void addOrRemoveMetric(boolean includeMetric, NodesInfoRequest.Metric metric) {
if (includeMetric) {
request.addMetric(metric.metricName());
} else {
request.removeMetric(metric.metricName());
}
}
}

View File

@ -46,6 +46,28 @@ public class NodesStatsRequestBuilder
return this;
}
/**
* Add a single metric to the request.
*
* @param metric Name of metric as a string.
* @return This, for request chaining.
*/
public NodesStatsRequestBuilder addMetric(String metric) {
request.addMetric(metric);
return this;
}
/**
* Add an array of metrics to the request.
*
* @param metrics Metric names as strings.
* @return This, for request chaining.
*/
public NodesStatsRequestBuilder addMetrics(String... metrics) {
request.addMetrics(metrics);
return this;
}
/**
* Should the node indices stats be returned.
*/
@ -54,16 +76,6 @@ public class NodesStatsRequestBuilder
return this;
}
public NodesStatsRequestBuilder setBreaker(boolean breaker) {
addOrRemoveMetric(breaker, NodesStatsRequest.Metric.BREAKER);
return this;
}
public NodesStatsRequestBuilder setScript(boolean script) {
addOrRemoveMetric(script, NodesStatsRequest.Metric.SCRIPT);
return this;
}
/**
* Should the node indices stats be returned.
*/
@ -71,101 +83,4 @@ public class NodesStatsRequestBuilder
request.indices(indices);
return this;
}
/**
* Should the node OS stats be returned.
*/
public NodesStatsRequestBuilder setOs(boolean os) {
addOrRemoveMetric(os, NodesStatsRequest.Metric.OS);
return this;
}
/**
* Should the node OS stats be returned.
*/
public NodesStatsRequestBuilder setProcess(boolean process) {
addOrRemoveMetric(process, NodesStatsRequest.Metric.PROCESS);
return this;
}
/**
* Should the node JVM stats be returned.
*/
public NodesStatsRequestBuilder setJvm(boolean jvm) {
addOrRemoveMetric(jvm, NodesStatsRequest.Metric.JVM);
return this;
}
/**
* Should the node thread pool stats be returned.
*/
public NodesStatsRequestBuilder setThreadPool(boolean threadPool) {
addOrRemoveMetric(threadPool, NodesStatsRequest.Metric.THREAD_POOL);
return this;
}
/**
* Should the node file system stats be returned.
*/
public NodesStatsRequestBuilder setFs(boolean fs) {
addOrRemoveMetric(fs, NodesStatsRequest.Metric.FS);
return this;
}
/**
* Should the node Transport stats be returned.
*/
public NodesStatsRequestBuilder setTransport(boolean transport) {
addOrRemoveMetric(transport, NodesStatsRequest.Metric.TRANSPORT);
return this;
}
/**
* Should the node HTTP stats be returned.
*/
public NodesStatsRequestBuilder setHttp(boolean http) {
addOrRemoveMetric(http, NodesStatsRequest.Metric.HTTP);
return this;
}
/**
* Should the discovery stats be returned.
*/
public NodesStatsRequestBuilder setDiscovery(boolean discovery) {
addOrRemoveMetric(discovery, NodesStatsRequest.Metric.DISCOVERY);
return this;
}
/**
* Should ingest statistics be returned.
*/
public NodesStatsRequestBuilder setIngest(boolean ingest) {
addOrRemoveMetric(ingest, NodesStatsRequest.Metric.INGEST);
return this;
}
public NodesStatsRequestBuilder setAdaptiveSelection(boolean adaptiveSelection) {
addOrRemoveMetric(adaptiveSelection, NodesStatsRequest.Metric.ADAPTIVE_SELECTION);
return this;
}
/**
* Should script context cache statistics be returned
*/
public NodesStatsRequestBuilder setScriptCache(boolean scriptCache) {
addOrRemoveMetric(scriptCache, NodesStatsRequest.Metric.SCRIPT_CACHE);
return this;
}
/**
* Helper method for adding metrics to a request
*/
private void addOrRemoveMetric(boolean includeMetric, NodesStatsRequest.Metric metric) {
if (includeMetric) {
request.addMetric(metric.metricName());
} else {
request.removeMetric(metric.metricName());
}
}
}

View File

@ -41,6 +41,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.OS;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
@ -193,7 +194,7 @@ public class ClusterStatsIT extends ESIntegTestCase {
assertThat(msg, response.nodesStats.getProcess().getMinOpenFileDescriptors(), Matchers.greaterThanOrEqualTo(-1L));
assertThat(msg, response.nodesStats.getProcess().getMaxOpenFileDescriptors(), Matchers.greaterThanOrEqualTo(-1L));
NodesStatsResponse nodesStatsResponse = client().admin().cluster().prepareNodesStats().setOs(true).get();
NodesStatsResponse nodesStatsResponse = client().admin().cluster().prepareNodesStats().addMetric(OS.metricName()).get();
long total = 0;
long free = 0;
long used = 0;

View File

@ -47,6 +47,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.DISCOVERY;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -165,7 +166,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
equalTo(0))); // see https://github.com/elastic/elasticsearch/issues/24388
logger.info("--> request node discovery stats");
NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().setDiscovery(true).get();
NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().addMetric(DISCOVERY.metricName()).get();
assertThat(statsResponse.getNodes().size(), equalTo(1));
DiscoveryStats stats = statsResponse.getNodes().get(0).getDiscoveryStats();

View File

@ -33,6 +33,7 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.SETTINGS;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
@ -89,7 +90,7 @@ public class SettingsFilteringIT extends ESIntegTestCase {
}
public void testNodeInfoIsFiltered() {
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().setSettings(true).get();
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().addMetric(SETTINGS.metricName()).get();
for(NodeInfo info : nodeInfos.getNodes()) {
Settings settings = info.getSettings();
assertNotNull(settings);

View File

@ -110,6 +110,7 @@ import java.util.stream.Stream;
import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiLettersOfLength;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.BREAKER;
import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.NONE;
import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
@ -583,7 +584,7 @@ public class IndexShardIT extends ESSingleNodeTestCase {
CircuitBreaker acctBreaker = breakerService.getBreaker(CircuitBreaker.ACCOUNTING);
long usedMem = acctBreaker.getUsed();
assertThat(usedMem, greaterThan(0L));
NodesStatsResponse response = client().admin().cluster().prepareNodesStats().setIndices(true).setBreaker(true).get();
NodesStatsResponse response = client().admin().cluster().prepareNodesStats().setIndices(true).addMetric(BREAKER.metricName()).get();
NodeStats stats = response.getNodes().get(0);
assertNotNull(stats);
SegmentsStats segmentsStats = stats.getIndices().getSegments();

View File

@ -89,6 +89,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS;
import static org.elasticsearch.common.util.CollectionUtils.iterableAsArrayList;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -599,7 +600,7 @@ public class RemoveCorruptedShardDataCommandIT extends ESIntegTestCase {
}
public static Path getPathToShardData(String nodeId, ShardId shardId, String shardPathSubdirectory) {
final NodesStatsResponse nodeStatsResponse = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get();
final NodesStatsResponse nodeStatsResponse = client().admin().cluster().prepareNodesStats(nodeId).addMetric(FS.metricName()).get();
final Set<Path> paths = StreamSupport.stream(nodeStatsResponse.getNodes().get(0).getFs().spliterator(), false)
.map(nodePath -> PathUtils.get(nodePath.getPath())
.resolve(NodeEnvironment.INDICES_FOLDER)

View File

@ -96,6 +96,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS;
import static org.elasticsearch.common.util.CollectionUtils.iterableAsArrayList;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSuccessful;
@ -649,7 +650,9 @@ public class CorruptedFileIT extends ESIntegTestCase {
assertTrue(shardRouting.primary());
assertTrue(shardRouting.assignedToNode());
String nodeId = shardRouting.currentNodeId();
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get();
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId)
.addMetric(FS.metricName())
.get();
Set<Path> files = new TreeSet<>(); // treeset makes sure iteration order is deterministic
for (FsInfo.Path info : nodeStatses.getNodes().get(0).getFs()) {
String path = info.getPath();
@ -690,7 +693,9 @@ public class CorruptedFileIT extends ESIntegTestCase {
}
public List<Path> listShardFiles(ShardRouting routing) throws IOException {
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(routing.currentNodeId()).setFs(true).get();
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(routing.currentNodeId())
.addMetric(FS.metricName())
.get();
ClusterState state = client().admin().cluster().prepareState().get().getState();
final Index test = state.metadata().index("test").getIndex();
assertThat(routing.toString(), nodeStatses.getNodes().size(), equalTo(1));

View File

@ -58,6 +58,7 @@ import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.BREAKER;
import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.search.aggregations.AggregationBuilders.cardinality;
@ -108,7 +109,9 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
/** Returns true if any of the nodes used a noop breaker */
private boolean noopBreakerUsed() {
NodesStatsResponse stats = client().admin().cluster().prepareNodesStats().setBreaker(true).get();
NodesStatsResponse stats = client().admin().cluster().prepareNodesStats()
.addMetric(BREAKER.metricName())
.get();
for (NodeStats nodeStats : stats.getNodes()) {
if (nodeStats.getBreaker().getStats(CircuitBreaker.REQUEST).getLimit() == NoopCircuitBreaker.LIMIT) {
return true;
@ -159,7 +162,9 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
errMsg = "which is larger than the limit of [100/100b]";
assertFailures(searchRequest, RestStatus.INTERNAL_SERVER_ERROR, containsString(errMsg));
NodesStatsResponse stats = client.admin().cluster().prepareNodesStats().setBreaker(true).get();
NodesStatsResponse stats = client.admin().cluster().prepareNodesStats()
.addMetric(BREAKER.metricName())
.get();
int breaks = 0;
for (NodeStats stat : stats.getNodes()) {
CircuitBreakerStats breakerStats = stat.getBreaker().getStats(CircuitBreaker.FIELDDATA);
@ -211,7 +216,9 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
errMsg = "which is larger than the limit of [100/100b]";
assertFailures(searchRequest, RestStatus.INTERNAL_SERVER_ERROR, containsString(errMsg));
NodesStatsResponse stats = client.admin().cluster().prepareNodesStats().setBreaker(true).get();
NodesStatsResponse stats = client.admin().cluster().prepareNodesStats()
.addMetric(BREAKER.metricName())
.get();
int breaks = 0;
for (NodeStats stat : stats.getNodes()) {
CircuitBreakerStats breakerStats = stat.getBreaker().getStats(CircuitBreaker.FIELDDATA);
@ -298,8 +305,9 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
public void clearFieldData() throws Exception {
client().admin().indices().prepareClearCache().setFieldDataCache(true).execute().actionGet();
assertBusy(() -> {
NodesStatsResponse resp = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).get(new TimeValue(15, TimeUnit.SECONDS));
NodesStatsResponse resp = client().admin().cluster().prepareNodesStats().clear()
.addMetric(BREAKER.metricName())
.get(new TimeValue(15, TimeUnit.SECONDS));
for (NodeStats nStats : resp.getNodes()) {
assertThat("fielddata breaker never reset back to 0",
nStats.getBreaker().getStats(CircuitBreaker.FIELDDATA).getEstimated(),
@ -328,7 +336,9 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
}
}
NodesStatsResponse stats = client().admin().cluster().prepareNodesStats().clear().setBreaker(true).get();
NodesStatsResponse stats = client().admin().cluster().prepareNodesStats().clear()
.addMetric(BREAKER.metricName())
.get();
int breaks = 0;
for (NodeStats stat : stats.getNodes()) {
CircuitBreakerStats breakerStats = stat.getBreaker().getStats(breakerName);

View File

@ -55,6 +55,7 @@ import java.util.List;
import java.util.Random;
import java.util.concurrent.ExecutionException;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.BREAKER;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSuccessful;
import static org.hamcrest.Matchers.equalTo;
@ -73,8 +74,9 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
}
public void testBreakerWithRandomExceptions() throws IOException, InterruptedException, ExecutionException {
for (NodeStats node : client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).execute().actionGet().getNodes()) {
for (NodeStats node : client().admin().cluster().prepareNodesStats().clear()
.addMetric(BREAKER.metricName())
.execute().actionGet().getNodes()) {
assertThat("Breaker is not set to 0", node.getBreaker().getStats(CircuitBreaker.FIELDDATA).getEstimated(), equalTo(0L));
}
@ -156,8 +158,9 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
refreshFailed, refreshResponse.getFailedShards(), refreshResponse.getShardFailures().length,
refreshResponse.getSuccessfulShards(), refreshResponse.getTotalShards());
final int numSearches = scaledRandomIntBetween(50, 150);
NodesStatsResponse resp = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).execute().actionGet();
NodesStatsResponse resp = client().admin().cluster().prepareNodesStats().clear()
.addMetric(BREAKER.metricName())
.execute().actionGet();
for (NodeStats stats : resp.getNodes()) {
assertThat("Breaker is set to 0", stats.getBreaker().getStats(CircuitBreaker.FIELDDATA).getEstimated(), equalTo(0L));
}
@ -192,8 +195,9 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
// Clean up the cache, ensuring that entries' listeners have been called
fdCache.getCache().refresh();
}
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).execute().actionGet();
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().clear()
.addMetric(BREAKER.metricName())
.execute().actionGet();
for (NodeStats stats : nodeStats.getNodes()) {
assertThat("Breaker reset to 0 last search success: " + success + " mapping: " + mapping,
stats.getBreaker().getStats(CircuitBreaker.FIELDDATA).getEstimated(), equalTo(0L));

View File

@ -30,6 +30,7 @@ import org.elasticsearch.test.ESIntegTestCase.Scope;
import java.util.List;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.INDICES;
import static org.elasticsearch.client.Requests.nodesInfoRequest;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
@ -101,7 +102,7 @@ public class SimpleNodesInfoIT extends ESIntegTestCase {
assertThat(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer().getBytes(), greaterThan(0L));
// again, using only the indices flag
response = client().admin().cluster().prepareNodesInfo().clear().setIndices(true).execute().actionGet();
response = client().admin().cluster().prepareNodesInfo().clear().addMetric(INDICES.metricName()).execute().actionGet();
assertThat(response.getNodes().size(), is(2));
assertThat(response.getNodesMap().get(server1NodeId), notNullValue());
assertNotNull(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer());

View File

@ -46,6 +46,9 @@ import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicInteger;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.HTTP;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.SETTINGS;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.BREAKER;
import static org.elasticsearch.test.ESTestCase.getTestTransportType;
import static org.hamcrest.Matchers.equalTo;
import static org.junit.Assert.assertThat;
@ -94,7 +97,9 @@ public final class ExternalTestCluster extends TestCluster {
MockTransportClient client = new MockTransportClient(clientSettings, pluginClasses);
try {
client.addTransportAddresses(transportAddresses);
NodesInfoResponse nodeInfos = client.admin().cluster().prepareNodesInfo().clear().setSettings(true).setHttp(true).get();
NodesInfoResponse nodeInfos = client.admin().cluster().prepareNodesInfo().clear()
.addMetrics(SETTINGS.metricName(), HTTP.metricName())
.get();
httpAddresses = new InetSocketAddress[nodeInfos.getNodes().size()];
this.clusterName = nodeInfos.getClusterName().value();
int dataNodes = 0;
@ -158,8 +163,10 @@ public final class ExternalTestCluster extends TestCluster {
@Override
public void ensureEstimatedStats() {
if (size() > 0) {
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).setIndices(true).execute().actionGet();
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().clear()
.setIndices(true)
.addMetric(BREAKER.metricName())
.execute().actionGet();
for (NodeStats stats : nodeStats.getNodes()) {
assertThat("Fielddata breaker not reset to 0 on node: " + stats.getNode(),
stats.getBreaker().getStats(CircuitBreaker.FIELDDATA).getEstimated(), equalTo(0L));

View File

@ -67,6 +67,7 @@ import java.util.Optional;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.THREAD_POOL;
import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractValue;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -631,7 +632,9 @@ public class MonitoringIT extends ESSingleNodeTestCase {
assertBusy(() -> {
try {
// now wait until Monitoring has actually stopped
final NodesStatsResponse response = client().admin().cluster().prepareNodesStats().clear().setThreadPool(true).get();
final NodesStatsResponse response = client().admin().cluster().prepareNodesStats().clear()
.addMetric(THREAD_POOL.metricName())
.get();
for (final NodeStats nodeStats : response.getNodes()) {
boolean foundBulkThreads = false;

View File

@ -64,6 +64,7 @@ import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.PLUGINS;
import static org.elasticsearch.test.SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout;
@ -225,7 +226,7 @@ public abstract class SecurityIntegTestCase extends ESIntegTestCase {
}
protected void doAssertXPackIsInstalled() {
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().setPlugins(true).get();
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));

View File

@ -39,6 +39,7 @@ import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.PLUGINS;
import static org.elasticsearch.test.SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING;
import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
import static org.hamcrest.core.IsCollectionContaining.hasItem;
@ -149,7 +150,7 @@ public abstract class SecuritySingleNodeTestCase extends ESSingleNodeTestCase {
}
private void doAssertXPackIsInstalled() {
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().setPlugins(true).get();
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));

View File

@ -33,6 +33,7 @@ import java.util.Collections;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric.TRANSPORT;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
@ -86,7 +87,7 @@ public class EllipticCurveSSLTests extends SecurityIntegTestCase {
new TrustManager[]{CertParsingUtils.trustManager(CertParsingUtils.readCertificates(Collections.singletonList(certPath)))},
new SecureRandom());
SSLSocketFactory socketFactory = sslContext.getSocketFactory();
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().setTransport(true).get();
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().addMetric(TRANSPORT.metricName()).get();
TransportAddress address = randomFrom(response.getNodes()).getTransport().getAddress().publishAddress();
final CountDownLatch latch = new CountDownLatch(1);

View File

@ -47,6 +47,8 @@ import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.JVM;
import static org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.THREAD_POOL;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
@ -173,7 +175,9 @@ public class WatcherScheduleEngineBenchmark {
public void run() {
try {
while (start.get()) {
NodesStatsResponse response = client.admin().cluster().prepareNodesStats("_master").setJvm(true).get();
NodesStatsResponse response = client.admin().cluster().prepareNodesStats("_master")
.addMetric(JVM.metricName())
.get();
ByteSizeValue heapUsed = response.getNodes().get(0).getJvm().getMem().getHeapUsed();
jvmUsedHeapSpace.inc(heapUsed.getBytes());
Thread.sleep(1000);
@ -187,7 +191,7 @@ public class WatcherScheduleEngineBenchmark {
start.set(false);
sampleThread.join();
NodesStatsResponse response = client.admin().cluster().prepareNodesStats().setThreadPool(true).get();
NodesStatsResponse response = client.admin().cluster().prepareNodesStats().addMetric(THREAD_POOL.metricName()).get();
for (NodeStats nodeStats : response.getNodes()) {
for (ThreadPoolStats.Stats threadPoolStats : nodeStats.getThreadPool()) {
if ("watcher".equals(threadPoolStats.getName())) {

View File

@ -28,6 +28,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric;
import static org.hamcrest.Matchers.is;
/**
@ -114,7 +115,7 @@ public class CustomRealmIT extends ESIntegTestCase {
}
public void testSettingsFiltering() throws Exception {
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().setSettings(true).get();
NodesInfoResponse nodeInfos = client().admin().cluster().prepareNodesInfo().clear().addMetric(Metric.SETTINGS.metricName()).get();
for(NodeInfo info : nodeInfos.getNodes()) {
Settings settings = info.getSettings();
assertNotNull(settings);

View File

@ -31,6 +31,7 @@ import java.util.List;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest.Metric;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
@ -170,7 +171,7 @@ public class SmokeTestMonitoringWithSecurityIT extends ESIntegTestCase {
}
private String randomNodeHttpAddress() {
List<NodeInfo> nodes = client().admin().cluster().prepareNodesInfo().clear().setHttp(true).get().getNodes();
List<NodeInfo> nodes = client().admin().cluster().prepareNodesInfo().clear().addMetric(Metric.HTTP.metricName()).get().getNodes();
assertThat(nodes.size(), greaterThan(0));
InetSocketAddress[] httpAddresses = new InetSocketAddress[nodes.size()];