|
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
|
+import static org.junit.Assert.assertNotEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
@@ -177,4 +179,57 @@ public class TestDataNodeVolumeMetrics {
|
|
LOG.info("fileIoErrorMean : " + metrics.getFileIoErrorMean());
|
|
LOG.info("fileIoErrorMean : " + metrics.getFileIoErrorMean());
|
|
LOG.info("fileIoErrorStdDev : " + metrics.getFileIoErrorStdDev());
|
|
LOG.info("fileIoErrorStdDev : " + metrics.getFileIoErrorStdDev());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testWriteIoVolumeMetrics() throws IOException {
|
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
|
+ conf.setInt(
|
|
|
|
+ DFSConfigKeys.DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY,
|
|
|
|
+ 100);
|
|
|
|
+ MiniDFSCluster cluster =
|
|
|
|
+ new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(NUM_DATANODES)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[]{StorageType.RAM_DISK, StorageType.DISK})
|
|
|
|
+ .storagesPerDatanode(2).build();
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
|
+ final Path fileName = new Path("/test.dat");
|
|
|
|
+ final long fileLen = Integer.MAX_VALUE + 1L;
|
|
|
|
+ long lastWriteIoSampleCount;
|
|
|
|
+
|
|
|
|
+ DFSTestUtil.createFile(fs, fileName, false, BLOCK_SIZE, fileLen,
|
|
|
|
+ fs.getDefaultBlockSize(fileName), REPL, 1L, true);
|
|
|
|
+
|
|
|
|
+ List<DataNode> datanodes = cluster.getDataNodes();
|
|
|
|
+ DataNode datanode = datanodes.get(0);
|
|
|
|
+
|
|
|
|
+ final ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, fileName);
|
|
|
|
+ final FsVolumeSpi volume = datanode.getFSDataset().getVolume(block);
|
|
|
|
+ DataNodeVolumeMetrics metrics = volume.getMetrics();
|
|
|
|
+
|
|
|
|
+ assertEquals(0, metrics.getSyncIoSampleCount());
|
|
|
|
+ assertNotEquals(0, metrics.getWriteIoSampleCount());
|
|
|
|
+ assertTrue(metrics.getFlushIoSampleCount() > metrics
|
|
|
|
+ .getSyncIoSampleCount());
|
|
|
|
+ assertTrue(metrics.getWriteIoSampleCount() > metrics
|
|
|
|
+ .getFlushIoSampleCount());
|
|
|
|
+
|
|
|
|
+ lastWriteIoSampleCount = metrics.getWriteIoSampleCount();
|
|
|
|
+
|
|
|
|
+ try (FSDataOutputStream out = fs.append(fileName)) {
|
|
|
|
+ out.writeBytes("hello world");
|
|
|
|
+ out.hflush();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ assertEquals(0, metrics.getSyncIoSampleCount());
|
|
|
|
+ assertTrue(metrics.getWriteIoSampleCount() > lastWriteIoSampleCount);
|
|
|
|
+
|
|
|
|
+ } finally {
|
|
|
|
+ if (cluster != null) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|