|
@@ -22,7 +22,6 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.metrics.*;
|
|
|
import org.apache.hadoop.metrics.jvm.JvmMetrics;
|
|
|
-import org.apache.hadoop.metrics.util.MetricsLongValue;
|
|
|
import org.apache.hadoop.metrics.util.MetricsIntValue;
|
|
|
import org.apache.hadoop.metrics.util.MetricsTimeVaryingInt;
|
|
|
import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
|
|
@@ -44,15 +43,15 @@ public class FSNamesystemMetrics implements Updater {
|
|
|
private final MetricsRecord metricsRecord;
|
|
|
private FSNamesystem fsNameSystem;
|
|
|
|
|
|
- public MetricsLongValue filesTotal = new MetricsLongValue("FilesTotal");
|
|
|
- public MetricsLongValue blocksTotal = new MetricsLongValue("BlocksTotal");
|
|
|
- public MetricsLongValue capacityTotal = new MetricsLongValue("CapacityTotal");
|
|
|
- public MetricsLongValue capacityUsed = new MetricsLongValue("CapacityUsed");
|
|
|
- public MetricsLongValue capacityRemaining = new MetricsLongValue("CapacityRemaining");
|
|
|
+ public MetricsIntValue filesTotal = new MetricsIntValue("FilesTotal");
|
|
|
+ public MetricsIntValue blocksTotal = new MetricsIntValue("BlocksTotal");
|
|
|
+ public MetricsIntValue capacityTotalGB = new MetricsIntValue("CapacityTotalGB");
|
|
|
+ public MetricsIntValue capacityUsedGB = new MetricsIntValue("CapacityUsedGB");
|
|
|
+ public MetricsIntValue capacityRemainingGB = new MetricsIntValue("CapacityRemainingGB");
|
|
|
public MetricsIntValue totalLoad = new MetricsIntValue("TotalLoad");
|
|
|
- public MetricsLongValue pendingReplicationBlocks = new MetricsLongValue("PendingReplicationBlocks");
|
|
|
- public MetricsLongValue underReplicatedBlocks = new MetricsLongValue("UnderReplicatedBlocks");
|
|
|
- public MetricsLongValue scheduledReplicationBlocks = new MetricsLongValue("ScheduledReplicationBlocks");
|
|
|
+ public MetricsIntValue pendingReplicationBlocks = new MetricsIntValue("PendingReplicationBlocks");
|
|
|
+ public MetricsIntValue underReplicatedBlocks = new MetricsIntValue("UnderReplicatedBlocks");
|
|
|
+ public MetricsIntValue scheduledReplicationBlocks = new MetricsIntValue("ScheduledReplicationBlocks");
|
|
|
FSNamesystemMetrics(Configuration conf, FSNamesystem fsNameSystem) {
|
|
|
String sessionId = conf.get("session.id");
|
|
|
this.fsNameSystem = fsNameSystem;
|
|
@@ -69,6 +68,10 @@ public class FSNamesystemMetrics implements Updater {
|
|
|
if (fsNameSystem != null)
|
|
|
fsNameSystem.shutdown();
|
|
|
}
|
|
|
+
|
|
|
+ private int roundBytesToGBytes(long bytes) {
|
|
|
+ return Math.round(((float)bytes/(1024 * 1024 * 1024)));
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Since this object is a registered updater, this method will be called
|
|
@@ -76,34 +79,42 @@ public class FSNamesystemMetrics implements Updater {
|
|
|
* We set the metrics value within this function before pushing it out.
|
|
|
* FSNamesystem updates its own local variables which are
|
|
|
* light weight compared to Metrics counters.
|
|
|
+ *
|
|
|
+ * Some of the metrics are explicity casted to int. Few metrics collectors
|
|
|
+ * do not handle long values. It is safe to cast to int for now as all these
|
|
|
+ * values fit in int value.
|
|
|
+ * Metrics related to DFS capacity are stored in bytes which do not fit in
|
|
|
+ * int, so they are rounded to GB
|
|
|
*/
|
|
|
public void doUpdates(MetricsContext unused) {
|
|
|
synchronized (this) {
|
|
|
- filesTotal.set(fsNameSystem.getFilesTotal());
|
|
|
+ filesTotal.set((int)fsNameSystem.getFilesTotal());
|
|
|
filesTotal.pushMetric(metricsRecord);
|
|
|
|
|
|
- blocksTotal.set(fsNameSystem.getBlocksTotal());
|
|
|
+ blocksTotal.set((int)fsNameSystem.getBlocksTotal());
|
|
|
blocksTotal.pushMetric(metricsRecord);
|
|
|
|
|
|
- capacityTotal.set(fsNameSystem.getCapacityTotal());
|
|
|
- capacityTotal.pushMetric(metricsRecord);
|
|
|
+ capacityTotalGB.set(roundBytesToGBytes(fsNameSystem.getCapacityTotal()));
|
|
|
+ capacityTotalGB.pushMetric(metricsRecord);
|
|
|
|
|
|
- capacityUsed.set(fsNameSystem.getCapacityUsed());
|
|
|
- capacityUsed.pushMetric(metricsRecord);
|
|
|
+ capacityUsedGB.set(roundBytesToGBytes(fsNameSystem.getCapacityUsed()));
|
|
|
+ capacityUsedGB.pushMetric(metricsRecord);
|
|
|
|
|
|
- capacityRemaining.set(fsNameSystem.getCapacityRemaining());
|
|
|
- capacityRemaining.pushMetric(metricsRecord);
|
|
|
+ capacityRemainingGB.set(roundBytesToGBytes(fsNameSystem.
|
|
|
+ getCapacityRemaining()));
|
|
|
+ capacityRemainingGB.pushMetric(metricsRecord);
|
|
|
|
|
|
totalLoad.set(fsNameSystem.getTotalLoad());
|
|
|
totalLoad.pushMetric(metricsRecord);
|
|
|
|
|
|
- pendingReplicationBlocks.set(fsNameSystem.getPendingReplicationBlocks());
|
|
|
+ pendingReplicationBlocks.set((int)fsNameSystem.
|
|
|
+ getPendingReplicationBlocks());
|
|
|
pendingReplicationBlocks.pushMetric(metricsRecord);
|
|
|
|
|
|
- underReplicatedBlocks.set(fsNameSystem.getUnderReplicatedBlocks());
|
|
|
+ underReplicatedBlocks.set((int)fsNameSystem.getUnderReplicatedBlocks());
|
|
|
underReplicatedBlocks.pushMetric(metricsRecord);
|
|
|
|
|
|
- scheduledReplicationBlocks.set(fsNameSystem.
|
|
|
+ scheduledReplicationBlocks.set((int)fsNameSystem.
|
|
|
getScheduledReplicationBlocks());
|
|
|
scheduledReplicationBlocks.pushMetric(metricsRecord);
|
|
|
}
|