|
@@ -38,6 +38,8 @@ import java.util.List;
|
|
|
import java.util.function.Supplier;
|
|
|
|
|
|
import net.jcip.annotations.NotThreadSafe;
|
|
|
+
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.net.unix.DomainSocket;
|
|
@@ -751,4 +753,67 @@ public class TestDataNodeMetrics {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDataNodeReadWriteXceiversCount() throws Exception {
|
|
|
+ try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()).build()) {
|
|
|
+ cluster.waitActive();
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+ List<DataNode> datanodes = cluster.getDataNodes();
|
|
|
+ assertEquals(1, datanodes.size());
|
|
|
+ DataNode datanode = datanodes.get(0);
|
|
|
+
|
|
|
+ // Test DataNodeWriteActiveXceiversCount Metric
|
|
|
+ long writeXceiversCount = MetricsAsserts.getIntGauge("DataNodeWriteActiveXceiversCount",
|
|
|
+ getMetrics(datanode.getMetrics().name()));
|
|
|
+ assertEquals(0, writeXceiversCount);
|
|
|
+
|
|
|
+ Path path = new Path("/testDataNodeReadWriteXceiversCount.txt");
|
|
|
+ try (FSDataOutputStream output = fs.create(path)) {
|
|
|
+ output.write(new byte[1024]);
|
|
|
+ output.hsync();
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ int writeXceiversCount = MetricsAsserts.getIntGauge("DataNodeWriteActiveXceiversCount",
|
|
|
+ getMetrics(datanode.getMetrics().name()));
|
|
|
+ return writeXceiversCount == 1;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+ }
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ int writeXceiversCount = MetricsAsserts.getIntGauge("DataNodeWriteActiveXceiversCount",
|
|
|
+ getMetrics(datanode.getMetrics().name()));
|
|
|
+ return writeXceiversCount == 0;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+
|
|
|
+ // Test DataNodeReadActiveXceiversCount Metric
|
|
|
+ long readXceiversCount = MetricsAsserts.getIntGauge("DataNodeReadActiveXceiversCount",
|
|
|
+ getMetrics(datanode.getMetrics().name()));
|
|
|
+ assertEquals(0, readXceiversCount);
|
|
|
+ try (FSDataInputStream input = fs.open(path)) {
|
|
|
+ byte[] byteArray = new byte[1024];
|
|
|
+ input.read(byteArray);
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ int readXceiversCount = MetricsAsserts.getIntGauge("DataNodeReadActiveXceiversCount",
|
|
|
+ getMetrics(datanode.getMetrics().name()));
|
|
|
+ return readXceiversCount == 1;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+ }
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ int readXceiversCount = MetricsAsserts.getIntGauge("DataNodeReadActiveXceiversCount",
|
|
|
+ getMetrics(datanode.getMetrics().name()));
|
|
|
+ return readXceiversCount == 0;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|