Browse Source

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

Tanguy Leroux 10 years ago
parent
commit
fbcf4dbbf7

+ 4 - 4
core/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java

@@ -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);

+ 1 - 1
core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java

@@ -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();
                             }

+ 15 - 15
core/src/main/java/org/elasticsearch/env/NodeEnvironment.java

@@ -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)));
         }

+ 1 - 2
core/src/main/java/org/elasticsearch/monitor/MonitorModule.java

@@ -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();

+ 42 - 4
core/src/main/java/org/elasticsearch/monitor/fs/FsProbe.java

@@ -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 {
+
+    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();
 
-    FsStats stats() throws IOException;
+        // 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;
+    }
 }

+ 34 - 168
core/src/main/java/org/elasticsearch/monitor/fs/FsStats.java

@@ -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();

+ 0 - 68
core/src/main/java/org/elasticsearch/monitor/fs/JmxFsProbe.java

@@ -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;
-    }
-}

+ 5 - 5
core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesTests.java

@@ -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,

+ 3 - 3
core/src/test/java/org/elasticsearch/index/store/CorruptedFileTest.java

@@ -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

+ 2 - 2
core/src/test/java/org/elasticsearch/index/store/CorruptedTranslogTests.java

@@ -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)) {

+ 59 - 0
core/src/test/java/org/elasticsearch/monitor/fs/FsProbeTests.java

@@ -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));
+            }
+        }
+    }
+}

+ 49 - 0
docs/reference/cluster/nodes-stats.asciidoc

@@ -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