Node Stats: Add fs level stats (size + iostats), closes #1622.

This commit is contained in:
Shay Banon 2012-01-18 21:00:09 +02:00
parent 64358948ef
commit 0a3c941947
18 changed files with 639 additions and 4 deletions

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.http.HttpStats;
import org.elasticsearch.indices.NodeIndicesStats;
import org.elasticsearch.monitor.fs.FsStats;
import org.elasticsearch.monitor.jvm.JvmStats;
import org.elasticsearch.monitor.network.NetworkStats;
import org.elasticsearch.monitor.os.OsStats;
@ -61,6 +62,9 @@ public class NodeStats extends NodeOperationResponse {
@Nullable
private NetworkStats network;
@Nullable
private FsStats fs;
@Nullable
private TransportStats transport;
@ -72,7 +76,7 @@ public class NodeStats extends NodeOperationResponse {
public NodeStats(DiscoveryNode node, @Nullable String hostname, @Nullable NodeIndicesStats indices,
@Nullable OsStats os, @Nullable ProcessStats process, @Nullable JvmStats jvm, @Nullable ThreadPoolStats threadPool, @Nullable NetworkStats network,
@Nullable TransportStats transport, @Nullable HttpStats http) {
@Nullable FsStats fs, @Nullable TransportStats transport, @Nullable HttpStats http) {
super(node);
this.hostname = hostname;
this.indices = indices;
@ -81,6 +85,7 @@ public class NodeStats extends NodeOperationResponse {
this.jvm = jvm;
this.threadPool = threadPool;
this.network = network;
this.fs = fs;
this.transport = transport;
this.http = http;
}
@ -191,6 +196,22 @@ public class NodeStats extends NodeOperationResponse {
return network();
}
/**
* File system level stats.
*/
@Nullable
FsStats fs() {
return fs;
}
/**
* File system level stats.
*/
@Nullable
FsStats getFs() {
return fs();
}
@Nullable
public TransportStats transport() {
return this.transport;
@ -241,6 +262,9 @@ public class NodeStats extends NodeOperationResponse {
if (in.readBoolean()) {
network = NetworkStats.readNetworkStats(in);
}
if (in.readBoolean()) {
fs = FsStats.readFsStats(in);
}
if (in.readBoolean()) {
transport = TransportStats.readTransportStats(in);
}
@ -294,6 +318,12 @@ public class NodeStats extends NodeOperationResponse {
out.writeBoolean(true);
network.writeTo(out);
}
if (fs == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
fs.writeTo(out);
}
if (transport == null) {
out.writeBoolean(false);
} else {

View File

@ -36,6 +36,7 @@ public class NodesStatsRequest extends NodesOperationRequest {
private boolean jvm;
private boolean threadPool;
private boolean network;
private boolean fs;
private boolean transport;
private boolean http;
@ -60,6 +61,7 @@ public class NodesStatsRequest extends NodesOperationRequest {
this.jvm = false;
this.threadPool = false;
this.network = false;
this.fs = false;
this.transport = false;
this.http = false;
return this;
@ -155,6 +157,21 @@ public class NodesStatsRequest extends NodesOperationRequest {
return this;
}
/**
* Should the node file system stats be returned.
*/
public boolean fs() {
return this.fs;
}
/**
* Should the node file system stats be returned.
*/
public NodesStatsRequest fs(boolean fs) {
this.fs = fs;
return this;
}
/**
* Should the node Transport be returned.
*/

View File

@ -93,6 +93,14 @@ public class NodesStatsRequestBuilder extends BaseClusterRequestBuilder<NodesSta
return this;
}
/**
* Should the node file system stats be returned.
*/
public NodesStatsRequestBuilder setFs(boolean fs) {
request.fs(fs);
return this;
}
/**
* Should the node Transport stats be returned.
*/

View File

@ -101,6 +101,9 @@ public class NodesStatsResponse extends NodesOperationResponse<NodeStats> implem
if (nodeStats.network() != null) {
nodeStats.network().toXContent(builder, params);
}
if (nodeStats.fs() != null) {
nodeStats.fs().toXContent(builder, params);
}
if (nodeStats.transport() != null) {
nodeStats.transport().toXContent(builder, params);
}

View File

@ -97,7 +97,7 @@ public class TransportNodesStatsAction extends TransportNodesOperationAction<Nod
@Override
protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) throws ElasticSearchException {
NodesStatsRequest request = nodeStatsRequest.request;
return nodeService.stats(request.indices(), request.os(), request.process(), request.jvm(), request.threadPool(), request.network(), request.transport(), request.http());
return nodeService.stats(request.indices(), request.os(), request.process(), request.jvm(), request.threadPool(), request.network(), request.fs(), request.transport(), request.http());
}
@Override

View File

@ -84,6 +84,11 @@ public abstract class AdapterStreamInput extends StreamInput {
return in.readVLong();
}
@Override
public String readOptionalUTF() throws IOException {
return in.readOptionalUTF();
}
@Override
public String readUTF() throws IOException {
return in.readUTF();

View File

@ -107,6 +107,11 @@ public class AdapterStreamOutput extends StreamOutput {
out.writeVLong(i);
}
@Override
public void writeOptionalUTF(@Nullable String str) throws IOException {
super.writeOptionalUTF(str);
}
@Override
public void writeUTF(String str) throws IOException {
out.writeUTF(str);

View File

@ -147,6 +147,14 @@ public abstract class StreamInput extends InputStream {
return i | ((b & 0x7FL) << 56);
}
@Nullable
public String readOptionalUTF() throws IOException {
if (readBoolean()) {
return readUTF();
}
return null;
}
public String readUTF() throws IOException {
int charCount = readVInt();
char[] chars = CachedStreamInput.getCharArray(charCount);

View File

@ -129,6 +129,15 @@ public abstract class StreamOutput extends OutputStream {
writeByte((byte) i);
}
public void writeOptionalUTF(@Nullable String str) throws IOException {
if (str == null) {
writeBoolean(false);
} else {
writeBoolean(true);
writeUTF(str);
}
}
/**
* Writes a string.
*/

View File

@ -31,6 +31,10 @@ import org.elasticsearch.monitor.dump.cluster.ClusterDumpContributor;
import org.elasticsearch.monitor.dump.heap.HeapDumpContributor;
import org.elasticsearch.monitor.dump.summary.SummaryDumpContributor;
import org.elasticsearch.monitor.dump.thread.ThreadDumpContributor;
import org.elasticsearch.monitor.fs.FsProbe;
import org.elasticsearch.monitor.fs.FsService;
import org.elasticsearch.monitor.fs.JmxFsProbe;
import org.elasticsearch.monitor.fs.SigarFsProbe;
import org.elasticsearch.monitor.jvm.JvmMonitorService;
import org.elasticsearch.monitor.jvm.JvmService;
import org.elasticsearch.monitor.network.JmxNetworkProbe;
@ -80,6 +84,7 @@ public class MonitorModule extends AbstractModule {
bind(ProcessProbe.class).to(SigarProcessProbe.class).asEagerSingleton();
bind(OsProbe.class).to(SigarOsProbe.class).asEagerSingleton();
bind(NetworkProbe.class).to(SigarNetworkProbe.class).asEagerSingleton();
bind(FsProbe.class).to(SigarFsProbe.class).asEagerSingleton();
sigarLoaded = true;
}
} catch (Throwable e) {
@ -91,12 +96,14 @@ public class MonitorModule extends AbstractModule {
bind(ProcessProbe.class).to(JmxProcessProbe.class).asEagerSingleton();
bind(OsProbe.class).to(JmxOsProbe.class).asEagerSingleton();
bind(NetworkProbe.class).to(JmxNetworkProbe.class).asEagerSingleton();
bind(FsProbe.class).to(JmxFsProbe.class).asEagerSingleton();
}
// bind other services
bind(ProcessService.class).asEagerSingleton();
bind(OsService.class).asEagerSingleton();
bind(NetworkService.class).asEagerSingleton();
bind(JvmService.class).asEagerSingleton();
bind(FsService.class).asEagerSingleton();
bind(JvmMonitorService.class).asEagerSingleton();

View File

@ -23,6 +23,7 @@ import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.monitor.fs.FsService;
import org.elasticsearch.monitor.jvm.JvmMonitorService;
import org.elasticsearch.monitor.jvm.JvmService;
import org.elasticsearch.monitor.network.NetworkService;
@ -44,15 +45,19 @@ public class MonitorService extends AbstractLifecycleComponent<MonitorService> {
private final NetworkService networkService;
private final FsService fsService;
@Inject
public MonitorService(Settings settings, JvmMonitorService jvmMonitorService,
OsService osService, ProcessService processService, JvmService jvmService, NetworkService networkService) {
OsService osService, ProcessService processService, JvmService jvmService, NetworkService networkService,
FsService fsService) {
super(settings);
this.jvmMonitorService = jvmMonitorService;
this.osService = osService;
this.processService = processService;
this.jvmService = jvmService;
this.networkService = networkService;
this.fsService = fsService;
}
public OsService osService() {
@ -71,6 +76,10 @@ public class MonitorService extends AbstractLifecycleComponent<MonitorService> {
return this.networkService;
}
public FsService fsService() {
return this.fsService;
}
@Override
protected void doStart() throws ElasticSearchException {
jvmMonitorService.start();

View File

@ -0,0 +1,27 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.monitor.fs;
/**
*/
public interface FsProbe {
FsStats stats();
}

View File

@ -0,0 +1,55 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.monitor.fs;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
/**
*/
public class FsService extends AbstractComponent {
private final FsProbe probe;
private final TimeValue refreshInterval;
private FsStats cachedStats;
@Inject
public FsService(Settings settings, FsProbe probe) {
super(settings);
this.probe = probe;
this.cachedStats = probe.stats();
this.refreshInterval = componentSettings.getAsTime("refresh_interval", TimeValue.timeValueSeconds(1));
logger.debug("Using probe [{}] with refresh_interval [{}]", probe, refreshInterval);
}
public synchronized FsStats stats() {
if ((System.currentTimeMillis() - cachedStats.timestamp()) > refreshInterval.millis()) {
cachedStats = probe.stats();
}
return cachedStats;
}
}

View File

@ -0,0 +1,284 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.monitor.fs;
import com.google.common.collect.Iterators;
import org.elasticsearch.common.Nullable;
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.Streamable;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Iterator;
/**
*/
public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
public static class Info implements Streamable {
String path;
@Nullable
String mount;
@Nullable
String dev;
long total = -1;
long free = -1;
long available = -1;
long diskReads = -1;
long diskWrites = -1;
long diskReadBytes = -1;
long diskWriteBytes = -1;
double diskQueue = -1;
double diskServiceTime = -1;
@Override
public void readFrom(StreamInput in) throws IOException {
path = in.readUTF();
mount = in.readOptionalUTF();
dev = in.readOptionalUTF();
total = in.readLong();
free = in.readLong();
available = in.readLong();
diskReads = in.readLong();
diskWrites = in.readLong();
diskReadBytes = in.readLong();
diskWriteBytes = in.readLong();
diskQueue = in.readDouble();
diskServiceTime = in.readDouble();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeUTF(path);
out.writeOptionalUTF(mount);
out.writeOptionalUTF(dev);
out.writeLong(total);
out.writeLong(free);
out.writeLong(available);
out.writeLong(diskReads);
out.writeLong(diskWrites);
out.writeLong(diskReadBytes);
out.writeLong(diskWriteBytes);
out.writeDouble(diskQueue);
out.writeDouble(diskServiceTime);
}
public ByteSizeValue total() {
return new ByteSizeValue(total);
}
public ByteSizeValue getTotal() {
return total();
}
public ByteSizeValue free() {
return new ByteSizeValue(free);
}
public ByteSizeValue getFree() {
return free();
}
public ByteSizeValue available() {
return new ByteSizeValue(available);
}
public ByteSizeValue getAvailable() {
return available();
}
public long diskReads() {
return this.diskReads;
}
public long getDiskReads() {
return this.diskReads;
}
public long diskWrites() {
return this.diskWrites;
}
public long getDiskWrites() {
return this.diskWrites;
}
public long diskReadSizeInBytes() {
return diskReadBytes;
}
public long getDiskReadSizeInBytes() {
return diskReadBytes;
}
public ByteSizeValue diskReadSizeSize() {
return new ByteSizeValue(diskReadBytes);
}
public ByteSizeValue getDiskReadSizeSize() {
return new ByteSizeValue(diskReadBytes);
}
public long diskWriteSizeInBytes() {
return diskWriteBytes;
}
public long getDiskWriteSizeInBytes() {
return diskWriteBytes;
}
public ByteSizeValue diskWriteSizeSize() {
return new ByteSizeValue(diskWriteBytes);
}
public ByteSizeValue getDiskWriteSizeSize() {
return new ByteSizeValue(diskWriteBytes);
}
public double diskQueue() {
return diskQueue;
}
public double getDiskQueue() {
return diskQueue;
}
public double diskServiceTime() {
return diskServiceTime;
}
public double getDiskServiceTime() {
return diskServiceTime;
}
}
long timestamp;
Info[] infos;
FsStats() {
}
FsStats(long timestamp, Info[] infos) {
this.timestamp = timestamp;
this.infos = infos;
}
public long timestamp() {
return timestamp;
}
public long getTimestamp() {
return timestamp();
}
@Override
public Iterator<Info> iterator() {
return Iterators.forArray(infos);
}
public static FsStats readFsStats(StreamInput in) throws IOException {
FsStats stats = new FsStats();
stats.readFrom(in);
return stats;
}
@Override
public void readFrom(StreamInput in) throws IOException {
timestamp = in.readVLong();
infos = new Info[in.readVInt()];
for (int i = 0; i < infos.length; i++) {
infos[i] = new Info();
infos[i].readFrom(in);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(timestamp);
out.writeVInt(infos.length);
for (Info info : infos) {
info.writeTo(out);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("fs");
builder.field("timestamp", timestamp);
builder.startArray("data");
for (Info info : infos) {
builder.startObject();
builder.field("path", info.path);
if (info.mount != null) {
builder.field("mount", info.mount);
}
if (info.dev != null) {
builder.field("dev", info.dev);
}
if (info.total != -1) {
builder.field("total", info.total().toString());
builder.field("total_in_bytes", info.total);
}
if (info.free != -1) {
builder.field("free", info.free().toString());
builder.field("free_in_bytes", info.free);
}
if (info.available != -1) {
builder.field("available", info.available().toString());
builder.field("available_in_bytes", info.available);
}
if (info.diskReads != -1) {
builder.field("disk_reads", info.diskReads);
}
if (info.diskWrites != -1) {
builder.field("disk_writes", info.diskWrites);
}
if (info.diskReadBytes != -1) {
builder.field("disk_read_size", info.diskReadSizeSize().toString());
builder.field("disk_read_size_bytes", info.diskReadSizeInBytes());
}
if (info.diskWriteBytes != -1) {
builder.field("disk_write_size", info.diskWriteSizeSize().toString());
builder.field("disk_write_size_bytes", info.diskWriteSizeInBytes());
}
if (info.diskQueue != -1) {
builder.field("disk_queue", Strings.format1Decimals(info.diskQueue, ""));
}
if (info.diskServiceTime != -1) {
builder.field("disk_service_time", Strings.format1Decimals(info.diskServiceTime, ""));
}
builder.endObject();
}
builder.endArray();
builder.endObject();
return builder;
}
}

View File

@ -0,0 +1,59 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.monitor.fs;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import java.io.File;
/**
*/
public class JmxFsProbe extends AbstractComponent implements FsProbe {
private final NodeEnvironment nodeEnv;
@Inject
public JmxFsProbe(Settings settings, NodeEnvironment nodeEnv) {
super(settings);
this.nodeEnv = nodeEnv;
}
@Override
public FsStats stats() {
if (!nodeEnv.hasNodeFile()) {
return new FsStats(System.currentTimeMillis(), new FsStats.Info[0]);
}
File[] dataLocations = nodeEnv.nodeDataLocations();
FsStats.Info[] infos = new FsStats.Info[dataLocations.length];
for (int i = 0; i < dataLocations.length; i++) {
File dataLocation = dataLocations[i];
FsStats.Info info = new FsStats.Info();
info.path = dataLocation.getAbsolutePath();
info.total = dataLocation.getTotalSpace();
info.free = dataLocation.getFreeSpace();
info.available = dataLocation.getUsableSpace();
infos[i] = info;
}
return new FsStats(System.currentTimeMillis(), infos);
}
}

View File

@ -0,0 +1,94 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.monitor.fs;
import com.google.common.collect.Maps;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.monitor.sigar.SigarService;
import org.hyperic.sigar.FileSystem;
import org.hyperic.sigar.FileSystemUsage;
import org.hyperic.sigar.SigarException;
import java.io.File;
import java.util.Map;
/**
*/
public class SigarFsProbe extends AbstractComponent implements FsProbe {
private final NodeEnvironment nodeEnv;
private final SigarService sigarService;
private Map<File, FileSystem> fileSystems = Maps.newHashMap();
@Inject
public SigarFsProbe(Settings settings, NodeEnvironment nodeEnv, SigarService sigarService) {
super(settings);
this.nodeEnv = nodeEnv;
this.sigarService = sigarService;
}
@Override
public synchronized FsStats stats() {
if (!nodeEnv.hasNodeFile()) {
return new FsStats(System.currentTimeMillis(), new FsStats.Info[0]);
}
File[] dataLocations = nodeEnv.nodeDataLocations();
FsStats.Info[] infos = new FsStats.Info[dataLocations.length];
for (int i = 0; i < dataLocations.length; i++) {
File dataLocation = dataLocations[i];
FsStats.Info info = new FsStats.Info();
info.path = dataLocation.getAbsolutePath();
try {
FileSystem fileSystem = fileSystems.get(dataLocation);
if (fileSystem == null) {
fileSystem = sigarService.sigar().getFileSystemMap().getMountPoint(dataLocation.getPath());
fileSystems.put(dataLocation, fileSystem);
}
FileSystemUsage fileSystemUsage = sigarService.sigar().getFileSystemUsage(fileSystem.getDirName());
info.mount = fileSystem.getDirName();
info.dev = fileSystem.getDevName();
// total/free/available seem to be in megabytes?
info.total = fileSystemUsage.getTotal() * 1024;
info.free = fileSystemUsage.getFree() * 1024;
info.available = fileSystemUsage.getAvail() * 1024;
info.diskReads = fileSystemUsage.getDiskReads();
info.diskWrites = fileSystemUsage.getDiskWrites();
info.diskReadBytes = fileSystemUsage.getDiskReadBytes();
info.diskWriteBytes = fileSystemUsage.getDiskWriteBytes();
info.diskQueue = fileSystemUsage.getDiskQueue();
info.diskServiceTime = fileSystemUsage.getDiskServiceTime();
} catch (SigarException e) {
// failed...
}
infos[i] = info;
}
return new FsStats(System.currentTimeMillis(), infos);
}
}

View File

@ -140,12 +140,13 @@ public class NodeService extends AbstractComponent {
monitorService.jvmService().stats(),
threadPool.stats(),
monitorService.networkService().stats(),
monitorService.fsService().stats(),
transportService.stats(),
httpServer == null ? null : httpServer.stats()
);
}
public NodeStats stats(boolean indices, boolean os, boolean process, boolean jvm, boolean threadPool, boolean network, boolean transport, boolean http) {
public NodeStats stats(boolean indices, boolean os, boolean process, boolean jvm, boolean threadPool, boolean network, boolean fs, boolean transport, boolean http) {
// for indices stats we want to include previous allocated shards stats as well (it will
// only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
return new NodeStats(clusterService.state().nodes().localNode(), hostname,
@ -155,6 +156,7 @@ public class NodeService extends AbstractComponent {
jvm ? monitorService.jvmService().stats() : null,
threadPool ? this.threadPool.stats() : null,
network ? monitorService.networkService().stats() : null,
fs ? monitorService.fsService().stats() : null,
transport ? transportService.stats() : null,
http ? (httpServer == null ? null : httpServer.stats()) : null
);

View File

@ -64,6 +64,9 @@ public class RestNodesStatsAction extends BaseRestHandler {
controller.registerHandler(RestRequest.Method.GET, "/_nodes/stats/network", new RestNetworkHandler());
controller.registerHandler(RestRequest.Method.GET, "/_nodes/{nodeId}/stats/network", new RestNetworkHandler());
controller.registerHandler(RestRequest.Method.GET, "/_nodes/stats/fs", new RestFsHandler());
controller.registerHandler(RestRequest.Method.GET, "/_nodes/{nodeId}/stats/fs", new RestFsHandler());
controller.registerHandler(RestRequest.Method.GET, "/_nodes/stats/transport", new RestTransportHandler());
controller.registerHandler(RestRequest.Method.GET, "/_nodes/{nodeId}/stats/transport", new RestTransportHandler());
@ -85,6 +88,7 @@ public class RestNodesStatsAction extends BaseRestHandler {
nodesStatsRequest.jvm(request.paramAsBoolean("jvm", nodesStatsRequest.jvm()));
nodesStatsRequest.threadPool(request.paramAsBoolean("thread_pool", nodesStatsRequest.threadPool()));
nodesStatsRequest.network(request.paramAsBoolean("network", nodesStatsRequest.network()));
nodesStatsRequest.fs(request.paramAsBoolean("fs", nodesStatsRequest.fs()));
nodesStatsRequest.transport(request.paramAsBoolean("transport", nodesStatsRequest.transport()));
nodesStatsRequest.http(request.paramAsBoolean("http", nodesStatsRequest.http()));
executeNodeStats(request, channel, nodesStatsRequest);
@ -171,6 +175,15 @@ public class RestNodesStatsAction extends BaseRestHandler {
}
}
class RestFsHandler implements RestHandler {
@Override
public void handleRequest(final RestRequest request, final RestChannel channel) {
NodesStatsRequest nodesStatsRequest = new NodesStatsRequest(RestActions.splitNodes(request.param("nodeId")));
nodesStatsRequest.clear().fs(true);
executeNodeStats(request, channel, nodesStatsRequest);
}
}
class RestTransportHandler implements RestHandler {
@Override
public void handleRequest(final RestRequest request, final RestChannel channel) {