more Writeable in ClusterStatsNodes

This commit is contained in:
javanna 2016-03-24 15:28:10 +01:00 committed by Luca Cavanna
parent 82014ebec3
commit 2c6e78e16c
2 changed files with 190 additions and 198 deletions

View File

@ -28,6 +28,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.InetSocketTransportAddress;
import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
@ -41,42 +42,57 @@ import org.elasticsearch.plugins.PluginInfo;
import java.io.IOException; import java.io.IOException;
import java.net.InetAddress; import java.net.InetAddress;
import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
public class ClusterStatsNodes implements ToXContent, Streamable { public class ClusterStatsNodes implements ToXContent, Writeable<ClusterStatsNodes> {
private Counts counts; private final Counts counts;
private Set<Version> versions; private final Set<Version> versions;
private OsStats os; private final OsStats os;
private ProcessStats process; private final ProcessStats process;
private JvmStats jvm; private final JvmStats jvm;
private FsInfo.Path fs; private final FsInfo.Path fs;
private Set<PluginInfo> plugins; private final Set<PluginInfo> plugins;
private ClusterStatsNodes() { ClusterStatsNodes(StreamInput in) throws IOException {
this.counts = new Counts(in);
int size = in.readVInt();
this.versions = new HashSet<>(size);
for (; size > 0; size--) {
this.versions.add(Version.readVersion(in));
} }
public ClusterStatsNodes(ClusterStatsNodeResponse[] nodeResponses) { this.os = new OsStats(in);
this.counts = new Counts(); this.process = new ProcessStats(in);
this.jvm = new JvmStats(in);
this.fs = FsInfo.Path.readInfoFrom(in);
size = in.readVInt();
this.plugins = new HashSet<>(size);
for (; size > 0; size--) {
this.plugins.add(PluginInfo.readFromStream(in));
}
}
ClusterStatsNodes(ClusterStatsNodeResponse[] nodeResponses) {
this.versions = new HashSet<>(); this.versions = new HashSet<>();
this.os = new OsStats();
this.jvm = new JvmStats();
this.fs = new FsInfo.Path(); this.fs = new FsInfo.Path();
this.plugins = new HashSet<>(); this.plugins = new HashSet<>();
this.process = new ProcessStats();
Set<InetAddress> seenAddresses = new HashSet<>(nodeResponses.length); Set<InetAddress> seenAddresses = new HashSet<>(nodeResponses.length);
List<NodeInfo> nodeInfos = new ArrayList<>();
List<NodeStats> nodeStats = new ArrayList<>();
for (ClusterStatsNodeResponse nodeResponse : nodeResponses) { for (ClusterStatsNodeResponse nodeResponse : nodeResponses) {
nodeInfos.add(nodeResponse.nodeInfo());
counts.addNodeInfo(nodeResponse.nodeInfo()); nodeStats.add(nodeResponse.nodeStats());
versions.add(nodeResponse.nodeInfo().getVersion()); this.versions.add(nodeResponse.nodeInfo().getVersion());
process.addNodeStats(nodeResponse.nodeStats()); this.plugins.addAll(nodeResponse.nodeInfo().getPlugins().getPluginInfos());
jvm.addNodeInfoStats(nodeResponse.nodeInfo(), nodeResponse.nodeStats());
plugins.addAll(nodeResponse.nodeInfo().getPlugins().getPluginInfos());
// now do the stats that should be deduped by hardware (implemented by ip deduping) // now do the stats that should be deduped by hardware (implemented by ip deduping)
TransportAddress publishAddress = nodeResponse.nodeInfo().getTransport().address().publishAddress(); TransportAddress publishAddress = nodeResponse.nodeInfo().getTransport().address().publishAddress();
@ -84,19 +100,19 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
if (publishAddress.uniqueAddressTypeId() == 1) { if (publishAddress.uniqueAddressTypeId() == 1) {
inetAddress = ((InetSocketTransportAddress) publishAddress).address().getAddress(); inetAddress = ((InetSocketTransportAddress) publishAddress).address().getAddress();
} }
if (!seenAddresses.add(inetAddress)) { if (!seenAddresses.add(inetAddress)) {
continue; continue;
} }
os.addNodeInfo(nodeResponse.nodeInfo());
if (nodeResponse.nodeStats().getFs() != null) { if (nodeResponse.nodeStats().getFs() != null) {
fs.add(nodeResponse.nodeStats().getFs().total()); this.fs.add(nodeResponse.nodeStats().getFs().total());
} }
} }
this.counts = new Counts(nodeInfos);
this.os = new OsStats(nodeInfos);
this.process = new ProcessStats(nodeStats);
this.jvm = new JvmStats(nodeInfos, nodeStats);
} }
public Counts getCounts() { public Counts getCounts() {
return this.counts; return this.counts;
} }
@ -127,25 +143,8 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
@Override @Override
public void readFrom(StreamInput in) throws IOException { public ClusterStatsNodes readFrom(StreamInput in) throws IOException {
counts = Counts.readCounts(in); return new ClusterStatsNodes(in);
int size = in.readVInt();
versions = new HashSet<>(size);
for (; size > 0; size--) {
versions.add(Version.readVersion(in));
}
os = OsStats.readOsStats(in);
process = ProcessStats.readStats(in);
jvm = JvmStats.readJvmStats(in);
fs = FsInfo.Path.readInfoFrom(in);
size = in.readVInt();
plugins = new HashSet<>(size);
for (; size > 0; size--) {
plugins.add(PluginInfo.readFromStream(in));
}
} }
@Override @Override
@ -163,12 +162,6 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
} }
} }
public static ClusterStatsNodes readNodeStats(StreamInput in) throws IOException {
ClusterStatsNodes nodeStats = new ClusterStatsNodes();
nodeStats.readFrom(in);
return nodeStats;
}
static final class Fields { static final class Fields {
static final XContentBuilderString COUNT = new XContentBuilderString("count"); static final XContentBuilderString COUNT = new XContentBuilderString("count");
static final XContentBuilderString VERSIONS = new XContentBuilderString("versions"); static final XContentBuilderString VERSIONS = new XContentBuilderString("versions");
@ -214,23 +207,29 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
return builder; return builder;
} }
public static class Counts implements Streamable, ToXContent { public static class Counts implements Writeable<Counts>, ToXContent {
static final String COORDINATING_ONLY = "coordinating_only"; static final String COORDINATING_ONLY = "coordinating_only";
private int total; private final int total;
private Map<String, Integer> roles; private final Map<String, Integer> roles;
Counts() { @SuppressWarnings("unchecked")
roles = new HashMap<>(); private Counts(StreamInput in) throws IOException {
this.total = in.readVInt();
this.roles = (Map<String, Integer>)in.readGenericValue();
}
private Counts(List<NodeInfo> nodeInfos) {
this.roles = new HashMap<>();
for (DiscoveryNode.Role role : DiscoveryNode.Role.values()) { for (DiscoveryNode.Role role : DiscoveryNode.Role.values()) {
roles.put(role.getRoleName(), 0); this.roles.put(role.getRoleName(), 0);
}
roles.put(COORDINATING_ONLY, 0);
} }
this.roles.put(COORDINATING_ONLY, 0);
public void addNodeInfo(NodeInfo nodeInfo) { int total = 0;
for (NodeInfo nodeInfo : nodeInfos) {
total++; total++;
if (nodeInfo.getNode().getRoles().size() == 0) { if (nodeInfo.getNode().getRoles().isEmpty()) {
Integer count = roles.get(COORDINATING_ONLY); Integer count = roles.get(COORDINATING_ONLY);
roles.put(COORDINATING_ONLY, ++count); roles.put(COORDINATING_ONLY, ++count);
} else { } else {
@ -240,6 +239,8 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
} }
} }
} }
this.total = total;
}
public int getTotal() { public int getTotal() {
return total; return total;
@ -249,17 +250,9 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
return roles; return roles;
} }
public static Counts readCounts(StreamInput in) throws IOException {
Counts c = new Counts();
c.readFrom(in);
return c;
}
@Override @Override
@SuppressWarnings("unchecked") public Counts readFrom(StreamInput in) throws IOException {
public void readFrom(StreamInput in) throws IOException { return new Counts(in);
total = in.readVInt();
roles = (Map<String, Integer>)in.readGenericValue();
} }
@Override @Override
@ -282,18 +275,27 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
} }
} }
public static class OsStats implements ToXContent, Streamable { public static class OsStats implements ToXContent, Writeable<OsStats> {
final int availableProcessors;
int availableProcessors; final int allocatedProcessors;
int allocatedProcessors;
long availableMemory;
final ObjectIntHashMap<String> names; final ObjectIntHashMap<String> names;
public OsStats() { @SuppressWarnings("unchecked")
names = new ObjectIntHashMap<>(); private OsStats(StreamInput in) throws IOException {
this.availableProcessors = in.readVInt();
this.allocatedProcessors = in.readVInt();
int size = in.readVInt();
this.names = new ObjectIntHashMap<>();
for (int i = 0; i < size; i++) {
names.addTo(in.readString(), in.readVInt());
}
} }
public void addNodeInfo(NodeInfo nodeInfo) { private OsStats(List<NodeInfo> nodeInfos) {
this.names = new ObjectIntHashMap<>();
int availableProcessors = 0;
int allocatedProcessors = 0;
for (NodeInfo nodeInfo : nodeInfos) {
availableProcessors += nodeInfo.getOs().getAvailableProcessors(); availableProcessors += nodeInfo.getOs().getAvailableProcessors();
allocatedProcessors += nodeInfo.getOs().getAllocatedProcessors(); allocatedProcessors += nodeInfo.getOs().getAllocatedProcessors();
@ -301,6 +303,9 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
names.addTo(nodeInfo.getOs().getName(), 1); names.addTo(nodeInfo.getOs().getName(), 1);
} }
} }
this.availableProcessors = availableProcessors;
this.allocatedProcessors = allocatedProcessors;
}
public int getAvailableProcessors() { public int getAvailableProcessors() {
return availableProcessors; return availableProcessors;
@ -310,27 +315,15 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
return allocatedProcessors; return allocatedProcessors;
} }
public ByteSizeValue getAvailableMemory() {
return new ByteSizeValue(availableMemory);
}
@Override @Override
public void readFrom(StreamInput in) throws IOException { public OsStats readFrom(StreamInput in) throws IOException {
availableProcessors = in.readVInt(); return new OsStats(in);
allocatedProcessors = in.readVInt();
availableMemory = in.readLong();
int size = in.readVInt();
names.clear();
for (int i = 0; i < size; i++) {
names.addTo(in.readString(), in.readVInt());
}
} }
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(availableProcessors); out.writeVInt(availableProcessors);
out.writeVInt(allocatedProcessors); out.writeVInt(allocatedProcessors);
out.writeLong(availableMemory);
out.writeVInt(names.size()); out.writeVInt(names.size());
for (ObjectIntCursor<String> name : names) { for (ObjectIntCursor<String> name : names) {
out.writeString(name.key); out.writeString(name.key);
@ -338,20 +331,11 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
} }
} }
public static OsStats readOsStats(StreamInput in) throws IOException {
OsStats os = new OsStats();
os.readFrom(in);
return os;
}
static final class Fields { static final class Fields {
static final XContentBuilderString AVAILABLE_PROCESSORS = new XContentBuilderString("available_processors"); static final XContentBuilderString AVAILABLE_PROCESSORS = new XContentBuilderString("available_processors");
static final XContentBuilderString ALLOCATED_PROCESSORS = new XContentBuilderString("allocated_processors"); static final XContentBuilderString ALLOCATED_PROCESSORS = new XContentBuilderString("allocated_processors");
static final XContentBuilderString NAME = new XContentBuilderString("name"); static final XContentBuilderString NAME = new XContentBuilderString("name");
static final XContentBuilderString NAMES = new XContentBuilderString("names"); static final XContentBuilderString NAMES = new XContentBuilderString("names");
static final XContentBuilderString MEM = new XContentBuilderString("mem");
static final XContentBuilderString TOTAL = new XContentBuilderString("total");
static final XContentBuilderString TOTAL_IN_BYTES = new XContentBuilderString("total_in_bytes");
static final XContentBuilderString COUNT = new XContentBuilderString("count"); static final XContentBuilderString COUNT = new XContentBuilderString("count");
} }
@ -359,10 +343,6 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field(Fields.AVAILABLE_PROCESSORS, availableProcessors); builder.field(Fields.AVAILABLE_PROCESSORS, availableProcessors);
builder.field(Fields.ALLOCATED_PROCESSORS, allocatedProcessors); builder.field(Fields.ALLOCATED_PROCESSORS, allocatedProcessors);
builder.startObject(Fields.MEM);
builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, availableMemory);
builder.endObject();
builder.startArray(Fields.NAMES); builder.startArray(Fields.NAMES);
for (ObjectIntCursor<String> name : names) { for (ObjectIntCursor<String> name : names) {
builder.startObject(); builder.startObject();
@ -371,22 +351,35 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
builder.endObject(); builder.endObject();
} }
builder.endArray(); builder.endArray();
return builder; return builder;
} }
} }
public static class ProcessStats implements ToXContent, Streamable { public static class ProcessStats implements ToXContent, Writeable<ProcessStats> {
int count; final int count;
int cpuPercent; final int cpuPercent;
long totalOpenFileDescriptors; final long totalOpenFileDescriptors;
final long minOpenFileDescriptors;
final long maxOpenFileDescriptors;
private ProcessStats(StreamInput in) throws IOException {
this.count = in.readVInt();
this.cpuPercent = in.readVInt();
this.totalOpenFileDescriptors = in.readVLong();
this.minOpenFileDescriptors = in.readLong();
this.maxOpenFileDescriptors = in.readLong();
}
private ProcessStats(List<NodeStats> nodeStatsList) {
int count = 0;
int cpuPercent = 0;
long totalOpenFileDescriptors = 0;
long minOpenFileDescriptors = Long.MAX_VALUE; long minOpenFileDescriptors = Long.MAX_VALUE;
long maxOpenFileDescriptors = Long.MIN_VALUE; long maxOpenFileDescriptors = Long.MIN_VALUE;
for (NodeStats nodeStats : nodeStatsList) {
public void addNodeStats(NodeStats nodeStats) {
if (nodeStats.getProcess() == null) { if (nodeStats.getProcess() == null) {
return; continue;
} }
count++; count++;
if (nodeStats.getProcess().getCpu() != null) { if (nodeStats.getProcess().getCpu() != null) {
@ -401,6 +394,12 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
minOpenFileDescriptors = Math.min(minOpenFileDescriptors, fd); minOpenFileDescriptors = Math.min(minOpenFileDescriptors, fd);
maxOpenFileDescriptors = Math.max(maxOpenFileDescriptors, fd); maxOpenFileDescriptors = Math.max(maxOpenFileDescriptors, fd);
} }
this.count = count;
this.cpuPercent = cpuPercent;
this.totalOpenFileDescriptors = totalOpenFileDescriptors;
this.minOpenFileDescriptors = minOpenFileDescriptors;
this.maxOpenFileDescriptors = maxOpenFileDescriptors;
}
/** /**
* Cpu usage in percentages - 100 is 1 core. * Cpu usage in percentages - 100 is 1 core.
@ -431,12 +430,8 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
} }
@Override @Override
public void readFrom(StreamInput in) throws IOException { public ProcessStats readFrom(StreamInput in) throws IOException {
count = in.readVInt(); return new ProcessStats(in);
cpuPercent = in.readVInt();
totalOpenFileDescriptors = in.readVLong();
minOpenFileDescriptors = in.readLong();
maxOpenFileDescriptors = in.readLong();
} }
@Override @Override
@ -448,12 +443,6 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
out.writeLong(maxOpenFileDescriptors); out.writeLong(maxOpenFileDescriptors);
} }
public static ProcessStats readStats(StreamInput in) throws IOException {
ProcessStats cpu = new ProcessStats();
cpu.readFrom(in);
return cpu;
}
static final class Fields { static final class Fields {
static final XContentBuilderString CPU = new XContentBuilderString("cpu"); static final XContentBuilderString CPU = new XContentBuilderString("cpu");
static final XContentBuilderString PERCENT = new XContentBuilderString("percent"); static final XContentBuilderString PERCENT = new XContentBuilderString("percent");
@ -477,20 +466,54 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
} }
} }
public static class JvmStats implements Streamable, ToXContent { public static class JvmStats implements Writeable<JvmStats>, ToXContent {
ObjectIntHashMap<JvmVersion> versions; private final ObjectIntHashMap<JvmVersion> versions;
long threads; private final long threads;
long maxUptime; private final long maxUptime;
long heapUsed; private final long heapUsed;
long heapMax; private final long heapMax;
JvmStats() { private JvmStats(StreamInput in) throws IOException {
versions = new ObjectIntHashMap<>(); int size = in.readVInt();
threads = 0; this.versions = new ObjectIntHashMap<>(size);
maxUptime = 0; for (; size > 0; size--) {
heapMax = 0; this.versions.addTo(JvmVersion.readJvmVersion(in), in.readVInt());
heapUsed = 0; }
this.threads = in.readVLong();
this.maxUptime = in.readVLong();
this.heapUsed = in.readVLong();
this.heapMax = in.readVLong();
}
private JvmStats(List<NodeInfo> nodeInfos, List<NodeStats> nodeStatsList) {
this.versions = new ObjectIntHashMap<>();
long threads = 0;
long maxUptime = 0;
long heapMax = 0;
long heapUsed = 0;
for (NodeInfo nodeInfo : nodeInfos) {
versions.addTo(new JvmVersion(nodeInfo.getJvm()), 1);
}
for (NodeStats nodeStats : nodeStatsList) {
org.elasticsearch.monitor.jvm.JvmStats js = nodeStats.getJvm();
if (js == null) {
continue;
}
if (js.getThreads() != null) {
threads += js.getThreads().getCount();
}
maxUptime = Math.max(maxUptime, js.getUptime().millis());
if (js.getMem() != null) {
heapUsed += js.getMem().getHeapUsed().bytes();
heapMax += js.getMem().getHeapMax().bytes();
}
}
this.threads = threads;
this.maxUptime = maxUptime;
this.heapUsed = heapUsed;
this.heapMax = heapMax;
} }
public ObjectIntHashMap<JvmVersion> getVersions() { public ObjectIntHashMap<JvmVersion> getVersions() {
@ -525,33 +548,9 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
return new ByteSizeValue(heapMax); return new ByteSizeValue(heapMax);
} }
public void addNodeInfoStats(NodeInfo nodeInfo, NodeStats nodeStats) {
versions.addTo(new JvmVersion(nodeInfo.getJvm()), 1);
org.elasticsearch.monitor.jvm.JvmStats js = nodeStats.getJvm();
if (js == null) {
return;
}
if (js.getThreads() != null) {
threads += js.getThreads().getCount();
}
maxUptime = Math.max(maxUptime, js.getUptime().millis());
if (js.getMem() != null) {
heapUsed += js.getMem().getHeapUsed().bytes();
heapMax += js.getMem().getHeapMax().bytes();
}
}
@Override @Override
public void readFrom(StreamInput in) throws IOException { public JvmStats readFrom(StreamInput in) throws IOException {
int size = in.readVInt(); return new JvmStats(in);
versions = new ObjectIntHashMap<>(size);
for (; size > 0; size--) {
versions.addTo(JvmVersion.readJvmVersion(in), in.readVInt());
}
threads = in.readVLong();
maxUptime = in.readVLong();
heapUsed = in.readVLong();
heapMax = in.readVLong();
} }
@Override @Override
@ -561,19 +560,12 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
v.key.writeTo(out); v.key.writeTo(out);
out.writeVInt(v.value); out.writeVInt(v.value);
} }
out.writeVLong(threads); out.writeVLong(threads);
out.writeVLong(maxUptime); out.writeVLong(maxUptime);
out.writeVLong(heapUsed); out.writeVLong(heapUsed);
out.writeVLong(heapMax); out.writeVLong(heapMax);
} }
public static JvmStats readJvmStats(StreamInput in) throws IOException {
JvmStats jvmStats = new JvmStats();
jvmStats.readFrom(in);
return jvmStats;
}
static final class Fields { static final class Fields {
static final XContentBuilderString VERSIONS = new XContentBuilderString("versions"); static final XContentBuilderString VERSIONS = new XContentBuilderString("versions");
static final XContentBuilderString VERSION = new XContentBuilderString("version"); static final XContentBuilderString VERSION = new XContentBuilderString("version");

View File

@ -110,7 +110,7 @@ public class ClusterStatsResponse extends BaseNodesResponse<ClusterStatsNodeResp
status = ClusterHealthStatus.fromValue(in.readByte()); status = ClusterHealthStatus.fromValue(in.readByte());
} }
clusterUUID = in.readString(); clusterUUID = in.readString();
nodesStats = ClusterStatsNodes.readNodeStats(in); nodesStats = new ClusterStatsNodes(in);
indicesStats = ClusterStatsIndices.readIndicesStats(in); indicesStats = ClusterStatsIndices.readIndicesStats(in);
} }