[Monitoring] Remove Specific ClusterStateCollector/Resolver and Tests (elastic/x-pack-elasticsearch#1664)
This removes the Cluster State collector and resolver and moves the collection of the cluster state (and cluster health, which is already included in cluster stats). This makes the tests a little more stable and removes an extra network hop during monitoring data collection. Original commit: elastic/x-pack-elasticsearch@44851d2dd6
This commit is contained in:
parent
3f5ae2d54f
commit
b31c8e2661
|
@ -28,7 +28,6 @@ import org.elasticsearch.xpack.monitoring.action.MonitoringBulkAction;
|
|||
import org.elasticsearch.xpack.monitoring.action.TransportMonitoringBulkAction;
|
||||
import org.elasticsearch.xpack.monitoring.cleaner.CleanerService;
|
||||
import org.elasticsearch.xpack.monitoring.collector.Collector;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateCollector;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStatsCollector;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndexRecoveryCollector;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndexStatsCollector;
|
||||
|
@ -121,7 +120,6 @@ public class Monitoring implements ActionPlugin {
|
|||
collectors.add(new IndicesStatsCollector(settings, clusterService, monitoringSettings, licenseState, client));
|
||||
collectors.add(new IndexStatsCollector(settings, clusterService, monitoringSettings, licenseState, client));
|
||||
collectors.add(new ClusterStatsCollector(settings, clusterService, monitoringSettings, licenseState, client, licenseService));
|
||||
collectors.add(new ClusterStateCollector(settings, clusterService, monitoringSettings, licenseState, client));
|
||||
collectors.add(new ShardsCollector(settings, clusterService, monitoringSettings, licenseState));
|
||||
collectors.add(new NodeStatsCollector(settings, clusterService, monitoringSettings, licenseState, client));
|
||||
collectors.add(new IndexRecoveryCollector(settings, clusterService, monitoringSettings, licenseState, client));
|
||||
|
|
|
@ -1,57 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.monitoring.collector.cluster;
|
||||
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoringSettings;
|
||||
import org.elasticsearch.xpack.monitoring.collector.Collector;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.MonitoringDoc;
|
||||
import org.elasticsearch.xpack.security.InternalClient;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Collector for cluster state.
|
||||
* <p>
|
||||
* This collector runs on the master node only and collects {@link ClusterStateMonitoringDoc}
|
||||
* document at a given frequency.
|
||||
*/
|
||||
public class ClusterStateCollector extends Collector {
|
||||
|
||||
private final Client client;
|
||||
|
||||
public ClusterStateCollector(Settings settings, ClusterService clusterService,
|
||||
MonitoringSettings monitoringSettings,
|
||||
XPackLicenseState licenseState, InternalClient client) {
|
||||
super(settings, "cluster-state", clusterService, monitoringSettings, licenseState);
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean shouldCollect() {
|
||||
return super.shouldCollect() && isLocalNodeMaster();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<MonitoringDoc> doCollect() throws Exception {
|
||||
ClusterState clusterState = clusterService.state();
|
||||
String clusterUUID = clusterState.metaData().clusterUUID();
|
||||
long timestamp = System.currentTimeMillis();
|
||||
|
||||
ClusterHealthResponse clusterHealth = client.admin().cluster().prepareHealth().get(monitoringSettings.clusterStateTimeout());
|
||||
|
||||
// Adds a cluster_state document with associated status
|
||||
return Collections.singleton(
|
||||
new ClusterStateMonitoringDoc(monitoringId(), monitoringVersion(), clusterUUID,
|
||||
timestamp, localNode(), clusterState, clusterHealth.getStatus()));
|
||||
}
|
||||
}
|
|
@ -1,39 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.monitoring.collector.cluster;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.MonitoringDoc;
|
||||
|
||||
/**
|
||||
* Monitoring document collected by {@link ClusterStateCollector} that contains the
|
||||
* current cluster state.
|
||||
*/
|
||||
public class ClusterStateMonitoringDoc extends MonitoringDoc {
|
||||
|
||||
public static final String TYPE = "cluster_state";
|
||||
|
||||
private final ClusterState clusterState;
|
||||
private final ClusterHealthStatus status;
|
||||
|
||||
public ClusterStateMonitoringDoc(String monitoringId, String monitoringVersion,
|
||||
String clusterUUID, long timestamp, DiscoveryNode node,
|
||||
ClusterState clusterState, ClusterHealthStatus status) {
|
||||
super(monitoringId, monitoringVersion, TYPE, null, clusterUUID, timestamp, node);
|
||||
this.clusterState = clusterState;
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public ClusterState getClusterState() {
|
||||
return clusterState;
|
||||
}
|
||||
|
||||
public ClusterHealthStatus getStatus() {
|
||||
return status;
|
||||
}
|
||||
}
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.ElasticsearchSecurityException;
|
|||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
|
@ -75,6 +76,7 @@ public class ClusterStatsCollector extends Collector {
|
|||
final DiscoveryNode sourceNode = localNode();
|
||||
final String clusterName = clusterService.getClusterName().value();
|
||||
final String version = Version.CURRENT.toString();
|
||||
final ClusterState clusterState = clusterService.state();
|
||||
final License license = licenseService.getLicense();
|
||||
final List<XPackFeatureSet.Usage> usage = collect(usageSupplier);
|
||||
|
||||
|
@ -82,7 +84,7 @@ public class ClusterStatsCollector extends Collector {
|
|||
return Collections.singleton(
|
||||
new ClusterStatsMonitoringDoc(monitoringId(), monitoringVersion(),
|
||||
clusterUUID, timestamp, sourceNode, clusterName, version, license, usage,
|
||||
clusterStats));
|
||||
clusterStats, clusterState, clusterStats.getStatus()));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -6,6 +6,8 @@
|
|||
package org.elasticsearch.xpack.monitoring.collector.cluster;
|
||||
|
||||
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.license.License;
|
||||
import org.elasticsearch.xpack.XPackFeatureSet;
|
||||
|
@ -31,18 +33,23 @@ public class ClusterStatsMonitoringDoc extends MonitoringDoc {
|
|||
private final License license;
|
||||
private final List<XPackFeatureSet.Usage> usage;
|
||||
private final ClusterStatsResponse clusterStats;
|
||||
private final ClusterState clusterState;
|
||||
private final ClusterHealthStatus status;
|
||||
|
||||
public ClusterStatsMonitoringDoc(String monitoringId, String monitoringVersion,
|
||||
String clusterUUID, long timestamp, DiscoveryNode node,
|
||||
String clusterName, String version, License license,
|
||||
List<XPackFeatureSet.Usage> usage,
|
||||
ClusterStatsResponse clusterStats) {
|
||||
ClusterStatsResponse clusterStats,
|
||||
ClusterState clusterState, ClusterHealthStatus status) {
|
||||
super(monitoringId, monitoringVersion, TYPE, null, clusterUUID, timestamp, node);
|
||||
this.clusterName = clusterName;
|
||||
this.version = version;
|
||||
this.license = license;
|
||||
this.usage = usage;
|
||||
this.clusterStats = clusterStats;
|
||||
this.clusterState = clusterState;
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public String getClusterName() {
|
||||
|
@ -64,4 +71,13 @@ public class ClusterStatsMonitoringDoc extends MonitoringDoc {
|
|||
public ClusterStatsResponse getClusterStats() {
|
||||
return clusterStats;
|
||||
}
|
||||
|
||||
public ClusterState getClusterState() {
|
||||
return clusterState;
|
||||
}
|
||||
|
||||
public ClusterHealthStatus getStatus() {
|
||||
return status;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -10,7 +10,6 @@ import org.elasticsearch.xpack.monitoring.MonitoredSystem;
|
|||
import org.elasticsearch.xpack.monitoring.action.MonitoringBulkDoc;
|
||||
import org.elasticsearch.xpack.monitoring.action.MonitoringIndex;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStatsMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndexRecoveryMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndexStatsMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndicesStatsMonitoringDoc;
|
||||
|
@ -21,7 +20,6 @@ import org.elasticsearch.xpack.monitoring.exporter.MonitoringDoc;
|
|||
import org.elasticsearch.xpack.monitoring.exporter.MonitoringTemplateUtils;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.bulk.MonitoringBulkTimestampedResolver;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.cluster.ClusterStatsResolver;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.cluster.ClusterStateResolver;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.indices.IndexRecoveryResolver;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.indices.IndexStatsResolver;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.indices.IndicesStatsResolver;
|
||||
|
@ -61,7 +59,6 @@ public class ResolversRegistry implements Iterable<MonitoringIndexNameResolver>
|
|||
* Registers resolvers for elasticsearch documents collected by the monitoring plugin
|
||||
*/
|
||||
private void registerBuiltIn(MonitoredSystem id, Settings settings) {
|
||||
registrations.add(resolveByClass(ClusterStateMonitoringDoc.class, new ClusterStateResolver(id, settings)));
|
||||
registrations.add(resolveByClass(ClusterStatsMonitoringDoc.class, new ClusterStatsResolver(id, settings)));
|
||||
registrations.add(resolveByClass(IndexRecoveryMonitoringDoc.class, new IndexRecoveryResolver(id, settings)));
|
||||
registrations.add(resolveByClass(IndexStatsMonitoringDoc.class, new IndexStatsResolver(id, settings)));
|
||||
|
|
|
@ -1,65 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.monitoring.resolver.cluster;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoredSystem;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.MonitoringIndexNameResolver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
|
||||
public class ClusterStateResolver extends MonitoringIndexNameResolver.Timestamped<ClusterStateMonitoringDoc> {
|
||||
|
||||
public static final String TYPE = "cluster_state";
|
||||
|
||||
static final Set<String> FILTERS;
|
||||
static {
|
||||
Set<String> filters = Sets.newHashSet(
|
||||
"cluster_uuid",
|
||||
"timestamp",
|
||||
"type",
|
||||
"source_node",
|
||||
"cluster_state.version",
|
||||
"cluster_state.master_node",
|
||||
"cluster_state.state_uuid",
|
||||
"cluster_state.status",
|
||||
"cluster_state.nodes");
|
||||
FILTERS = Collections.unmodifiableSet(filters);
|
||||
}
|
||||
|
||||
public ClusterStateResolver(MonitoredSystem id, Settings settings) {
|
||||
super(id, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> filters() {
|
||||
return FILTERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void buildXContent(ClusterStateMonitoringDoc document, XContentBuilder builder, ToXContent.Params params) throws IOException {
|
||||
builder.startObject(Fields.CLUSTER_STATE);
|
||||
ClusterState clusterState = document.getClusterState();
|
||||
if (clusterState != null) {
|
||||
builder.field(Fields.STATUS, document.getStatus().name().toLowerCase(Locale.ROOT));
|
||||
clusterState.toXContent(builder, params);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
static final class Fields {
|
||||
static final String CLUSTER_STATE = TYPE;
|
||||
static final String STATUS = "status";
|
||||
}
|
||||
}
|
|
@ -6,6 +6,7 @@
|
|||
package org.elasticsearch.xpack.monitoring.resolver.cluster;
|
||||
|
||||
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.hash.MessageDigests;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -19,11 +20,20 @@ import org.elasticsearch.xpack.monitoring.resolver.MonitoringIndexNameResolver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
||||
public class ClusterStatsResolver extends MonitoringIndexNameResolver.Timestamped<ClusterStatsMonitoringDoc> {
|
||||
|
||||
private static final ToXContent.MapParams CLUSTER_STATS_PARAMS =
|
||||
new ToXContent.MapParams(
|
||||
Collections.singletonMap("metric",
|
||||
ClusterState.Metric.VERSION + "," +
|
||||
ClusterState.Metric.MASTER_NODE + "," +
|
||||
ClusterState.Metric.NODES));
|
||||
|
||||
public ClusterStatsResolver(MonitoredSystem system, Settings settings) {
|
||||
super(system, settings);
|
||||
}
|
||||
|
@ -52,6 +62,14 @@ public class ClusterStatsResolver extends MonitoringIndexNameResolver.Timestampe
|
|||
builder.endObject();
|
||||
}
|
||||
|
||||
final ClusterState clusterState = document.getClusterState();
|
||||
if (clusterState != null) {
|
||||
builder.startObject("cluster_state");
|
||||
builder.field("status", document.getStatus().name().toLowerCase(Locale.ROOT));
|
||||
clusterState.toXContent(builder, CLUSTER_STATS_PARAMS);
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
final List<XPackFeatureSet.Usage> usages = document.getUsage();
|
||||
if (usages != null) {
|
||||
// in the future we may choose to add other usages under the stack_stats section, but it is only xpack for now
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||
import org.elasticsearch.test.SecuritySettingsSource;
|
||||
import org.elasticsearch.xpack.XPackSettings;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStatsMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndexStatsMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndicesStatsMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.node.NodeStatsMonitoringDoc;
|
||||
|
@ -131,14 +131,10 @@ public class OldMonitoringIndicesBackwardsCompatibilityTests extends AbstractOld
|
|||
search(indexPattern, IndexStatsMonitoringDoc.TYPE, greaterThanOrEqualTo(10L));
|
||||
|
||||
// All the other aliases should have been created by now so we can assert that we have the data we saved in the bwc indexes
|
||||
SearchResponse firstShards =
|
||||
search(indexPattern, ShardMonitoringDoc.TYPE, greaterThanOrEqualTo(10L));
|
||||
SearchResponse firstIndices =
|
||||
search(indexPattern, IndicesStatsMonitoringDoc.TYPE, greaterThanOrEqualTo(3L));
|
||||
SearchResponse firstNode =
|
||||
search(indexPattern, NodeStatsMonitoringDoc.TYPE, greaterThanOrEqualTo(3L));
|
||||
SearchResponse firstState =
|
||||
search(indexPattern, ClusterStateMonitoringDoc.TYPE, greaterThanOrEqualTo(3L));
|
||||
SearchResponse firstShards = search(indexPattern, ShardMonitoringDoc.TYPE, greaterThanOrEqualTo(10L));
|
||||
SearchResponse firstIndices = search(indexPattern, IndicesStatsMonitoringDoc.TYPE, greaterThanOrEqualTo(3L));
|
||||
SearchResponse firstNode = search(indexPattern, NodeStatsMonitoringDoc.TYPE, greaterThanOrEqualTo(3L));
|
||||
SearchResponse firstState = search(indexPattern, ClusterStatsMonitoringDoc.TYPE, greaterThanOrEqualTo(3L));
|
||||
|
||||
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get();
|
||||
final String masterNodeId = clusterStateResponse.getState().getNodes().getMasterNodeId();
|
||||
|
@ -164,7 +160,7 @@ public class OldMonitoringIndicesBackwardsCompatibilityTests extends AbstractOld
|
|||
greaterThan(firstIndices.getHits().getTotalHits())), 1, TimeUnit.MINUTES);
|
||||
assertBusy(() -> search(indexPattern, NodeStatsMonitoringDoc.TYPE,
|
||||
greaterThan(firstNode.getHits().getTotalHits())), 1, TimeUnit.MINUTES);
|
||||
assertBusy(() -> search(indexPattern, ClusterStateMonitoringDoc.TYPE,
|
||||
assertBusy(() -> search(indexPattern, ClusterStatsMonitoringDoc.TYPE,
|
||||
greaterThan(firstState.getHits().getTotalHits())), 1, TimeUnit.MINUTES);
|
||||
|
||||
} finally {
|
||||
|
|
|
@ -1,144 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.monitoring.collector.cluster;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoredSystem;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoringSettings;
|
||||
import org.elasticsearch.xpack.monitoring.collector.AbstractCollectorTestCase;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.MonitoringDoc;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
public class ClusterStateCollectorTests extends AbstractCollectorTestCase {
|
||||
|
||||
public void testClusterStateCollectorNoIndices() throws Exception {
|
||||
// waits for pending tasks before collecing cluster state.
|
||||
// prevents the collector to read an older cluster state than the one used in assert later
|
||||
ensureGreen();
|
||||
assertMonitoringDocs(newClusterStateCollector().doCollect(), 0);
|
||||
}
|
||||
|
||||
public void testClusterStateCollectorOneIndex() throws Exception {
|
||||
int nbShards = randomIntBetween(1, 5);
|
||||
assertAcked(prepareCreate("test").setSettings(Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, nbShards)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||
.build()));
|
||||
|
||||
int nbDocs = randomIntBetween(1, 20);
|
||||
for (int i = 0; i < nbDocs; i++) {
|
||||
client().prepareIndex("test", "test").setSource("num", i).get();
|
||||
}
|
||||
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
assertHitCount(client().prepareSearch().setSize(0).get(), nbDocs);
|
||||
assertMonitoringDocs(newClusterStateCollector().doCollect(), nbShards);
|
||||
}
|
||||
|
||||
public void testClusterStateCollectorMultipleIndices() throws Exception {
|
||||
int nbIndices = randomIntBetween(1, 5);
|
||||
int[] docsPerIndex = new int[nbIndices];
|
||||
int[] shardsPerIndex = new int[nbIndices];
|
||||
int nbShards = 0;
|
||||
|
||||
for (int i = 0; i < nbIndices; i++) {
|
||||
shardsPerIndex[i] = randomIntBetween(1, 5);
|
||||
nbShards += shardsPerIndex[i];
|
||||
assertAcked(prepareCreate("test-" + i).setSettings(Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, shardsPerIndex[i])
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||
.build()));
|
||||
|
||||
docsPerIndex[i] = randomIntBetween(1, 20);
|
||||
for (int j = 0; j < docsPerIndex[i]; j++) {
|
||||
client().prepareIndex("test-" + i, "test").setSource("num", i).get();
|
||||
}
|
||||
}
|
||||
|
||||
refresh();
|
||||
|
||||
for (int i = 0; i < nbIndices; i++) {
|
||||
assertHitCount(client().prepareSearch("test-" + i).setSize(0).get(), docsPerIndex[i]);
|
||||
}
|
||||
|
||||
Collection<MonitoringDoc> results = newClusterStateCollector().doCollect();
|
||||
assertMonitoringDocs(results, nbShards);
|
||||
|
||||
MonitoringDoc monitoringDoc = results.iterator().next();
|
||||
assertNotNull(monitoringDoc);
|
||||
assertThat(monitoringDoc, instanceOf(ClusterStateMonitoringDoc.class));
|
||||
|
||||
ClusterStateMonitoringDoc clusterStateMonitoringDoc = (ClusterStateMonitoringDoc) monitoringDoc;
|
||||
|
||||
assertThat(clusterStateMonitoringDoc.getMonitoringId(), equalTo(MonitoredSystem.ES.getSystem()));
|
||||
assertThat(clusterStateMonitoringDoc.getMonitoringVersion(), equalTo(Version.CURRENT.toString()));
|
||||
assertThat(clusterStateMonitoringDoc.getClusterUUID(),
|
||||
equalTo(client().admin().cluster().prepareState().setMetaData(true).get().getState().metaData().clusterUUID()));
|
||||
assertThat(clusterStateMonitoringDoc.getTimestamp(), greaterThan(0L));
|
||||
assertThat(clusterStateMonitoringDoc.getSourceNode(), notNullValue());
|
||||
assertNotNull(clusterStateMonitoringDoc.getClusterState());
|
||||
|
||||
ClusterState clusterState = clusterStateMonitoringDoc.getClusterState();
|
||||
for (int i = 0; i < nbIndices; i++) {
|
||||
assertThat(clusterState.getRoutingTable().allShards("test-" + i), hasSize(shardsPerIndex[i]));
|
||||
}
|
||||
}
|
||||
|
||||
private ClusterStateCollector newClusterStateCollector() {
|
||||
// This collector runs on master node only
|
||||
return newClusterStateCollector(internalCluster().getMasterName());
|
||||
}
|
||||
|
||||
private ClusterStateCollector newClusterStateCollector(String nodeId) {
|
||||
assertNotNull(nodeId);
|
||||
return new ClusterStateCollector(internalCluster().getInstance(Settings.class, nodeId),
|
||||
internalCluster().getInstance(ClusterService.class, nodeId),
|
||||
internalCluster().getInstance(MonitoringSettings.class, nodeId),
|
||||
internalCluster().getInstance(XPackLicenseState.class, nodeId),
|
||||
securedClient(nodeId));
|
||||
}
|
||||
|
||||
private void assertMonitoringDocs(Collection<MonitoringDoc> results, final int nbShards) {
|
||||
assertThat("expecting 1 document for cluster state", results, hasSize(1));
|
||||
|
||||
final ClusterState clusterState = securedClient().admin().cluster().prepareState().get().getState();
|
||||
final String clusterUUID = clusterState.getMetaData().clusterUUID();
|
||||
|
||||
for (MonitoringDoc doc : results) {
|
||||
assertThat(doc.getMonitoringId(), equalTo(MonitoredSystem.ES.getSystem()));
|
||||
assertThat(doc.getMonitoringVersion(), equalTo(Version.CURRENT.toString()));
|
||||
assertThat(doc.getClusterUUID(), equalTo(clusterUUID));
|
||||
assertThat(doc.getTimestamp(), greaterThan(0L));
|
||||
assertThat(doc.getSourceNode(), notNullValue());
|
||||
assertThat(doc, instanceOf(ClusterStateMonitoringDoc.class));
|
||||
|
||||
if (doc instanceof ClusterStateMonitoringDoc) {
|
||||
ClusterStateMonitoringDoc clusterStateMonitoringDoc = (ClusterStateMonitoringDoc) doc;
|
||||
assertThat(clusterStateMonitoringDoc.getClusterState().getRoutingTable().allShards(), hasSize(nbShards));
|
||||
assertThat(clusterStateMonitoringDoc.getClusterState().getNodes().getSize(), equalTo(internalCluster().size()));
|
||||
|
||||
} else {
|
||||
fail("unknown monitoring document type " + doc);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -16,7 +16,6 @@ import org.elasticsearch.action.bulk.BulkRequest;
|
|||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -37,7 +36,6 @@ import org.elasticsearch.test.http.MockResponse;
|
|||
import org.elasticsearch.test.http.MockWebServer;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoredSystem;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoringSettings;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.collector.indices.IndexRecoveryMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.ClusterAlertsUtil;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.Exporter;
|
||||
|
@ -659,13 +657,7 @@ public class HttpExporterIT extends MonitoringIntegTestCase {
|
|||
long timestamp = System.currentTimeMillis();
|
||||
DiscoveryNode sourceNode = new DiscoveryNode("id", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
|
||||
|
||||
if (randomBoolean()) {
|
||||
return new IndexRecoveryMonitoringDoc(monitoringId, monitoringVersion, clusterUUID,
|
||||
timestamp, sourceNode, new RecoveryResponse());
|
||||
} else {
|
||||
return new ClusterStateMonitoringDoc(monitoringId, monitoringVersion,clusterUUID,
|
||||
timestamp, sourceNode, ClusterState.EMPTY_STATE, ClusterHealthStatus.GREEN);
|
||||
}
|
||||
return new IndexRecoveryMonitoringDoc(monitoringId, monitoringVersion, clusterUUID, timestamp, sourceNode, new RecoveryResponse());
|
||||
}
|
||||
|
||||
private List<MonitoringDoc> newRandomMonitoringDocs(int nb) {
|
||||
|
|
|
@ -148,11 +148,6 @@ public class LocalExporterIntegTests extends LocalExporterIntegTestCase {
|
|||
assertThat(client().admin().indices().prepareExists(".monitoring-*").get().isExists(), is(true));
|
||||
ensureYellow(".monitoring-*");
|
||||
|
||||
assertThat(client().prepareSearch(".monitoring-es-*")
|
||||
.setSize(0)
|
||||
.setQuery(QueryBuilders.termQuery("type", "cluster_state"))
|
||||
.get().getHits().getTotalHits(), greaterThan(0L));
|
||||
|
||||
assertThat(client().prepareSearch(".monitoring-es-*")
|
||||
.setSize(0)
|
||||
.setQuery(QueryBuilders.termQuery("type", "cluster_stats"))
|
||||
|
|
|
@ -1,58 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.monitoring.resolver.cluster;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.MonitoringTemplateUtils;
|
||||
import org.elasticsearch.xpack.monitoring.resolver.MonitoringIndexNameResolverTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class ClusterStateResolverTests extends MonitoringIndexNameResolverTestCase<ClusterStateMonitoringDoc, ClusterStateResolver> {
|
||||
|
||||
@Override
|
||||
protected ClusterStateMonitoringDoc newMonitoringDoc() {
|
||||
DiscoveryNode masterNode = new DiscoveryNode("master", buildNewFakeTransportAddress(),
|
||||
emptyMap(), emptySet(), Version.CURRENT);
|
||||
DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
|
||||
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(masterNode).add(otherNode).masterNodeId(masterNode.getId()).build();
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build();
|
||||
|
||||
ClusterStateMonitoringDoc doc = new ClusterStateMonitoringDoc(randomMonitoringId(),
|
||||
randomAlphaOfLength(2), randomAlphaOfLength(5), 1437580442979L,
|
||||
new DiscoveryNode("id", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT),
|
||||
clusterState, randomFrom(ClusterHealthStatus.values()));
|
||||
|
||||
return doc;
|
||||
}
|
||||
|
||||
public void testClusterStateResolver() throws IOException {
|
||||
ClusterStateMonitoringDoc doc = newMonitoringDoc();
|
||||
|
||||
ClusterStateResolver resolver = newResolver();
|
||||
assertThat(resolver.index(doc), equalTo(".monitoring-es-" + MonitoringTemplateUtils.TEMPLATE_VERSION + "-2015.07.22"));
|
||||
|
||||
assertSource(resolver.source(doc, XContentType.JSON),
|
||||
Sets.newHashSet(
|
||||
"cluster_uuid",
|
||||
"timestamp",
|
||||
"type",
|
||||
"source_node",
|
||||
"cluster_state"), XContentType.JSON);
|
||||
}
|
||||
}
|
|
@ -1,93 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.monitoring.resolver.cluster;
|
||||
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoredSystem;
|
||||
import org.elasticsearch.xpack.monitoring.MonitoringSettings;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateCollector;
|
||||
import org.elasticsearch.xpack.monitoring.collector.cluster.ClusterStateMonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.exporter.MonitoringDoc;
|
||||
import org.elasticsearch.xpack.monitoring.test.MonitoringIntegTestCase;
|
||||
import org.elasticsearch.xpack.security.InternalClient;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
@ESIntegTestCase.ClusterScope(numDataNodes = 1, supportsDedicatedMasters = false)
|
||||
public class ClusterStateTests extends MonitoringIntegTestCase {
|
||||
|
||||
private int randomInt = randomInt();
|
||||
private ThreadPool threadPool = null;
|
||||
|
||||
@Before
|
||||
public void setupThreadPool() {
|
||||
threadPool = new TestThreadPool(getTestName());
|
||||
}
|
||||
|
||||
@After
|
||||
public void removeThreadPool() throws InterruptedException {
|
||||
if (threadPool != null) {
|
||||
terminate(threadPool);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return Settings.builder()
|
||||
.put(super.nodeSettings(nodeOrdinal))
|
||||
.put(MonitoringSettings.INTERVAL.getKey(), "-1")
|
||||
.put("xpack.monitoring.exporters.default_local.type", "local")
|
||||
.put("node.attr.custom", randomInt)
|
||||
.build();
|
||||
}
|
||||
|
||||
public void testClusterState() throws Exception {
|
||||
final String masterNodeName = internalCluster().getMasterName();
|
||||
final MonitoringSettings monitoringSettings = new MonitoringSettings(Settings.EMPTY, clusterService().getClusterSettings());
|
||||
final InternalClient client = new InternalClient(Settings.EMPTY, threadPool, internalCluster().client(masterNodeName));
|
||||
final ClusterStateCollector collector =
|
||||
new ClusterStateCollector(Settings.EMPTY,
|
||||
internalCluster().clusterService(masterNodeName),
|
||||
monitoringSettings, new XPackLicenseState(), client);
|
||||
|
||||
final Collection<MonitoringDoc> monitoringDocs = collector.collect();
|
||||
|
||||
// just one cluster state
|
||||
assertThat(monitoringDocs, hasSize(1));
|
||||
|
||||
// get the cluster state document that we fetched
|
||||
final ClusterStateMonitoringDoc clusterStateDoc = (ClusterStateMonitoringDoc)monitoringDocs.iterator().next();
|
||||
|
||||
assertThat(clusterStateDoc.getClusterState(), notNullValue());
|
||||
assertThat(clusterStateDoc.getStatus(), notNullValue());
|
||||
|
||||
// turn the monitoring doc into JSON
|
||||
final ClusterStateResolver resolver = new ClusterStateResolver(MonitoredSystem.ES, Settings.EMPTY);
|
||||
final BytesReference jsonBytes = resolver.source(clusterStateDoc, XContentType.JSON);
|
||||
|
||||
// parse the JSON to figure out what we just did
|
||||
final Map<String, Object> fields = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, jsonBytes).map();
|
||||
|
||||
// ensure we did what we wanted
|
||||
for (final String filter : ClusterStateResolver.FILTERS) {
|
||||
assertContains(filter, fields);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -8,8 +8,12 @@ package org.elasticsearch.xpack.monitoring.resolver.cluster;
|
|||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
|
@ -25,6 +29,8 @@ import java.util.UUID;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static org.hamcrest.Matchers.startsWith;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class ClusterStatsResolverTests extends MonitoringIndexNameResolverTestCase<ClusterStatsMonitoringDoc, ClusterStatsResolver> {
|
||||
|
||||
|
@ -40,6 +46,15 @@ public class ClusterStatsResolverTests extends MonitoringIndexNameResolverTestCa
|
|||
.issueDate(1437580442979L)
|
||||
.expiryDate(1437580442979L + TimeValue.timeValueHours(2).getMillis());
|
||||
|
||||
final String nodeUuid = "the-master-nodes-uuid";
|
||||
final ClusterName clusterName = ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY);
|
||||
final DiscoveryNodes nodes = mock(DiscoveryNodes.class);
|
||||
final DiscoveryNode node =
|
||||
new DiscoveryNode(nodeUuid, new TransportAddress(TransportAddress.META_ADDRESS, 9300), Version.CURRENT);
|
||||
|
||||
when(nodes.getMasterNodeId()).thenReturn(nodeUuid);
|
||||
when(nodes.iterator()).thenReturn(Collections.singleton(node).iterator());
|
||||
|
||||
return new ClusterStatsMonitoringDoc(randomMonitoringId(),
|
||||
randomAlphaOfLength(2), randomAlphaOfLength(5),
|
||||
Math.abs(randomLong()),
|
||||
|
@ -50,10 +65,12 @@ public class ClusterStatsResolverTests extends MonitoringIndexNameResolverTestCa
|
|||
Collections.singletonList(new MonitoringFeatureSet.Usage(randomBoolean(), randomBoolean(), emptyMap())),
|
||||
new ClusterStatsResponse(
|
||||
Math.abs(randomLong()),
|
||||
ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY),
|
||||
clusterName,
|
||||
Collections.emptyList(),
|
||||
Collections.emptyList())
|
||||
);
|
||||
Collections.emptyList()),
|
||||
new ClusterState(clusterName, randomLong(), "a-real-state-uuid", null, null, nodes, null, null, false),
|
||||
randomFrom(ClusterHealthStatus.values())
|
||||
);
|
||||
} catch (Exception e) {
|
||||
throw new IllegalStateException("Failed to generated random ClusterStatsMonitoringDoc", e);
|
||||
}
|
||||
|
@ -79,6 +96,11 @@ public class ClusterStatsResolverTests extends MonitoringIndexNameResolverTestCa
|
|||
"version",
|
||||
"license",
|
||||
"cluster_stats",
|
||||
"cluster_state.status",
|
||||
"cluster_state.version",
|
||||
"cluster_state.state_uuid",
|
||||
"cluster_state.master_node",
|
||||
"cluster_state.nodes",
|
||||
"stack_stats.xpack"), XContentType.JSON);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import static org.elasticsearch.test.ESIntegTestCase.Scope.TEST;
|
|||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.isEmptyOrNullString;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
@ -55,6 +56,7 @@ public class ClusterStatsTests extends MonitoringIntegTestCase {
|
|||
wipeMonitoringIndices();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testClusterStats() throws Exception {
|
||||
ensureGreen();
|
||||
|
||||
|
@ -135,5 +137,18 @@ public class ClusterStatsTests extends MonitoringIntegTestCase {
|
|||
assertNotNull(monitoring);
|
||||
// we don't make any assumptions about what's in it, only that it's there
|
||||
assertThat(monitoring, instanceOf(Map.class));
|
||||
|
||||
Object clusterState = source.get("cluster_state");
|
||||
assertNotNull(clusterState);
|
||||
assertThat(clusterState, instanceOf(Map.class));
|
||||
|
||||
Map<String, Object> clusterStateMap = (Map<String, Object>)clusterState;
|
||||
|
||||
assertThat(clusterStateMap.keySet(), hasSize(5));
|
||||
assertThat(clusterStateMap.remove("status"), notNullValue());
|
||||
assertThat(clusterStateMap.remove("version"), notNullValue());
|
||||
assertThat(clusterStateMap.remove("state_uuid"), notNullValue());
|
||||
assertThat(clusterStateMap.remove("master_node"), notNullValue());
|
||||
assertThat(clusterStateMap.remove("nodes"), notNullValue());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue