Backports the following commits to 7.x: Add telemetry for data tiers (#63031)
This commit is contained in:
parent
6a9cde2918
commit
f0f0da2188
|
@ -154,6 +154,10 @@ Example response:
|
|||
"data_streams" : {
|
||||
"available" : true,
|
||||
"enabled" : true,
|
||||
},
|
||||
"data_tiers" : {
|
||||
"available" : true,
|
||||
"enabled" : true,
|
||||
}
|
||||
},
|
||||
"tagline" : "You know, for X"
|
||||
|
|
|
@ -295,6 +295,58 @@ GET /_xpack/usage
|
|||
"enabled" : true,
|
||||
"data_streams" : 0,
|
||||
"indices_count" : 0
|
||||
},
|
||||
"data_tiers" : {
|
||||
"available" : true,
|
||||
"enabled" : true,
|
||||
"data_warm" : {
|
||||
"node_count" : 0,
|
||||
"index_count" : 0,
|
||||
"total_shard_count" : 0,
|
||||
"primary_shard_count" : 0,
|
||||
"doc_count" : 0,
|
||||
"total_size_bytes" : 0,
|
||||
"primary_size_bytes" : 0,
|
||||
"primary_shard_size_avg_bytes" : 0,
|
||||
"primary_shard_size_median_bytes" : 0,
|
||||
"primary_shard_size_mad_bytes" : 0
|
||||
},
|
||||
"data_cold" : {
|
||||
"node_count" : 0,
|
||||
"index_count" : 0,
|
||||
"total_shard_count" : 0,
|
||||
"primary_shard_count" : 0,
|
||||
"doc_count" : 0,
|
||||
"total_size_bytes" : 0,
|
||||
"primary_size_bytes" : 0,
|
||||
"primary_shard_size_avg_bytes" : 0,
|
||||
"primary_shard_size_median_bytes" : 0,
|
||||
"primary_shard_size_mad_bytes" : 0
|
||||
},
|
||||
"data_content" : {
|
||||
"node_count" : 0,
|
||||
"index_count" : 0,
|
||||
"total_shard_count" : 0,
|
||||
"primary_shard_count" : 0,
|
||||
"doc_count" : 0,
|
||||
"total_size_bytes" : 0,
|
||||
"primary_size_bytes" : 0,
|
||||
"primary_shard_size_avg_bytes" : 0,
|
||||
"primary_shard_size_median_bytes" : 0,
|
||||
"primary_shard_size_mad_bytes" : 0
|
||||
},
|
||||
"data_hot" : {
|
||||
"node_count" : 0,
|
||||
"index_count" : 0,
|
||||
"total_shard_count" : 0,
|
||||
"primary_shard_count" : 0,
|
||||
"doc_count" : 0,
|
||||
"total_size_bytes" : 0,
|
||||
"primary_size_bytes" : 0,
|
||||
"primary_shard_size_avg_bytes" : 0,
|
||||
"primary_shard_size_median_bytes" : 0,
|
||||
"primary_shard_size_mad_bytes" : 0
|
||||
}
|
||||
}
|
||||
}
|
||||
------------------------------------------------------------
|
||||
|
|
|
@ -13,16 +13,23 @@ import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
|
|||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.cluster.metadata.Template;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.xpack.core.DataTier;
|
||||
import org.elasticsearch.xpack.core.DataTiersFeatureSetUsage;
|
||||
import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
|
||||
import org.elasticsearch.xpack.core.XPackFeatureSet;
|
||||
import org.elasticsearch.xpack.core.action.XPackUsageRequestBuilder;
|
||||
import org.elasticsearch.xpack.core.action.XPackUsageResponse;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0, transportClientRatio = 0)
|
||||
public class DataTierIT extends ESIntegTestCase {
|
||||
|
@ -194,6 +201,64 @@ public class DataTierIT extends ESIntegTestCase {
|
|||
ensureYellow(index);
|
||||
}
|
||||
|
||||
public void testDataTierTelemetry() {
|
||||
startContentOnlyNode();
|
||||
startContentOnlyNode();
|
||||
startHotOnlyNode();
|
||||
|
||||
client().admin().indices().prepareCreate(index)
|
||||
.setSettings(Settings.builder()
|
||||
.put(DataTierAllocationDecider.INDEX_ROUTING_PREFER, "data_hot")
|
||||
.put("index.number_of_shards", 2)
|
||||
.put("index.number_of_replicas", 0))
|
||||
.setWaitForActiveShards(0)
|
||||
.get();
|
||||
|
||||
client().admin().indices().prepareCreate(index + "2")
|
||||
.setSettings(Settings.builder()
|
||||
.put("index.number_of_shards", 1)
|
||||
.put("index.number_of_replicas", 1))
|
||||
.setWaitForActiveShards(0)
|
||||
.get();
|
||||
|
||||
ensureGreen();
|
||||
client().prepareIndex(index, MapperService.SINGLE_MAPPING_NAME).setSource("foo", "bar").get();
|
||||
client().prepareIndex(index + "2", MapperService.SINGLE_MAPPING_NAME).setSource("foo", "bar").get();
|
||||
client().prepareIndex(index + "2", MapperService.SINGLE_MAPPING_NAME).setSource("foo", "bar").get();
|
||||
refresh(index, index + "2");
|
||||
|
||||
DataTiersFeatureSetUsage usage = getUsage();
|
||||
// We can't guarantee that internal indices aren't created, so some of these are >= checks
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).nodeCount, equalTo(2));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).indexCount, greaterThanOrEqualTo(1));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).totalShardCount, greaterThanOrEqualTo(2));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).primaryShardCount, greaterThanOrEqualTo(1));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).docCount, greaterThanOrEqualTo(2L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).primaryByteCount, greaterThanOrEqualTo(1L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).primaryByteCountMedian, greaterThanOrEqualTo(1L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_CONTENT).primaryShardBytesMAD, greaterThanOrEqualTo(0L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).nodeCount, equalTo(1));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).indexCount, greaterThanOrEqualTo(1));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).totalShardCount, greaterThanOrEqualTo(2));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).primaryShardCount, greaterThanOrEqualTo(2));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).docCount, greaterThanOrEqualTo(1L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).primaryByteCount, greaterThanOrEqualTo(1L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).primaryByteCountMedian, greaterThanOrEqualTo(1L));
|
||||
assertThat(usage.getTierStats().get(DataTier.DATA_HOT).primaryShardBytesMAD, greaterThanOrEqualTo(0L));
|
||||
}
|
||||
|
||||
private DataTiersFeatureSetUsage getUsage() {
|
||||
XPackUsageResponse usages = new XPackUsageRequestBuilder(client()).execute().actionGet();
|
||||
XPackFeatureSet.Usage dtUsage = usages.getUsages().stream()
|
||||
.filter(u -> u instanceof DataTiersFeatureSetUsage)
|
||||
.collect(Collectors.toList())
|
||||
.get(0);
|
||||
if (dtUsage == null) {
|
||||
throw new IllegalArgumentException("no data tier usage found");
|
||||
}
|
||||
return (DataTiersFeatureSetUsage) dtUsage;
|
||||
}
|
||||
|
||||
public void startDataNode() {
|
||||
Settings nodeSettings = Settings.builder()
|
||||
.putList("node.roles", Arrays.asList("master", "data", "ingest"))
|
||||
|
|
|
@ -16,6 +16,8 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.index.shard.IndexSettingProvider;
|
||||
import org.elasticsearch.xpack.cluster.routing.allocation.DataTierAllocationDecider;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -34,6 +36,8 @@ public class DataTier {
|
|||
public static final String DATA_WARM = "data_warm";
|
||||
public static final String DATA_COLD = "data_cold";
|
||||
|
||||
public static final Set<String> ALL_DATA_TIERS = new HashSet<>(Arrays.asList(DATA_CONTENT, DATA_HOT, DATA_WARM, DATA_COLD));
|
||||
|
||||
/**
|
||||
* Returns true if the given tier name is a valid tier
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,153 @@
|
|||
/*
|
||||
* 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.core;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
|
||||
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.RoutingNodes;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.store.StoreStats;
|
||||
import org.elasticsearch.search.aggregations.metrics.TDigestState;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class DataTiersFeatureSet implements XPackFeatureSet {
|
||||
|
||||
private final Client client;
|
||||
private final ClusterService clusterService;
|
||||
|
||||
@Inject
|
||||
public DataTiersFeatureSet(Client client, ClusterService clusterService) {
|
||||
this.client = client;
|
||||
this.clusterService = clusterService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return XPackField.DATA_TIERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean available() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean enabled() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> nativeCodeInfo() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void usage(ActionListener<Usage> listener) {
|
||||
final ClusterState state = clusterService.state();
|
||||
client.admin().cluster().prepareNodesStats()
|
||||
.all()
|
||||
.setIndices(CommonStatsFlags.ALL)
|
||||
.execute(ActionListener.wrap(nodesStatsResponse -> {
|
||||
final RoutingNodes routingNodes = state.getRoutingNodes();
|
||||
|
||||
// First separate the nodes into separate tiers, note that nodes *may* be duplicated
|
||||
Map<String, List<NodeStats>> tierSpecificNodeStats = separateTiers(nodesStatsResponse);
|
||||
|
||||
// Generate tier specific stats for the nodes
|
||||
Map<String, DataTiersFeatureSetUsage.TierSpecificStats> tierSpecificStats = tierSpecificNodeStats.entrySet()
|
||||
.stream().collect(Collectors.toMap(Map.Entry::getKey, ns -> calculateStats(ns.getValue(), routingNodes)));
|
||||
|
||||
listener.onResponse(new DataTiersFeatureSetUsage(tierSpecificStats));
|
||||
}, listener::onFailure));
|
||||
}
|
||||
|
||||
// Visible for testing
|
||||
static Map<String, List<NodeStats>> separateTiers(NodesStatsResponse nodesStatsResponse) {
|
||||
Map<String, List<NodeStats>> responses = new HashMap<>();
|
||||
DataTier.ALL_DATA_TIERS.forEach(tier ->
|
||||
responses.put(tier, nodesStatsResponse.getNodes().stream()
|
||||
.filter(stats -> stats.getNode().getRoles().stream()
|
||||
.map(DiscoveryNodeRole::roleName)
|
||||
.anyMatch(rn -> rn.equals(tier)))
|
||||
.collect(Collectors.toList())));
|
||||
return responses;
|
||||
}
|
||||
|
||||
private DataTiersFeatureSetUsage.TierSpecificStats calculateStats(List<NodeStats> nodesStats, RoutingNodes routingNodes) {
|
||||
int nodeCount = 0;
|
||||
int indexCount = 0;
|
||||
int totalShardCount = 0;
|
||||
long totalByteCount = 0;
|
||||
long docCount = 0;
|
||||
final AtomicInteger primaryShardCount = new AtomicInteger(0);
|
||||
final AtomicLong primaryByteCount = new AtomicLong(0);
|
||||
final TDigestState valueSketch = new TDigestState(1000);
|
||||
for (NodeStats nodeStats : nodesStats) {
|
||||
nodeCount++;
|
||||
totalByteCount += nodeStats.getIndices().getStore().getSizeInBytes();
|
||||
docCount += nodeStats.getIndices().getDocs().getCount();
|
||||
String nodeId = nodeStats.getNode().getId();
|
||||
final RoutingNode node = routingNodes.node(nodeId);
|
||||
if (node != null) {
|
||||
totalShardCount += node.shardsWithState(ShardRoutingState.STARTED).size();
|
||||
Set<Index> indicesOnNode = node.shardsWithState(ShardRoutingState.STARTED).stream()
|
||||
.map(ShardRouting::index)
|
||||
.collect(Collectors.toSet());
|
||||
indexCount += indicesOnNode.size();
|
||||
indicesOnNode.forEach(index -> {
|
||||
nodeStats.getIndices().getShardStats(index).stream()
|
||||
.filter(shardStats -> shardStats.getPrimary().getStore() != null)
|
||||
.forEach(shardStats -> {
|
||||
StoreStats primaryStoreStats = shardStats.getPrimary().getStore();
|
||||
// If storeStats is null, it means this is not a replica
|
||||
primaryShardCount.incrementAndGet();
|
||||
long primarySize = primaryStoreStats.getSizeInBytes();
|
||||
primaryByteCount.addAndGet(primarySize);
|
||||
valueSketch.add(primarySize);
|
||||
});
|
||||
});
|
||||
}
|
||||
}
|
||||
long primaryShardSizeMedian = (long) valueSketch.quantile(0.5);
|
||||
long primaryShardSizeMAD = computeMedianAbsoluteDeviation(valueSketch);
|
||||
return new DataTiersFeatureSetUsage.TierSpecificStats(nodeCount, indexCount, totalShardCount, primaryShardCount.get(), docCount,
|
||||
totalByteCount, primaryByteCount.get(), primaryShardSizeMedian, primaryShardSizeMAD);
|
||||
}
|
||||
|
||||
// Visible for testing
|
||||
static long computeMedianAbsoluteDeviation(TDigestState valuesSketch) {
|
||||
if (valuesSketch.size() == 0) {
|
||||
return 0;
|
||||
} else {
|
||||
final double approximateMedian = valuesSketch.quantile(0.5);
|
||||
final TDigestState approximatedDeviationsSketch = new TDigestState(valuesSketch.compression());
|
||||
valuesSketch.centroids().forEach(centroid -> {
|
||||
final double deviation = Math.abs(approximateMedian - centroid.mean());
|
||||
approximatedDeviationsSketch.add(deviation, centroid.count());
|
||||
});
|
||||
|
||||
return (long) approximatedDeviationsSketch.quantile(0.5);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,195 @@
|
|||
/*
|
||||
* 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.core;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Strings;
|
||||
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.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* {@link DataTiersFeatureSetUsage} represents the xpack usage for data tiers.
|
||||
* This includes things like the number of nodes per tier, indices, shards, etc.
|
||||
* See {@link TierSpecificStats} for the stats that are tracked on a per-tier
|
||||
* basis.
|
||||
*/
|
||||
public class DataTiersFeatureSetUsage extends XPackFeatureSet.Usage {
|
||||
private final Map<String, TierSpecificStats> tierStats;
|
||||
|
||||
public DataTiersFeatureSetUsage(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
this.tierStats = in.readMap(StreamInput::readString, TierSpecificStats::new);
|
||||
}
|
||||
|
||||
public DataTiersFeatureSetUsage(Map<String, TierSpecificStats> tierStats) {
|
||||
super(XPackField.DATA_TIERS, true, true);
|
||||
this.tierStats = tierStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Version getMinimalSupportedVersion() {
|
||||
return Version.V_7_10_0;
|
||||
}
|
||||
|
||||
public Map<String, TierSpecificStats> getTierStats() {
|
||||
return Collections.unmodifiableMap(tierStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeMap(tierStats, StreamOutput::writeString, (o, v) -> v.writeTo(o));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void innerXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
super.innerXContent(builder, params);
|
||||
for (Map.Entry<String, TierSpecificStats> tierStats : tierStats.entrySet()) {
|
||||
builder.field(tierStats.getKey(), tierStats.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(tierStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DataTiersFeatureSetUsage other = (DataTiersFeatureSetUsage) obj;
|
||||
return Objects.equals(available, other.available) &&
|
||||
Objects.equals(enabled, other.enabled) &&
|
||||
Objects.equals(tierStats, other.tierStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link TierSpecificStats} represents statistics about nodes in a single
|
||||
* tier, for example, how many nodes there are, the index count, shard
|
||||
* count, etc.
|
||||
*/
|
||||
public static class TierSpecificStats implements Writeable, ToXContentObject {
|
||||
|
||||
public final int nodeCount;
|
||||
public final int indexCount;
|
||||
public final int totalShardCount;
|
||||
public final int primaryShardCount;
|
||||
public final long docCount;
|
||||
public final long totalByteCount;
|
||||
public final long primaryByteCount;
|
||||
public final long primaryByteCountMedian;
|
||||
public final long primaryShardBytesMAD;
|
||||
|
||||
public TierSpecificStats(StreamInput in) throws IOException {
|
||||
this.nodeCount = in.readVInt();
|
||||
this.indexCount = in.readVInt();
|
||||
this.totalShardCount = in.readVInt();
|
||||
this.primaryShardCount = in.readVInt();
|
||||
this.docCount = in.readVLong();
|
||||
this.totalByteCount = in.readVLong();
|
||||
this.primaryByteCount = in.readVLong();
|
||||
this.primaryByteCountMedian = in.readVLong();
|
||||
this.primaryShardBytesMAD = in.readVLong();
|
||||
}
|
||||
|
||||
public TierSpecificStats(int nodeCount, int indexCount, int totalShardCount, int primaryShardCount, long docCount,
|
||||
long totalByteCount, long primaryByteCount, long primaryByteCountMedian, long primaryShardBytesMAD) {
|
||||
this.nodeCount = nodeCount;
|
||||
this.indexCount = indexCount;
|
||||
this.totalShardCount = totalShardCount;
|
||||
this.primaryShardCount = primaryShardCount;
|
||||
this.docCount = docCount;
|
||||
this.totalByteCount = totalByteCount;
|
||||
this.primaryByteCount = primaryByteCount;
|
||||
this.primaryByteCountMedian = primaryByteCountMedian;
|
||||
this.primaryShardBytesMAD = primaryShardBytesMAD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeVInt(this.nodeCount);
|
||||
out.writeVInt(this.indexCount);
|
||||
out.writeVInt(this.totalShardCount);
|
||||
out.writeVInt(this.primaryShardCount);
|
||||
out.writeVLong(this.docCount);
|
||||
out.writeVLong(this.totalByteCount);
|
||||
out.writeVLong(this.primaryByteCount);
|
||||
out.writeVLong(this.primaryByteCountMedian);
|
||||
out.writeVLong(this.primaryShardBytesMAD);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field("node_count", nodeCount);
|
||||
builder.field("index_count", indexCount);
|
||||
builder.field("total_shard_count", totalShardCount);
|
||||
builder.field("primary_shard_count", primaryShardCount);
|
||||
builder.field("doc_count", docCount);
|
||||
builder.humanReadableField("total_size_bytes", "total_size", new ByteSizeValue(totalByteCount));
|
||||
builder.humanReadableField("primary_size_bytes", "primary_size", new ByteSizeValue(primaryByteCount));
|
||||
builder.humanReadableField("primary_shard_size_avg_bytes", "primary_shard_size_avg",
|
||||
new ByteSizeValue(primaryShardCount == 0 ? 0 : (primaryByteCount / primaryShardCount)));
|
||||
builder.humanReadableField("primary_shard_size_median_bytes", "primary_shard_size_median",
|
||||
new ByteSizeValue(primaryByteCountMedian));
|
||||
builder.humanReadableField("primary_shard_size_mad_bytes", "primary_shard_size_mad",
|
||||
new ByteSizeValue(primaryShardBytesMAD));
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(this.nodeCount, this.indexCount, this.totalShardCount, this.primaryShardCount, this.totalByteCount,
|
||||
this.primaryByteCount, this.docCount, this.primaryByteCountMedian, this.primaryShardBytesMAD);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
TierSpecificStats other = (TierSpecificStats) obj;
|
||||
return nodeCount == other.nodeCount &&
|
||||
indexCount == other.indexCount &&
|
||||
totalShardCount == other.totalShardCount &&
|
||||
primaryShardCount == other.primaryShardCount &&
|
||||
docCount == other.docCount &&
|
||||
totalByteCount == other.totalByteCount &&
|
||||
primaryByteCount == other.primaryByteCount &&
|
||||
primaryByteCountMedian == other.primaryByteCountMedian &&
|
||||
primaryShardBytesMAD == other.primaryShardBytesMAD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -644,7 +644,9 @@ public class XPackClientPlugin extends Plugin implements ActionPlugin, NetworkPl
|
|||
new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.SEARCHABLE_SNAPSHOTS,
|
||||
SearchableSnapshotFeatureSetUsage::new),
|
||||
// Data Streams
|
||||
new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.DATA_STREAMS, DataStreamFeatureSetUsage::new)
|
||||
new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.DATA_STREAMS, DataStreamFeatureSetUsage::new),
|
||||
// Data Tiers
|
||||
new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.DATA_TIERS, DataTiersFeatureSetUsage::new)
|
||||
).stream(),
|
||||
MlEvaluationNamedXContentProvider.getNamedWriteables().stream()
|
||||
).collect(toList());
|
||||
|
|
|
@ -61,6 +61,8 @@ public final class XPackField {
|
|||
public static final String SEARCHABLE_SNAPSHOTS = "searchable_snapshots";
|
||||
/** Name constant for the data streams feature. */
|
||||
public static final String DATA_STREAMS = "data_streams";
|
||||
/** Name constant for the data tiers feature. */
|
||||
public static final String DATA_TIERS = "data_tiers";
|
||||
|
||||
private XPackField() {}
|
||||
|
||||
|
|
|
@ -268,7 +268,11 @@ public class XPackPlugin extends XPackClientPlugin implements ExtensiblePlugin,
|
|||
|
||||
if (transportClientMode) {
|
||||
modules.add(b -> b.bind(XPackLicenseState.class).toProvider(Providers.of(null)));
|
||||
return modules;
|
||||
}
|
||||
|
||||
modules.add(b -> XPackPlugin.bindFeatureSet(b, DataTiersFeatureSet.class));
|
||||
|
||||
return modules;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* 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.core;
|
||||
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
|
||||
import org.elasticsearch.search.aggregations.metrics.TDigestState;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.containsInAnyOrder;
|
||||
import static org.hamcrest.Matchers.empty;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class DataTiersFeatureSetTests extends ESTestCase {
|
||||
public void testCalculateMAD() {
|
||||
assertThat(DataTiersFeatureSet.computeMedianAbsoluteDeviation(new TDigestState(10)), equalTo(0L));
|
||||
|
||||
TDigestState sketch = new TDigestState(randomDoubleBetween(0, 1000, false));
|
||||
sketch.add(1);
|
||||
sketch.add(1);
|
||||
sketch.add(2);
|
||||
sketch.add(2);
|
||||
sketch.add(4);
|
||||
sketch.add(6);
|
||||
sketch.add(9);
|
||||
assertThat(DataTiersFeatureSet.computeMedianAbsoluteDeviation(sketch), equalTo(1L));
|
||||
}
|
||||
|
||||
public void testSeparateTiers() {
|
||||
NodeStats hotStats = fakeStats(DataTier.DATA_HOT_NODE_ROLE);
|
||||
NodeStats coldStats = fakeStats(DataTier.DATA_COLD_NODE_ROLE);
|
||||
NodeStats warmStats = fakeStats(DataTier.DATA_WARM_NODE_ROLE);
|
||||
NodeStats warmStats2 = fakeStats(DataTier.DATA_WARM_NODE_ROLE);
|
||||
|
||||
NodesStatsResponse nodesStats = new NodesStatsResponse(new ClusterName("cluster"),
|
||||
Arrays.asList(hotStats, coldStats, warmStats, warmStats2), Collections.emptyList());
|
||||
|
||||
Map<String, List<NodeStats>> tiers = DataTiersFeatureSet.separateTiers(nodesStats);
|
||||
assertThat(tiers.keySet(), equalTo(DataTier.ALL_DATA_TIERS));
|
||||
assertThat(tiers.get(DataTier.DATA_CONTENT), empty());
|
||||
assertThat(tiers.get(DataTier.DATA_HOT), containsInAnyOrder(hotStats));
|
||||
assertThat(tiers.get(DataTier.DATA_WARM), containsInAnyOrder(warmStats, warmStats2));
|
||||
assertThat(tiers.get(DataTier.DATA_COLD), containsInAnyOrder(coldStats));
|
||||
}
|
||||
|
||||
private static NodeStats fakeStats(DiscoveryNodeRole role) {
|
||||
NodeStats stats = mock(NodeStats.class);
|
||||
DiscoveryNode node = mock(DiscoveryNode.class);
|
||||
when(node.getRoles()).thenReturn(Collections.singleton(role));
|
||||
when(stats.getNode()).thenReturn(node);
|
||||
return stats;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.core;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class DataTiersFeatureSetUsageTests extends AbstractWireSerializingTestCase<DataTiersFeatureSetUsage> {
|
||||
@Override
|
||||
protected Writeable.Reader<DataTiersFeatureSetUsage> instanceReader() {
|
||||
return DataTiersFeatureSetUsage::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataTiersFeatureSetUsage mutateInstance(DataTiersFeatureSetUsage instance) throws IOException {
|
||||
return randomValueOtherThan(instance, DataTiersFeatureSetUsageTests::randomUsage);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataTiersFeatureSetUsage createTestInstance() {
|
||||
return randomUsage();
|
||||
}
|
||||
|
||||
public static DataTiersFeatureSetUsage randomUsage() {
|
||||
List<String> tiers = randomSubsetOf(DataTier.ALL_DATA_TIERS);
|
||||
Map<String, DataTiersFeatureSetUsage.TierSpecificStats> stats = new HashMap<>();
|
||||
tiers.forEach(tier ->
|
||||
stats.put(tier, new DataTiersFeatureSetUsage.TierSpecificStats(randomIntBetween(1, 10),
|
||||
randomIntBetween(5, 100),
|
||||
randomIntBetween(0, 1000),
|
||||
randomIntBetween(0, 1000),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong(),
|
||||
randomNonNegativeLong())));
|
||||
return new DataTiersFeatureSetUsage(stats);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue