|
@@ -19,6 +19,7 @@ import org.apache.commons.logging.*;
|
|
|
|
|
|
import org.apache.hadoop.ipc.*;
|
|
|
import org.apache.hadoop.conf.*;
|
|
|
+import org.apache.hadoop.metrics.Metrics;
|
|
|
import org.apache.hadoop.util.*;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
import org.apache.hadoop.mapred.StatusHttpServer;
|
|
@@ -26,6 +27,7 @@ import org.apache.hadoop.mapred.StatusHttpServer;
|
|
|
import java.io.*;
|
|
|
import java.net.*;
|
|
|
import java.util.*;
|
|
|
+import org.apache.hadoop.metrics.MetricsRecord;
|
|
|
|
|
|
/**********************************************************
|
|
|
* DataNode is a class (and program) that stores a set of
|
|
@@ -100,6 +102,54 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
private static InetSocketAddress nameNodeAddr;
|
|
|
private static DataNode datanodeObject = null;
|
|
|
static Date startTime = new Date(System.currentTimeMillis());
|
|
|
+ private class DataNodeMetrics {
|
|
|
+ private MetricsRecord metricsRecord = null;
|
|
|
+
|
|
|
+
|
|
|
+ private long bytesWritten = 0L;
|
|
|
+ private long bytesRead = 0L;
|
|
|
+ private long blocksWritten = 0L;
|
|
|
+ private long blocksRead = 0L;
|
|
|
+ private long blocksReplicated = 0L;
|
|
|
+ private long blocksRemoved = 0L;
|
|
|
+
|
|
|
+ DataNodeMetrics() {
|
|
|
+ metricsRecord = Metrics.createRecord("dfs", "datanode");
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void readBytes(int nbytes) {
|
|
|
+ bytesRead += nbytes;
|
|
|
+ Metrics.report(metricsRecord, "bytes-read", bytesRead);
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void wroteBytes(int nbytes) {
|
|
|
+ bytesWritten += nbytes;
|
|
|
+ Metrics.report(metricsRecord, "bytes-written", bytesWritten);
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void readBlocks(int nblocks) {
|
|
|
+ blocksRead += nblocks;
|
|
|
+ Metrics.report(metricsRecord, "blocks-read", blocksRead);
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void wroteBlocks(int nblocks) {
|
|
|
+ blocksWritten += nblocks;
|
|
|
+ Metrics.report(metricsRecord, "blocks-written", blocksWritten);
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void replicatedBlocks(int nblocks) {
|
|
|
+ blocksReplicated += nblocks;
|
|
|
+ Metrics.report(metricsRecord, "blocks-replicated", blocksReplicated);
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized void removedBlocks(int nblocks) {
|
|
|
+ blocksRemoved += nblocks;
|
|
|
+ Metrics.report(metricsRecord, "blocks-removed", blocksRemoved);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ DataNodeMetrics myMetrics = new DataNodeMetrics();
|
|
|
+
|
|
|
/**
|
|
|
* Create the DataNode given a configuration and a dataDir.
|
|
|
* 'dataDir' is where the blocks are stored.
|
|
@@ -344,7 +394,9 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
// Some local block(s) are obsolete and can be
|
|
|
// safely garbage-collected.
|
|
|
//
|
|
|
- data.invalidate(cmd.getBlocks());
|
|
|
+ Block toDelete[] = cmd.getBlocks();
|
|
|
+ data.invalidate(toDelete);
|
|
|
+ myMetrics.removedBlocks(toDelete.length);
|
|
|
} else if( cmd.shutdownNode()) {
|
|
|
// shut down the data node
|
|
|
this.shutdown();
|
|
@@ -556,6 +608,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
int bytesRead = 0;
|
|
|
try {
|
|
|
bytesRead = in2.read(buf);
|
|
|
+ myMetrics.readBytes(bytesRead);
|
|
|
} catch (IOException iex) {
|
|
|
shutdown();
|
|
|
throw iex;
|
|
@@ -565,6 +618,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
len -= bytesRead;
|
|
|
try {
|
|
|
bytesRead = in2.read(buf);
|
|
|
+ myMetrics.readBytes(bytesRead);
|
|
|
} catch (IOException iex) {
|
|
|
shutdown();
|
|
|
throw iex;
|
|
@@ -582,6 +636,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+ myMetrics.readBlocks(1);
|
|
|
LOG.info("Served block " + b + " to " + s.getInetAddress());
|
|
|
} finally {
|
|
|
out.close();
|
|
@@ -660,6 +715,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
}
|
|
|
out2.write(encodingType);
|
|
|
out2.writeLong(len);
|
|
|
+ myMetrics.replicatedBlocks(1);
|
|
|
} catch (IOException ie) {
|
|
|
if (out2 != null) {
|
|
|
LOG.info("Exception connecting to mirror " + mirrorNode
|
|
@@ -692,6 +748,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
if (bytesRead > 0) {
|
|
|
try {
|
|
|
out.write(buf, 0, bytesRead);
|
|
|
+ myMetrics.wroteBytes(bytesRead);
|
|
|
} catch (IOException iex) {
|
|
|
shutdown();
|
|
|
throw iex;
|
|
@@ -794,6 +851,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
}
|
|
|
}
|
|
|
data.finalizeBlock(b);
|
|
|
+ myMetrics.wroteBlocks(1);
|
|
|
|
|
|
//
|
|
|
// Tell the namenode that we've received this block
|