This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 1e785c599b7 HDDS-14219. Add metrics for ReadBlock (#9753)
1e785c599b7 is described below

commit 1e785c599b7ccd7359125061eefe56e66feca9fa
Author: Sergey Soldatov <[email protected]>
AuthorDate: Tue Feb 17 04:55:29 2026 -0800

    HDDS-14219. Add metrics for ReadBlock (#9753)
    
    Co-authored-by: rich7420 <[email protected]>
---
 .../container/common/impl/HddsDispatcher.java      |   3 +
 .../container/common/volume/VolumeIOStats.java     |  13 ++
 .../ozone/container/keyvalue/KeyValueHandler.java  |  16 ++-
 .../container/keyvalue/helpers/ChunkUtils.java     |   5 +-
 .../container/keyvalue/TestKeyValueHandler.java    | 148 +++++++++++++++++++++
 .../container/metrics/TestContainerMetrics.java    |   1 +
 6 files changed, 178 insertions(+), 8 deletions(-)

diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index d8a32c677a6..12fed16b364 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -839,6 +839,9 @@ public void streamDataReadOnly(ContainerCommandRequestProto 
msg,
       Container container = getContainer(containerID);
       long startTime = Time.monotonicNow();
 
+      // Increment operation count metrics
+      metrics.incContainerOpsMetrics(cmdType);
+
       if (DispatcherContext.op(dispatcherContext).validateToken()) {
         validateToken(msg);
       }
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
index d7d4bcb63ce..fd73316d5aa 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java
@@ -133,6 +133,19 @@ public void incReadTime(long time) {
     }
   }
 
+  /**
+   * Record a read operation with timing and bytes read.
+   * This is a convenience method that updates read time, operation count, and 
bytes read.
+   * @param startTime the start time of the read operation in milliseconds
+   * @param bytesRead the number of bytes read
+   */
+  public void recordReadOperation(long startTime, long bytesRead) {
+    long endTime = org.apache.hadoop.util.Time.monotonicNow();
+    incReadTime(endTime - startTime);
+    incReadOpCount();
+    incReadBytes(bytesRead);
+  }
+
   /**
    * Increment the time taken by write operation on the volume.
    * @param time
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 13bfa834dbc..ef598f3c0cb 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -2091,8 +2091,15 @@ public ContainerCommandResponseProto readBlock(
       return malformedRequest(request);
     }
     try {
-      readBlockImpl(request, blockFile, kvContainer, streamObserver, false);
-      // TODO metrics.incContainerBytesStats(Type.ReadBlock, 
readBlock.getLen());
+      final long startTime = Time.monotonicNow();
+      final long bytesRead = readBlockImpl(request, blockFile, kvContainer, 
streamObserver, false);
+      KeyValueContainerData containerData = (KeyValueContainerData) kvContainer
+          .getContainerData();
+      HddsVolume volume = containerData.getVolume();
+      if (volume != null) {
+        volume.getVolumeIOStats().recordReadOperation(startTime, bytesRead);
+      }
+      metrics.incContainerBytesStats(Type.ReadBlock, bytesRead);
     } catch (StorageContainerException ex) {
       responseProto = ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ioe) {
@@ -2108,7 +2115,7 @@ public ContainerCommandResponseProto readBlock(
     return responseProto;
   }
 
-  private void readBlockImpl(ContainerCommandRequestProto request, 
RandomAccessFileChannel blockFile,
+  private long readBlockImpl(ContainerCommandRequestProto request, 
RandomAccessFileChannel blockFile,
       Container kvContainer, StreamObserver<ContainerCommandResponseProto> 
streamObserver, boolean verifyChecksum)
       throws IOException {
     final ReadBlockRequestProto readBlock = request.getReadBlock();
@@ -2148,7 +2155,7 @@ private void readBlockImpl(ContainerCommandRequestProto 
request, RandomAccessFil
 
     final ByteBuffer buffer = ByteBuffer.allocate(responseDataSize);
     blockFile.position(adjustedOffset);
-    int totalDataLength = 0;
+    long totalDataLength = 0;
     int numResponses = 0;
     final long rounded = roundUp(readBlock.getLength() + offsetAlignment, 
bytesPerChecksum);
     final long requiredLength = Math.min(rounded, blockData.getSize() - 
adjustedOffset);
@@ -2186,6 +2193,7 @@ private void readBlockImpl(ContainerCommandRequestProto 
request, RandomAccessFil
       totalDataLength += dataLength;
       numResponses++;
     }
+    return totalDataLength;
   }
 
   static List<ByteString> getChecksums(long blockOffset, int readLength, int 
bytesPerChunk, int bytesPerChecksum,
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
index 7e9fb0b0d00..9cde9ba315e 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java
@@ -233,11 +233,8 @@ private static void readData(File file, long offset, long 
len,
     }
 
     // Increment volumeIO stats here.
-    long endTime = Time.monotonicNow();
     if (volume != null) {
-      volume.getVolumeIOStats().incReadTime(endTime - startTime);
-      volume.getVolumeIOStats().incReadOpCount();
-      volume.getVolumeIOStats().incReadBytes(bytesRead);
+      volume.getVolumeIOStats().recordReadOperation(startTime, bytesRead);
     }
 
     LOG.debug("Read {} bytes starting at offset {} from {}",
diff --git 
a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
 
b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
index 6afee1c5d77..0385564ebf7 100644
--- 
a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
+++ 
b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java
@@ -30,6 +30,8 @@
 import static 
org.apache.hadoop.ozone.container.checksum.ContainerMerkleTreeTestUtils.verifyAllDataChecksumsMatch;
 import static 
org.apache.hadoop.ozone.container.common.ContainerTestUtils.createBlockMetaData;
 import static 
org.apache.hadoop.ozone.container.common.impl.ContainerImplTestUtils.newContainerSet;
+import static org.apache.ozone.test.MetricsAsserts.assertCounter;
+import static org.apache.ozone.test.MetricsAsserts.getMetrics;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -37,6 +39,7 @@
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.eq;
@@ -53,6 +56,7 @@
 import com.google.common.collect.Sets;
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.time.Clock;
@@ -68,11 +72,13 @@
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
+import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State;
 import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType;
 import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
@@ -80,10 +86,16 @@
 import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.security.token.TokenVerifier;
+import org.apache.hadoop.hdds.utils.io.RandomAccessFileChannel;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager;
 import org.apache.hadoop.ozone.container.checksum.ContainerMerkleTreeWriter;
 import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient;
 import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.ContainerImplTestUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
@@ -94,6 +106,7 @@
 import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender;
 import 
org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
 import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import 
org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
 import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
@@ -106,6 +119,7 @@
 import org.apache.hadoop.util.Time;
 import org.apache.ozone.test.GenericTestUtils;
 import org.apache.ozone.test.GenericTestUtils.LogCapturer;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -938,4 +952,138 @@ private KeyValueHandler createKeyValueHandler(Path path) 
throws IOException {
 
     return kvHandler;
   }
+
+  private static class HandlerWithVolumeSet {
+    private final KeyValueHandler handler;
+    private final MutableVolumeSet volumeSet;
+    private final ContainerSet containerSet;
+
+    HandlerWithVolumeSet(KeyValueHandler handler, MutableVolumeSet volumeSet, 
ContainerSet containerSet) {
+      this.handler = handler;
+      this.volumeSet = volumeSet;
+      this.containerSet = containerSet;
+    }
+
+    KeyValueHandler getHandler() {
+      return handler;
+    }
+
+    MutableVolumeSet getVolumeSet() {
+      return volumeSet;
+    }
+
+    ContainerSet getContainerSet() {
+      return containerSet;
+    }
+  }
+
+  private HandlerWithVolumeSet createKeyValueHandlerWithVolumeSet(Path path) 
throws IOException {
+    ContainerMetrics.remove();
+    final ContainerSet containerSet = newContainerSet();
+    final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class);
+
+    HddsVolume hddsVolume = new HddsVolume.Builder(path.toString()).conf(conf)
+        .clusterID(CLUSTER_ID).datanodeUuid(DATANODE_UUID)
+        .volumeSet(volumeSet)
+        .build();
+    hddsVolume.format(CLUSTER_ID);
+    hddsVolume.createWorkingDir(CLUSTER_ID, null);
+    hddsVolume.createTmpDirs(CLUSTER_ID);
+    
when(volumeSet.getVolumesList()).thenReturn(Collections.singletonList(hddsVolume));
+
+    final KeyValueHandler kvHandler = 
ContainerTestUtils.getKeyValueHandler(conf,
+        DATANODE_UUID, containerSet, volumeSet);
+    kvHandler.setClusterID(CLUSTER_ID);
+    hddsVolume.getVolumeInfoStats().unregister();
+    hddsVolume.getVolumeIOStats().unregister();
+
+    ContainerController controller = new ContainerController(containerSet,
+        Collections.singletonMap(ContainerType.KeyValueContainer, kvHandler));
+    OnDemandContainerScanner onDemandScanner = new OnDemandContainerScanner(
+        conf.getObject(ContainerScannerConfiguration.class), controller);
+    containerSet.registerOnDemandScanner(onDemandScanner);
+
+    return new HandlerWithVolumeSet(kvHandler, volumeSet, containerSet);
+  }
+
+  @Test
+  public void testReadBlockMetrics() throws Exception {
+    Path testDir = Files.createTempDirectory("testReadBlockMetrics");
+    RandomAccessFileChannel blockFile = null;
+    try {
+      conf.set(OZONE_SCM_CONTAINER_LAYOUT_KEY, 
ContainerLayoutVersion.FILE_PER_BLOCK.name());
+      HandlerWithVolumeSet handlerWithVolume = 
createKeyValueHandlerWithVolumeSet(testDir);
+      KeyValueHandler kvHandler = handlerWithVolume.getHandler();
+      MutableVolumeSet volumeSet = handlerWithVolume.getVolumeSet();
+      ContainerSet containerSet = handlerWithVolume.getContainerSet();
+
+      long containerID = ContainerTestHelper.getTestContainerID();
+      KeyValueContainerData containerData = new KeyValueContainerData(
+          containerID, ContainerLayoutVersion.FILE_PER_BLOCK,
+          (long) StorageUnit.GB.toBytes(1), UUID.randomUUID().toString(),
+          DATANODE_UUID);
+      KeyValueContainer container = new KeyValueContainer(containerData, conf);
+      container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), 
CLUSTER_ID);
+      containerSet.addContainer(container);
+
+      BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+      BlockData blockData = new BlockData(blockID);
+      ChunkInfo chunkInfo = new ChunkInfo("chunk1", 0, 1024);
+      blockData.addChunk(chunkInfo.getProtoBufMessage());
+      kvHandler.getBlockManager().putBlock(container, blockData);
+
+      ChunkBuffer data = ChunkBuffer.wrap(ByteBuffer.allocate(1024));
+      kvHandler.getChunkManager().writeChunk(container, blockID, chunkInfo, 
data,
+          DispatcherContext.getHandleWriteChunk());
+
+      ContainerCommandRequestProto readBlockRequest =
+          ContainerCommandRequestProto.newBuilder()
+              .setCmdType(ContainerProtos.Type.ReadBlock)
+              .setContainerID(containerID)
+              .setDatanodeUuid(DATANODE_UUID)
+              .setReadBlock(ContainerProtos.ReadBlockRequestProto.newBuilder()
+                  .setBlockID(blockID.getDatanodeBlockIDProtobuf())
+                  .setOffset(0)
+                  .setLength(1024)
+                  .build())
+              .build();
+
+      final AtomicInteger responseCount = new AtomicInteger(0);
+
+      StreamObserver<ContainerCommandResponseProto> streamObserver =
+          new StreamObserver<ContainerCommandResponseProto>() {
+            @Override
+            public void onNext(ContainerCommandResponseProto response) {
+              assertEquals(ContainerProtos.Result.SUCCESS, 
response.getResult());
+              responseCount.incrementAndGet();
+            }
+
+            @Override
+            public void onError(Throwable t) {
+              fail("ReadBlock failed", t);
+            }
+
+            @Override
+            public void onCompleted() {
+            }
+          };
+
+      blockFile = new RandomAccessFileChannel();
+      ContainerCommandResponseProto response = kvHandler.readBlock(
+          readBlockRequest, container, blockFile, streamObserver);
+
+      assertNull(response, "ReadBlock should return null on success");
+      assertTrue(responseCount.get() > 0, "Should receive at least one 
response");
+
+      MetricsRecordBuilder containerMetrics = getMetrics(
+          ContainerMetrics.STORAGE_CONTAINER_METRICS);
+      assertCounter("bytesReadBlock", 1024L, containerMetrics);
+    } finally {
+      if (blockFile != null) {
+        blockFile.close();
+      }
+      FileUtils.deleteDirectory(testDir.toFile());
+      ContainerMetrics.remove();
+    }
+  }
 }
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
index 2f64167e02c..845ee1a3f9e 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
@@ -207,6 +207,7 @@ static void runTestClientServer(
       assertCounter("numReadChunk", 1L, containerMetrics);
       assertCounter("bytesWriteChunk", 1024L, containerMetrics);
       assertCounter("bytesReadChunk", 1024L, containerMetrics);
+      // bytesReadBlock is tested in TestKeyValueHandler.testReadBlockMetrics
 
       String sec = DFS_METRICS_PERCENTILES_INTERVALS + "s";
       Thread.sleep((DFS_METRICS_PERCENTILES_INTERVALS + 1) * 1000);


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to