HDFS-7773. Additional metrics in HDFS to be accessed via jmx. Contributed by Anu Engineer.
This commit is contained in:
parent
8c6ae0d619
commit
02e7dec79d
|
@ -191,6 +191,7 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
|
|||
| `GetImageAvgTime` | Average fsimage download time in milliseconds |
|
||||
| `PutImageNumOps` | Total number of fsimage uploads to SecondaryNameNode |
|
||||
| `PutImageAvgTime` | Average fsimage upload time in milliseconds |
|
||||
| `TotalFileOps`| Total number of file operations performed |
|
||||
|
||||
FSNamesystem
|
||||
------------
|
||||
|
@ -314,6 +315,10 @@ Each metrics record contains tags such as SessionId and Hostname as additional i
|
|||
| `SendDataPacketBlockedOnNetworkNanosAvgTime` | Average waiting time of sending packets in nanoseconds |
|
||||
| `SendDataPacketTransferNanosNumOps` | Total number of sending packets |
|
||||
| `SendDataPacketTransferNanosAvgTime` | Average transfer time of sending packets in nanoseconds |
|
||||
| `TotalWriteTime`| Total number of milliseconds spent on write operation |
|
||||
| `TotalReadTime` | Total number of milliseconds spent on read operation |
|
||||
| `RemoteBytesRead` | Number of bytes read by remote clients |
|
||||
| `RemoteBytesWritten` | Number of bytes written by remote clients |
|
||||
|
||||
yarn context
|
||||
============
|
||||
|
|
|
@ -663,6 +663,9 @@ Release 2.7.0 - UNRELEASED
|
|||
HDFS-7772. Document hdfs balancer -exclude/-include option in
|
||||
HDFSCommands.html (Xiaoyu Yao via cnauroth)
|
||||
|
||||
HDFS-7773. Additional metrics in HDFS to be accessed via jmx.
|
||||
(Anu Engineer via cnauroth)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
|
||||
|
|
|
@ -658,6 +658,7 @@ class BlockReceiver implements Closeable {
|
|||
replicaInfo.setLastChecksumAndDataLen(offsetInBlock, lastCrc);
|
||||
|
||||
datanode.metrics.incrBytesWritten(len);
|
||||
datanode.metrics.incrTotalWriteTime(duration);
|
||||
|
||||
manageWriterOsCache(offsetInBlock);
|
||||
}
|
||||
|
|
|
@ -86,6 +86,7 @@ import org.apache.hadoop.util.DataChecksum;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -480,7 +481,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
final boolean sendChecksum,
|
||||
final CachingStrategy cachingStrategy) throws IOException {
|
||||
previousOpClientName = clientName;
|
||||
|
||||
long read = 0;
|
||||
OutputStream baseStream = getOutputStream();
|
||||
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
|
||||
|
@ -515,8 +516,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
// send op status
|
||||
writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream()));
|
||||
|
||||
long read = blockSender.sendBlock(out, baseStream, null); // send data
|
||||
|
||||
long beginRead = Time.monotonicNow();
|
||||
read = blockSender.sendBlock(out, baseStream, null); // send data
|
||||
long duration = Time.monotonicNow() - beginRead;
|
||||
if (blockSender.didSendEntireByteRange()) {
|
||||
// If we sent the entire range, then we should expect the client
|
||||
// to respond with a Status enum.
|
||||
|
@ -539,6 +541,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
}
|
||||
datanode.metrics.incrBytesRead((int) read);
|
||||
datanode.metrics.incrBlocksRead();
|
||||
datanode.metrics.incrTotalReadTime(duration);
|
||||
} catch ( SocketException ignored ) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(dnR + ":Ignoring exception while serving " + block + " to " +
|
||||
|
@ -563,7 +566,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
|
||||
//update metrics
|
||||
datanode.metrics.addReadBlockOp(elapsed());
|
||||
datanode.metrics.incrReadsFromClient(peer.isLocal());
|
||||
datanode.metrics.incrReadsFromClient(peer.isLocal(), read);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -590,7 +593,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
final boolean isClient = !isDatanode;
|
||||
final boolean isTransfer = stage == BlockConstructionStage.TRANSFER_RBW
|
||||
|| stage == BlockConstructionStage.TRANSFER_FINALIZED;
|
||||
|
||||
long size = 0;
|
||||
// check single target for transfer-RBW/Finalized
|
||||
if (isTransfer && targets.length > 0) {
|
||||
throw new IOException(stage + " does not support multiple targets "
|
||||
|
@ -796,7 +799,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
+ localAddress + " of size " + block.getNumBytes());
|
||||
}
|
||||
|
||||
|
||||
if(isClient) {
|
||||
size = block.getNumBytes();
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("opWriteBlock " + block + " received exception " + ioe);
|
||||
incrDatanodeNetworkErrors();
|
||||
|
@ -813,7 +818,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
|
||||
//update metrics
|
||||
datanode.metrics.addWriteBlockOp(elapsed());
|
||||
datanode.metrics.incrWritesFromClient(peer.isLocal());
|
||||
datanode.metrics.incrWritesFromClient(peer.isLocal(), size);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -993,12 +998,15 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
|
||||
// send status first
|
||||
writeSuccessWithChecksumInfo(blockSender, reply);
|
||||
// send block content to the target
|
||||
long read = blockSender.sendBlock(reply, baseStream,
|
||||
dataXceiverServer.balanceThrottler);
|
||||
|
||||
long beginRead = Time.monotonicNow();
|
||||
// send block content to the target
|
||||
long read = blockSender.sendBlock(reply, baseStream,
|
||||
dataXceiverServer.balanceThrottler);
|
||||
long duration = Time.monotonicNow() - beginRead;
|
||||
datanode.metrics.incrBytesRead((int) read);
|
||||
datanode.metrics.incrBlocksRead();
|
||||
datanode.metrics.incrTotalReadTime(duration);
|
||||
|
||||
LOG.info("Copied " + block + " to " + peer.getRemoteAddressString());
|
||||
} catch (IOException ioe) {
|
||||
|
|
|
@ -50,7 +50,11 @@ import org.apache.hadoop.metrics2.source.JvmMetrics;
|
|||
public class DataNodeMetrics {
|
||||
|
||||
@Metric MutableCounterLong bytesWritten;
|
||||
@Metric("Milliseconds spent writing")
|
||||
MutableCounterLong totalWriteTime;
|
||||
@Metric MutableCounterLong bytesRead;
|
||||
@Metric("Milliseconds spent reading")
|
||||
MutableCounterLong totalReadTime;
|
||||
@Metric MutableCounterLong blocksWritten;
|
||||
@Metric MutableCounterLong blocksRead;
|
||||
@Metric MutableCounterLong blocksReplicated;
|
||||
|
@ -64,6 +68,10 @@ public class DataNodeMetrics {
|
|||
@Metric MutableCounterLong writesFromLocalClient;
|
||||
@Metric MutableCounterLong writesFromRemoteClient;
|
||||
@Metric MutableCounterLong blocksGetLocalPathInfo;
|
||||
@Metric("Bytes read by remote client")
|
||||
MutableCounterLong remoteBytesRead;
|
||||
@Metric("Bytes written by remote client")
|
||||
MutableCounterLong remoteBytesWritten;
|
||||
|
||||
// RamDisk metrics on read/write
|
||||
@Metric MutableCounterLong ramDiskBlocksWrite;
|
||||
|
@ -262,6 +270,15 @@ public class DataNodeMetrics {
|
|||
fsyncCount.incr();
|
||||
}
|
||||
|
||||
public void incrTotalWriteTime(long timeTaken) {
|
||||
totalWriteTime.incr(timeTaken);
|
||||
}
|
||||
|
||||
public void incrTotalReadTime(long timeTaken) {
|
||||
totalReadTime.incr(timeTaken);
|
||||
}
|
||||
|
||||
|
||||
public void addPacketAckRoundTripTimeNanos(long latencyNanos) {
|
||||
packetAckRoundTripTimeNanos.add(latencyNanos);
|
||||
for (MutableQuantiles q : packetAckRoundTripTimeNanosQuantiles) {
|
||||
|
@ -287,12 +304,23 @@ public class DataNodeMetrics {
|
|||
DefaultMetricsSystem.shutdown();
|
||||
}
|
||||
|
||||
public void incrWritesFromClient(boolean local) {
|
||||
(local ? writesFromLocalClient : writesFromRemoteClient).incr();
|
||||
public void incrWritesFromClient(boolean local, long size) {
|
||||
if(local) {
|
||||
writesFromLocalClient.incr();
|
||||
} else {
|
||||
writesFromRemoteClient.incr();
|
||||
remoteBytesWritten.incr(size);
|
||||
}
|
||||
}
|
||||
|
||||
public void incrReadsFromClient(boolean local) {
|
||||
(local ? readsFromLocalClient : readsFromRemoteClient).incr();
|
||||
public void incrReadsFromClient(boolean local, long size) {
|
||||
|
||||
if (local) {
|
||||
readsFromLocalClient.incr();
|
||||
} else {
|
||||
readsFromRemoteClient.incr();
|
||||
remoteBytesRead.incr(size);
|
||||
}
|
||||
}
|
||||
|
||||
public void incrVolumeFailures() {
|
||||
|
|
|
@ -77,6 +77,31 @@ public class NameNodeMetrics {
|
|||
@Metric("Number of blockReports from individual storages")
|
||||
MutableCounterLong storageBlockReportOps;
|
||||
|
||||
@Metric("Number of file system operations")
|
||||
public long totalFileOps(){
|
||||
return
|
||||
getBlockLocations.value() +
|
||||
createFileOps.value() +
|
||||
filesAppended.value() +
|
||||
addBlockOps.value() +
|
||||
getAdditionalDatanodeOps.value() +
|
||||
filesRenamed.value() +
|
||||
filesTruncated.value() +
|
||||
deleteFileOps.value() +
|
||||
getListingOps.value() +
|
||||
fileInfoOps.value() +
|
||||
getLinkTargetOps.value() +
|
||||
createSnapshotOps.value() +
|
||||
deleteSnapshotOps.value() +
|
||||
allowSnapshotOps.value() +
|
||||
disallowSnapshotOps.value() +
|
||||
renameSnapshotOps.value() +
|
||||
listSnapshottableDirOps.value() +
|
||||
createSymlinkOps.value() +
|
||||
snapshotDiffReportOps.value();
|
||||
}
|
||||
|
||||
|
||||
@Metric("Journal transactions") MutableRate transactions;
|
||||
@Metric("Journal syncs") MutableRate syncs;
|
||||
final MutableQuantiles[] syncsQuantiles;
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
|
@ -246,4 +247,48 @@ public class TestDataNodeMetrics {
|
|||
DataNodeFaultInjector.instance = new DataNodeFaultInjector();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This function ensures that writing causes TotalWritetime to increment
|
||||
* and reading causes totalReadTime to move.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testDataNodeTimeSpend() throws Exception {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
SimulatedFSDataset.setFactory(conf);
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
||||
try {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
List<DataNode> datanodes = cluster.getDataNodes();
|
||||
assertEquals(datanodes.size(), 1);
|
||||
DataNode datanode = datanodes.get(0);
|
||||
MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name());
|
||||
final long LONG_FILE_LEN = 1024 * 1024 * 10;
|
||||
|
||||
long startWriteValue = getLongCounter("TotalWriteTime", rb);
|
||||
long startReadValue = getLongCounter("TotalReadTime", rb);
|
||||
|
||||
for (int x =0; x < 50; x++) {
|
||||
DFSTestUtil.createFile(fs, new Path("/time.txt."+ x),
|
||||
LONG_FILE_LEN, (short) 1, Time.monotonicNow());
|
||||
}
|
||||
|
||||
for (int x =0; x < 50; x++) {
|
||||
String s = DFSTestUtil.readFile(fs, new Path("/time.txt." + x));
|
||||
}
|
||||
|
||||
MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
|
||||
long endWriteValue = getLongCounter("TotalWriteTime", rbNew);
|
||||
long endReadValue = getLongCounter("TotalReadTime", rbNew);
|
||||
|
||||
assertTrue(endReadValue > startReadValue);
|
||||
assertTrue(endWriteValue > startWriteValue);
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -455,4 +455,24 @@ public class TestNameNodeMetrics {
|
|||
assertQuantileGauges("Syncs1s", rb);
|
||||
assertQuantileGauges("BlockReport1s", rb);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test NN ReadOps Count and WriteOps Count
|
||||
*/
|
||||
@Test
|
||||
public void testReadWriteOps() throws Exception {
|
||||
MetricsRecordBuilder rb = getMetrics(NN_METRICS);
|
||||
long startWriteCounter = MetricsAsserts.getLongCounter("TransactionsNumOps",
|
||||
rb);
|
||||
Path file1_Path = new Path(TEST_ROOT_DIR_PATH, "ReadData.dat");
|
||||
|
||||
//Perform create file operation
|
||||
createFile(file1_Path, 1024 * 1024,(short)2);
|
||||
|
||||
// Perform read file operation on earlier created file
|
||||
readFile(fs, file1_Path);
|
||||
MetricsRecordBuilder rbNew = getMetrics(NN_METRICS);
|
||||
assertTrue(MetricsAsserts.getLongCounter("TransactionsNumOps", rbNew) >
|
||||
startWriteCounter);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue