소스 검색

HDFS-16315. Add metrics related to Transfer and NativeCopy for DataNode (#3643)

Reviewed-by: Hui Fei <ferhui@apache.org>
Reviewed-by: Ayush Saxena <ayushsaxena@apache.org>
litao 3 년 전
부모
커밋
c9f95b01ef

+ 6 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md

@@ -512,6 +512,12 @@ contains tags such as Hostname as additional information along with metrics.
 | `WriteIoRateNumOps` | The number of file write io operations within an interval time of metric |
 | `WriteIoRateAvgTime` | Mean time of file write io operations in milliseconds |
 | `WriteIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `TransferIoRateNumOps` | The number of file transfer io operations within an interval time of metric |
+| `TransferIoRateAvgTime` | Mean time of file transfer io operations in milliseconds |
+| `TransferIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file transfer io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `NativeCopyIoRateNumOps` | The number of file nativeCopy io operations within an interval time of metric |
+| `NativeCopyIoRateAvgTime` | Mean time of file nativeCopy io operations in milliseconds |
+| `NativeCopyIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file nativeCopy io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 | `TotalFileIoErrors` | Total number (monotonically increasing) of file io error operations |
 | `FileIoErrorRateNumOps` | The number of file io error operations within an interval time of metric |
 | `FileIoErrorRateAvgTime` | It measures the mean time in milliseconds from the start of an operation to hitting a failure |

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProfilingFileIoEvents.java

@@ -116,6 +116,12 @@ class ProfilingFileIoEvents {
         case WRITE:
           metrics.addWriteIoLatency(latency);
           break;
+        case TRANSFER:
+          metrics.addTransferIoLatency(latency);
+          break;
+        case NATIVE_COPY:
+          metrics.addNativeCopyIoLatency(latency);
+          break;
         default:
         }
       }

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/DataNodeVolumeMetrics.java

@@ -71,6 +71,14 @@ public class DataNodeVolumeMetrics {
   private MutableRate writeIoRate;
   private MutableQuantiles[] writeIoLatencyQuantiles;
 
+  @Metric("file io transfer rate")
+  private MutableRate transferIoRate;
+  private MutableQuantiles[] transferIoLatencyQuantiles;
+
+  @Metric("file io nativeCopy rate")
+  private MutableRate nativeCopyIoRate;
+  private MutableQuantiles[] nativeCopyIoLatencyQuantiles;
+
   @Metric("number of file io errors")
   private MutableCounterLong totalFileIoErrors;
   @Metric("file io error rate")
@@ -162,6 +170,40 @@ public class DataNodeVolumeMetrics {
     return writeIoRate.lastStat().stddev();
   }
 
+  // Based on transferIoRate
+  public long getTransferIoSampleCount() {
+    return transferIoRate.lastStat().numSamples();
+  }
+
+  public double getTransferIoMean() {
+    return transferIoRate.lastStat().mean();
+  }
+
+  public double getTransferIoStdDev() {
+    return transferIoRate.lastStat().stddev();
+  }
+
+  public MutableQuantiles[] getTransferIoQuantiles() {
+    return transferIoLatencyQuantiles;
+  }
+
+  // Based on nativeCopyIoRate
+  public long getNativeCopyIoSampleCount() {
+    return nativeCopyIoRate.lastStat().numSamples();
+  }
+
+  public double getNativeCopyIoMean() {
+    return nativeCopyIoRate.lastStat().mean();
+  }
+
+  public double getNativeCopyIoStdDev() {
+    return nativeCopyIoRate.lastStat().stddev();
+  }
+
+  public MutableQuantiles[] getNativeCopyIoQuantiles() {
+    return nativeCopyIoLatencyQuantiles;
+  }
+
   public long getTotalFileIoErrors() {
     return totalFileIoErrors.value();
   }
@@ -193,6 +235,8 @@ public class DataNodeVolumeMetrics {
     syncIoLatencyQuantiles = new MutableQuantiles[len];
     readIoLatencyQuantiles = new MutableQuantiles[len];
     writeIoLatencyQuantiles = new MutableQuantiles[len];
+    transferIoLatencyQuantiles = new MutableQuantiles[len];
+    nativeCopyIoLatencyQuantiles = new MutableQuantiles[len];
     for (int i = 0; i < len; i++) {
       int interval = intervals[i];
       metadataOperationLatencyQuantiles[i] = registry.newQuantiles(
@@ -213,6 +257,12 @@ public class DataNodeVolumeMetrics {
       writeIoLatencyQuantiles[i] = registry.newQuantiles(
           "writeIoLatency" + interval + "s",
           "Data write Io Latency in ms", "ops", "latency", interval);
+      transferIoLatencyQuantiles[i] = registry.newQuantiles(
+          "transferIoLatency" + interval + "s",
+          "Data transfer Io Latency in ms", "ops", "latency", interval);
+      nativeCopyIoLatencyQuantiles[i] = registry.newQuantiles(
+          "nativeCopyIoLatency" + interval + "s",
+          "Data nativeCopy Io Latency in ms", "ops", "latency", interval);
     }
   }
 
@@ -282,6 +332,20 @@ public class DataNodeVolumeMetrics {
     }
   }
 
+  public void addTransferIoLatency(final long latency) {
+    transferIoRate.add(latency);
+    for (MutableQuantiles q: transferIoLatencyQuantiles) {
+      q.add(latency);
+    }
+  }
+
+  public void addNativeCopyIoLatency(final long latency) {
+    nativeCopyIoRate.add(latency);
+    for (MutableQuantiles q: nativeCopyIoLatencyQuantiles) {
+      q.add(latency);
+    }
+  }
+
   public void addFileIoError(final long latency) {
     totalFileIoErrors.incr();
     fileIoErrorRate.add(latency);

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeMetrics.java

@@ -151,7 +151,7 @@ public class TestDataNodeVolumeMetrics {
     LOG.info("MetadataOperationSampleCount : " +
         metrics.getMetadataOperationSampleCount());
     LOG.info("MetadataOperationMean : " + metrics.getMetadataOperationMean());
-    LOG.info("MetadataFileIoStdDev : " +
+    LOG.info("MetadataOperationStdDev : " +
         metrics.getMetadataOperationStdDev());
 
     LOG.info("DataFileIoSampleCount : " + metrics.getDataFileIoSampleCount());
@@ -174,6 +174,15 @@ public class TestDataNodeVolumeMetrics {
     LOG.info("writeIoMean : " + metrics.getWriteIoMean());
     LOG.info("writeIoStdDev : " + metrics.getWriteIoStdDev());
 
+    LOG.info("transferIoSampleCount : " + metrics.getTransferIoSampleCount());
+    LOG.info("transferIoMean : " + metrics.getTransferIoMean());
+    LOG.info("transferIoStdDev : " + metrics.getTransferIoStdDev());
+
+    LOG.info("nativeCopyIoSampleCount : " +
+        metrics.getNativeCopyIoSampleCount());
+    LOG.info("nativeCopyIoMean : " + metrics.getNativeCopyIoMean());
+    LOG.info("nativeCopyIoStdDev : " + metrics.getNativeCopyIoStdDev());
+
     LOG.info("fileIoErrorSampleCount : "
         + metrics.getFileIoErrorSampleCount());
     LOG.info("fileIoErrorMean : " + metrics.getFileIoErrorMean());

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -1830,4 +1831,49 @@ public class TestFsDatasetImpl {
       cluster.shutdown();
     }
   }
+
+  @Test(timeout = 30000)
+  public void testTransferAndNativeCopyMetrics() throws IOException {
+    Configuration config = new HdfsConfiguration();
+    config.setInt(
+        DFSConfigKeys.DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY,
+        100);
+    config.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
+        "60,300,1500");
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(config)
+        .numDataNodes(1)
+        .storageTypes(new StorageType[]{StorageType.DISK, StorageType.DISK})
+        .storagesPerDatanode(2)
+        .build()) {
+      FileSystem fs = cluster.getFileSystem();
+      DataNode dataNode = cluster.getDataNodes().get(0);
+
+      // Create file that has one block with one replica.
+      Path filePath = new Path(name.getMethodName());
+      DFSTestUtil.createFile(fs, filePath, 100, (short) 1, 0);
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath);
+
+      // Copy a new replica to other volume.
+      FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
+      ReplicaInfo newReplicaInfo = createNewReplicaObj(block, fsDataSetImpl);
+      fsDataSetImpl.finalizeNewReplica(newReplicaInfo, block);
+
+      // Get the volume where the original replica resides.
+      FsVolumeSpi volume = null;
+      for (FsVolumeSpi fsVolumeReference :
+          fsDataSetImpl.getFsVolumeReferences()) {
+        if (!fsVolumeReference.getStorageID()
+            .equals(newReplicaInfo.getStorageUuid())) {
+          volume = fsVolumeReference;
+        }
+      }
+
+      // Assert metrics.
+      DataNodeVolumeMetrics metrics = volume.getMetrics();
+      assertEquals(2, metrics.getTransferIoSampleCount());
+      assertEquals(3, metrics.getTransferIoQuantiles().length);
+      assertEquals(2, metrics.getNativeCopyIoSampleCount());
+      assertEquals(3, metrics.getNativeCopyIoQuantiles().length);
+    }
+  }
 }