FS Stats: remove sigar specific stats from APIs:

- fs.*.disk_reads
- fs.*.disk_writes
- fs.*.disk_io_op
- fs.*.disk_read_size_in_bytes
- fs.*.disk_write_size_in_bytes
- fs.*.disk_io_size_in_bytes
- fs.*.disk_queue
- fs.*.disk_service_time
This commit is contained in:
Tanguy Leroux 2015-07-06 14:43:58 +02:00
parent 7dbc5c7ab9
commit fbcf4dbbf7
12 changed files with 215 additions and 272 deletions

View File

@ -52,7 +52,7 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
private OsStats os;
private ProcessStats process;
private JvmStats jvm;
private FsStats.Info fs;
private FsStats.Path fs;
private Set<PluginInfo> plugins;
private ClusterStatsNodes() {
@ -63,7 +63,7 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
this.versions = new HashSet<>();
this.os = new OsStats();
this.jvm = new JvmStats();
this.fs = new FsStats.Info();
this.fs = new FsStats.Path();
this.plugins = new HashSet<>();
this.process = new ProcessStats();
@ -116,7 +116,7 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
return jvm;
}
public FsStats.Info getFs() {
public FsStats.Path getFs() {
return fs;
}
@ -138,7 +138,7 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
os = OsStats.readOsStats(in);
process = ProcessStats.readStats(in);
jvm = JvmStats.readJvmStats(in);
fs = FsStats.Info.readInfoFrom(in);
fs = FsStats.Path.readInfoFrom(in);
size = in.readVInt();
plugins = new HashSet<>(size);

View File

@ -320,7 +320,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
long available = 0;
long total = 0;
for (FsStats.Info info : nodeStats.getFs()) {
for (FsStats.Path info : nodeStats.getFs()) {
available += info.getAvailable().bytes();
total += info.getTotal().bytes();
}

View File

@ -42,7 +42,7 @@ import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.monitor.fs.FsStats;
import org.elasticsearch.monitor.fs.JmxFsProbe;
import org.elasticsearch.monitor.fs.FsProbe;
import java.io.Closeable;
import java.io.IOException;
@ -225,38 +225,38 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
spinsDesc = "no";
}
FsStats.Info fsInfo = JmxFsProbe.getFSInfo(nodePath);
FsStats.Path fsPath = FsProbe.getFSInfo(nodePath);
sb.append(", free_space [")
.append(fsInfo.getFree())
.append(fsPath.getFree())
.append("], usable_space [")
.append(fsInfo.getAvailable())
.append(fsPath.getAvailable())
.append("], total_space [")
.append(fsInfo.getTotal())
.append(fsPath.getTotal())
.append("], spins? [")
.append(spinsDesc)
.append("], mount [")
.append(fsInfo.getMount())
.append(fsPath.getMount())
.append("], type [")
.append(fsInfo.getType())
.append(fsPath.getType())
.append(']');
}
logger.debug(sb.toString());
} else if (logger.isInfoEnabled()) {
FsStats.Info totFSInfo = new FsStats.Info();
FsStats.Path totFSPath = new FsStats.Path();
Set<String> allTypes = new HashSet<>();
Set<String> allSpins = new HashSet<>();
Set<String> allMounts = new HashSet<>();
for (NodePath nodePath : nodePaths) {
// TODO: can/should I use the chosen FsProbe instead (i.e. sigar if it's available)?
FsStats.Info fsInfo = JmxFsProbe.getFSInfo(nodePath);
String mount = fsInfo.getMount();
FsStats.Path fsPath = FsProbe.getFSInfo(nodePath);
String mount = fsPath.getMount();
if (allMounts.contains(mount) == false) {
allMounts.add(mount);
String type = fsInfo.getType();
String type = fsPath.getType();
if (type != null) {
allTypes.add(type);
}
Boolean spins = fsInfo.getSpins();
Boolean spins = fsPath.getSpins();
if (spins == null) {
allSpins.add("unknown");
} else if (spins.booleanValue()) {
@ -264,7 +264,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
} else {
allSpins.add("no");
}
totFSInfo.add(fsInfo);
totFSPath.add(fsPath);
}
}
@ -273,8 +273,8 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
"using [%d] data paths, mounts [%s], net usable_space [%s], net total_space [%s], spins? [%s], types [%s]",
nodePaths.length,
allMounts,
totFSInfo.getAvailable(),
totFSInfo.getTotal(),
totFSPath.getAvailable(),
totFSPath.getTotal(),
toString(allSpins),
toString(allTypes)));
}

View File

@ -23,7 +23,6 @@ import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.monitor.fs.FsProbe;
import org.elasticsearch.monitor.fs.FsService;
import org.elasticsearch.monitor.fs.JmxFsProbe;
import org.elasticsearch.monitor.jvm.JvmMonitorService;
import org.elasticsearch.monitor.jvm.JvmService;
import org.elasticsearch.monitor.os.JmxOsProbe;
@ -53,7 +52,7 @@ public class MonitorModule extends AbstractModule {
// bind default implementations
bind(ProcessProbe.class).to(JmxProcessProbe.class).asEagerSingleton();
bind(OsProbe.class).to(JmxOsProbe.class).asEagerSingleton();
bind(FsProbe.class).to(JmxFsProbe.class).asEagerSingleton();
bind(FsProbe.class).asEagerSingleton();
// bind other services
bind(ProcessService.class).asEagerSingleton();

View File

@ -19,11 +19,49 @@
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 org.elasticsearch.env.NodeEnvironment.NodePath;
import java.io.IOException;
/**
*/
public interface FsProbe {
public class FsProbe extends AbstractComponent {
FsStats stats() throws IOException;
private final NodeEnvironment nodeEnv;
@Inject
public FsProbe(Settings settings, NodeEnvironment nodeEnv) {
super(settings);
this.nodeEnv = nodeEnv;
}
public FsStats stats() throws IOException {
if (!nodeEnv.hasNodeFile()) {
return new FsStats(System.currentTimeMillis(), new FsStats.Path[0]);
}
NodePath[] dataLocations = nodeEnv.nodePaths();
FsStats.Path[] paths = new FsStats.Path[dataLocations.length];
for (int i = 0; i < dataLocations.length; i++) {
paths[i] = getFSInfo(dataLocations[i]);
}
return new FsStats(System.currentTimeMillis(), paths);
}
public static FsStats.Path getFSInfo(NodePath nodePath) throws IOException {
FsStats.Path fsPath = new FsStats.Path();
fsPath.path = nodePath.path.toAbsolutePath().toString();
// NOTE: we use already cached (on node startup) FileStore and spins
// since recomputing these once per second (default) could be costly,
// and they should not change:
fsPath.total = nodePath.fileStore.getTotalSpace();
fsPath.free = nodePath.fileStore.getUnallocatedSpace();
fsPath.available = nodePath.fileStore.getUsableSpace();
fsPath.type = nodePath.fileStore.type();
fsPath.mount = nodePath.fileStore.toString();
fsPath.spins = nodePath.spins;
return fsPath;
}
}

View File

@ -21,7 +21,6 @@ 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;
@ -35,54 +34,37 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.Set;
/**
*/
public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
public class FsStats implements Iterable<FsStats.Path>, Streamable, ToXContent {
public static class Info implements Streamable, ToXContent {
public static class Path implements Streamable, ToXContent {
String path;
@Nullable
String mount;
@Nullable
String dev;
/** File system type from {@code java.nio.file.FileStore type()}, if available. */
@Nullable
String type;
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;
/** Uses Lucene's {@code IOUtils.spins} method to try to determine if the device backed by spinning media.
* This is null if we could not determine it, true if it possibly spins, else false. */
Boolean spins = null;
public Info() {
public Path() {
}
public Info(String path, @Nullable String mount, @Nullable String dev, long total, long free, long available, long diskReads,
long diskWrites, long diskReadBytes, long diskWriteBytes, double diskQueue, double diskServiceTime) {
public Path(String path, @Nullable String mount, long total, long free, long available) {
this.path = path;
this.mount = mount;
this.dev = dev;
this.total = total;
this.free = free;
this.available = available;
this.diskReads = diskReads;
this.diskWrites = diskWrites;
this.diskReadBytes = diskReadBytes;
this.diskWriteBytes = diskWriteBytes;
this.diskQueue = diskQueue;
this.diskServiceTime = diskServiceTime;
}
static public Info readInfoFrom(StreamInput in) throws IOException {
Info i = new Info();
static public Path readInfoFrom(StreamInput in) throws IOException {
Path i = new Path();
i.readFrom(in);
return i;
}
@ -91,17 +73,10 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
public void readFrom(StreamInput in) throws IOException {
path = in.readOptionalString();
mount = in.readOptionalString();
dev = in.readOptionalString();
type = in.readOptionalString();
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();
spins = in.readOptionalBoolean();
}
@ -109,17 +84,10 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalString(path); // total aggregates do not have a path
out.writeOptionalString(mount);
out.writeOptionalString(dev);
out.writeOptionalString(type);
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);
out.writeOptionalBoolean(spins);
}
@ -131,10 +99,6 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
return mount;
}
public String getDev() {
return dev;
}
public String getType() {
return type;
}
@ -151,38 +115,6 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
return new ByteSizeValue(available);
}
public long getDiskReads() {
return this.diskReads;
}
public long getDiskWrites() {
return this.diskWrites;
}
public long getDiskReadSizeInBytes() {
return diskReadBytes;
}
public ByteSizeValue getDiskReadSizeSize() {
return new ByteSizeValue(diskReadBytes);
}
public long getDiskWriteSizeInBytes() {
return diskWriteBytes;
}
public ByteSizeValue getDiskWriteSizeSize() {
return new ByteSizeValue(diskWriteBytes);
}
public double getDiskQueue() {
return diskQueue;
}
public double getDiskServiceTime() {
return diskServiceTime;
}
public Boolean getSpins() {
return spins;
}
@ -207,17 +139,11 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
return current + other;
}
public void add(Info info) {
total = addLong(total, info.total);
free = addLong(free, info.free);
available = addLong(available, info.available);
diskReads = addLong(diskReads, info.diskReads);
diskWrites = addLong(diskWrites, info.diskWrites);
diskReadBytes = addLong(diskReadBytes, info.diskReadBytes);
diskWriteBytes = addLong(diskWriteBytes, info.diskWriteBytes);
diskQueue = addDouble(diskQueue, info.diskQueue);
diskServiceTime = addDouble(diskServiceTime, info.diskServiceTime);
if (info.spins != null && info.spins.booleanValue()) {
public void add(Path path) {
total = addLong(total, path.total);
free = addLong(free, path.free);
available = addLong(available, path.available);
if (path.spins != null && path.spins.booleanValue()) {
// Spinning is contagious!
spins = Boolean.TRUE;
}
@ -226,7 +152,6 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
static final class Fields {
static final XContentBuilderString PATH = new XContentBuilderString("path");
static final XContentBuilderString MOUNT = new XContentBuilderString("mount");
static final XContentBuilderString DEV = new XContentBuilderString("dev");
static final XContentBuilderString TYPE = new XContentBuilderString("type");
static final XContentBuilderString TOTAL = new XContentBuilderString("total");
static final XContentBuilderString TOTAL_IN_BYTES = new XContentBuilderString("total_in_bytes");
@ -234,17 +159,6 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
static final XContentBuilderString FREE_IN_BYTES = new XContentBuilderString("free_in_bytes");
static final XContentBuilderString AVAILABLE = new XContentBuilderString("available");
static final XContentBuilderString AVAILABLE_IN_BYTES = new XContentBuilderString("available_in_bytes");
static final XContentBuilderString DISK_READS = new XContentBuilderString("disk_reads");
static final XContentBuilderString DISK_WRITES = new XContentBuilderString("disk_writes");
static final XContentBuilderString DISK_IO_OP = new XContentBuilderString("disk_io_op");
static final XContentBuilderString DISK_READ_SIZE = new XContentBuilderString("disk_read_size");
static final XContentBuilderString DISK_READ_SIZE_IN_BYTES = new XContentBuilderString("disk_read_size_in_bytes");
static final XContentBuilderString DISK_WRITE_SIZE = new XContentBuilderString("disk_write_size");
static final XContentBuilderString DISK_WRITE_SIZE_IN_BYTES = new XContentBuilderString("disk_write_size_in_bytes");
static final XContentBuilderString DISK_IO_SIZE = new XContentBuilderString("disk_io_size");
static final XContentBuilderString DISK_IO_IN_BYTES = new XContentBuilderString("disk_io_size_in_bytes");
static final XContentBuilderString DISK_QUEUE = new XContentBuilderString("disk_queue");
static final XContentBuilderString DISK_SERVICE_TIME = new XContentBuilderString("disk_service_time");
static final XContentBuilderString SPINS = new XContentBuilderString("spins");
}
@ -257,9 +171,6 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
if (mount != null) {
builder.field(Fields.MOUNT, mount, XContentBuilder.FieldCaseConversion.NONE);
}
if (dev != null) {
builder.field(Fields.DEV, dev, XContentBuilder.FieldCaseConversion.NONE);
}
if (type != null) {
builder.field(Fields.TYPE, type, XContentBuilder.FieldCaseConversion.NONE);
}
@ -273,51 +184,6 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
if (available != -1) {
builder.byteSizeField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, available);
}
long iop = -1;
if (diskReads != -1) {
iop = diskReads;
builder.field(Fields.DISK_READS, diskReads);
}
if (diskWrites != -1) {
if (iop != -1) {
iop += diskWrites;
} else {
iop = diskWrites;
}
builder.field(Fields.DISK_WRITES, diskWrites);
}
if (iop != -1) {
builder.field(Fields.DISK_IO_OP, iop);
}
long ioBytes = -1;
if (diskReadBytes != -1) {
ioBytes = diskReadBytes;
builder.byteSizeField(Fields.DISK_READ_SIZE_IN_BYTES, Fields.DISK_READ_SIZE, diskReadBytes);
}
if (diskWriteBytes != -1) {
if (ioBytes != -1) {
ioBytes += diskWriteBytes;
} else {
ioBytes = diskWriteBytes;
}
builder.byteSizeField(Fields.DISK_WRITE_SIZE_IN_BYTES, Fields.DISK_WRITE_SIZE, diskWriteBytes);
}
if (ioBytes != -1) {
builder.byteSizeField(Fields.DISK_IO_IN_BYTES, Fields.DISK_IO_SIZE, ioBytes);
}
if (diskQueue != -1) {
builder.field(Fields.DISK_QUEUE, Strings.format1Decimals(diskQueue, ""));
}
if (diskServiceTime != -1) {
builder.field(Fields.DISK_SERVICE_TIME, Strings.format1Decimals(diskServiceTime, ""));
}
if (spins != null) {
builder.field(Fields.SPINS, spins.toString());
}
@ -328,36 +194,36 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
}
long timestamp;
Info total;
Info[] infos;
Path total;
Path[] paths;
FsStats() {
}
public FsStats(long timestamp, Info[] infos) {
public FsStats(long timestamp, Path[] paths) {
this.timestamp = timestamp;
this.infos = infos;
this.paths = paths;
this.total = null;
}
public Info getTotal() {
public Path getTotal() {
return total();
}
public Info total() {
public Path total() {
if (total != null) {
return total;
}
Info res = new Info();
Set<String> seenDevices = new HashSet<>(infos.length);
for (Info subInfo : infos) {
if (subInfo.dev != null) {
if (!seenDevices.add(subInfo.dev)) {
Path res = new Path();
Set<String> seenDevices = new HashSet<>(paths.length);
for (Path subPath : paths) {
if (subPath.path != null) {
if (!seenDevices.add(subPath.path)) {
continue; // already added numbers for this device;
}
}
res.add(subInfo);
res.add(subPath);
}
total = res;
return res;
@ -368,8 +234,8 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
}
@Override
public Iterator<Info> iterator() {
return Iterators.forArray(infos);
public Iterator<Path> iterator() {
return Iterators.forArray(paths);
}
public static FsStats readFsStats(StreamInput in) throws IOException {
@ -381,18 +247,18 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
@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] = Info.readInfoFrom(in);
paths = new Path[in.readVInt()];
for (int i = 0; i < paths.length; i++) {
paths[i] = Path.readInfoFrom(in);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(timestamp);
out.writeVInt(infos.length);
for (Info info : infos) {
info.writeTo(out);
out.writeVInt(paths.length);
for (Path path : paths) {
path.writeTo(out);
}
}
@ -410,8 +276,8 @@ public class FsStats implements Iterable<FsStats.Info>, Streamable, ToXContent {
builder.field(Fields.TOTAL);
total().toXContent(builder, params);
builder.startArray(Fields.DATA);
for (Info info : infos) {
info.toXContent(builder, params);
for (Path path : paths) {
path.toXContent(builder, params);
}
builder.endArray();
builder.endObject();

View File

@ -1,68 +0,0 @@
/*
* Licensed to Elasticsearch 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 org.elasticsearch.env.NodeEnvironment.NodePath;
import java.io.IOException;
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() throws IOException {
if (!nodeEnv.hasNodeFile()) {
return new FsStats(System.currentTimeMillis(), new FsStats.Info[0]);
}
NodePath[] dataLocations = nodeEnv.nodePaths();
FsStats.Info[] infos = new FsStats.Info[dataLocations.length];
for (int i = 0; i < dataLocations.length; i++) {
infos[i] = getFSInfo(dataLocations[i]);
}
return new FsStats(System.currentTimeMillis(), infos);
}
public static FsStats.Info getFSInfo(NodePath nodePath) throws IOException {
FsStats.Info info = new FsStats.Info();
info.path = nodePath.path.toAbsolutePath().toString();
// NOTE: we use already cached (on node startup) FileStore and spins
// since recomputing these once per second (default) could be costly,
// and they should not change:
info.total = nodePath.fileStore.getTotalSpace();
info.free = nodePath.fileStore.getUnallocatedSpace();
info.available = nodePath.fileStore.getUsableSpace();
info.type = nodePath.fileStore.type();
info.mount = nodePath.fileStore.toString();
info.spins = nodePath.spins;
return info;
}
}

View File

@ -141,11 +141,11 @@ public class MockDiskUsagesTests extends ElasticsearchIntegrationTest {
/** Create a fake NodeStats for the given node and usage */
public static NodeStats makeStats(String nodeName, DiskUsage usage) {
FsStats.Info[] infos = new FsStats.Info[1];
FsStats.Info info = new FsStats.Info("/path.data", null, null,
usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes(), -1, -1, -1, -1, -1, -1);
infos[0] = info;
FsStats fsStats = new FsStats(System.currentTimeMillis(), infos);
FsStats.Path[] paths = new FsStats.Path[1];
FsStats.Path path = new FsStats.Path("/path.data", null,
usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes());
paths[0] = path;
FsStats fsStats = new FsStats(System.currentTimeMillis(), paths);
return new NodeStats(new DiscoveryNode(nodeName, null, Version.V_2_0_0),
System.currentTimeMillis(),
null, null, null, null, null,

View File

@ -570,7 +570,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
NodeStats nodeStats = nodeStatses.getNodes()[0];
List<Path> files = new ArrayList<>();
filesToNodes.put(nodeStats.getNode().getName(), files);
for (FsStats.Info info : nodeStats.getFs()) {
for (FsStats.Path info : nodeStats.getFs()) {
String path = info.getPath();
final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/index";
Path file = PathUtils.get(path).resolve(relativeDataLocationPath);
@ -604,7 +604,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
String nodeId = shardRouting.currentNodeId();
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get();
Set<Path> files = new TreeSet<>(); // treeset makes sure iteration order is deterministic
for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) {
for (FsStats.Path info : nodeStatses.getNodes()[0].getFs()) {
String path = info.getPath();
final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/index";
Path file = PathUtils.get(path).resolve(relativeDataLocationPath);
@ -716,7 +716,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
assertThat(routing.toString(), nodeStatses.getNodes().length, equalTo(1));
List<Path> files = new ArrayList<>();
for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) {
for (FsStats.Path info : nodeStatses.getNodes()[0].getFs()) {
String path = info.getPath();
Path file = PathUtils.get(path).resolve("indices/test/" + Integer.toString(routing.getId()) + "/index");
if (Files.exists(file)) { // multi data path might only have one path in use

View File

@ -124,8 +124,8 @@ public class CorruptedTranslogTests extends ElasticsearchIntegrationTest {
String nodeId = shardRouting.currentNodeId();
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get();
Set<Path> files = new TreeSet<>(); // treeset makes sure iteration order is deterministic
for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) {
String path = info.getPath();
for (FsStats.Path fsPath : nodeStatses.getNodes()[0].getFs()) {
String path = fsPath.getPath();
final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/translog";
Path file = PathUtils.get(path).resolve(relativeDataLocationPath);
if (Files.exists(file)) {

View File

@ -0,0 +1,59 @@
/*
* Licensed to Elasticsearch 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.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
import java.io.IOException;
import static org.hamcrest.Matchers.*;
public class FsProbeTests extends ElasticsearchTestCase {
@Test
public void testFsStats() throws IOException {
try (NodeEnvironment env = newNodeEnvironment()) {
FsProbe probe = new FsProbe(Settings.EMPTY, env);
FsStats stats = probe.stats();
assertNotNull(stats);
assertThat(stats.getTimestamp(), greaterThan(0L));
FsStats.Path total = stats.getTotal();
assertNotNull(total);
assertThat(total.total, greaterThan(0L));
assertThat(total.free, greaterThan(0L));
assertThat(total.available, greaterThan(0L));
for (FsStats.Path path : stats) {
assertNotNull(path);
assertThat(path.getPath(), not(isEmptyOrNullString()));
assertThat(path.getMount(), not(isEmptyOrNullString()));
assertThat(path.getType(), not(isEmptyOrNullString()));
assertThat(path.total, greaterThan(0L));
assertThat(path.free, greaterThan(0L));
assertThat(path.available, greaterThan(0L));
}
}
}
}

View File

@ -69,6 +69,55 @@ curl -XGET 'http://localhost:9200/_nodes/10.0.0.1/stats/process'
The `all` flag can be set to return all the stats.
[[fs-stats]]
==== File Store statistics
A file store represents a storage pool, device,
partition, volume, concrete file system or other implementation
specific means of file storage.
The `fs` flag can be set to retrieve
statistics that concern the file stores:
`fs.timestamp`::
Last time the file stores statistics have been refreshed
`os.total.total_in_bytes`::
Total size (in bytes) of all file stores
`os.total.free_in_bytes`::
Total number of unallocated bytes in all file stores
`os.total.available_in_bytes`::
Totalnumber of bytes available to this Java virtual machine on all file stores
`os.data`::
List of all file stores
`os.data.path`::
Path to the file store
`os.data.mount`::
Mount point of the file store (ex: /dev/sda2)
`os.data.type`::
Type of the file store (ex: ext4)
`os.data.total_in_bytes`::
Total size (in bytes) of thefile store
`os.data.free_in_bytes`::
Total number of unallocated bytes in the file store
`os.data.available_in_bytes`::
Totalnumber of bytes available to this Java virtual machine on this file store
`os.data.spins` (Linux only)::
Indicates if the file store is backed by spinning storage.
`null` means we could not determine it, `true` means the device possibly spins
and `false` means it does not (ex: solid-state disks).
[float]
[[field-data]]
=== Field data statistics