Skip to content

Commit 1e785c5

Browse files
ss77892rich7420
andauthored
HDDS-14219. Add metrics for ReadBlock (#9753)
Co-authored-by: rich7420 <rc910420@gmail.com>
1 parent 5331a6b commit 1e785c5

6 files changed

Lines changed: 178 additions & 8 deletions

File tree

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -839,6 +839,9 @@ public void streamDataReadOnly(ContainerCommandRequestProto msg,
839839
Container container = getContainer(containerID);
840840
long startTime = Time.monotonicNow();
841841

842+
// Increment operation count metrics
843+
metrics.incContainerOpsMetrics(cmdType);
844+
842845
if (DispatcherContext.op(dispatcherContext).validateToken()) {
843846
validateToken(msg);
844847
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeIOStats.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -133,6 +133,19 @@ public void incReadTime(long time) {
133133
}
134134
}
135135

136+
/**
137+
* Record a read operation with timing and bytes read.
138+
* This is a convenience method that updates read time, operation count, and bytes read.
139+
* @param startTime the start time of the read operation in milliseconds
140+
* @param bytesRead the number of bytes read
141+
*/
142+
public void recordReadOperation(long startTime, long bytesRead) {
143+
long endTime = org.apache.hadoop.util.Time.monotonicNow();
144+
incReadTime(endTime - startTime);
145+
incReadOpCount();
146+
incReadBytes(bytesRead);
147+
}
148+
136149
/**
137150
* Increment the time taken by write operation on the volume.
138151
* @param time

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java

Lines changed: 12 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -2091,8 +2091,15 @@ public ContainerCommandResponseProto readBlock(
20912091
return malformedRequest(request);
20922092
}
20932093
try {
2094-
readBlockImpl(request, blockFile, kvContainer, streamObserver, false);
2095-
// TODO metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen());
2094+
final long startTime = Time.monotonicNow();
2095+
final long bytesRead = readBlockImpl(request, blockFile, kvContainer, streamObserver, false);
2096+
KeyValueContainerData containerData = (KeyValueContainerData) kvContainer
2097+
.getContainerData();
2098+
HddsVolume volume = containerData.getVolume();
2099+
if (volume != null) {
2100+
volume.getVolumeIOStats().recordReadOperation(startTime, bytesRead);
2101+
}
2102+
metrics.incContainerBytesStats(Type.ReadBlock, bytesRead);
20962103
} catch (StorageContainerException ex) {
20972104
responseProto = ContainerUtils.logAndReturnError(LOG, ex, request);
20982105
} catch (IOException ioe) {
@@ -2108,7 +2115,7 @@ public ContainerCommandResponseProto readBlock(
21082115
return responseProto;
21092116
}
21102117

2111-
private void readBlockImpl(ContainerCommandRequestProto request, RandomAccessFileChannel blockFile,
2118+
private long readBlockImpl(ContainerCommandRequestProto request, RandomAccessFileChannel blockFile,
21122119
Container kvContainer, StreamObserver<ContainerCommandResponseProto> streamObserver, boolean verifyChecksum)
21132120
throws IOException {
21142121
final ReadBlockRequestProto readBlock = request.getReadBlock();
@@ -2148,7 +2155,7 @@ private void readBlockImpl(ContainerCommandRequestProto request, RandomAccessFil
21482155

21492156
final ByteBuffer buffer = ByteBuffer.allocate(responseDataSize);
21502157
blockFile.position(adjustedOffset);
2151-
int totalDataLength = 0;
2158+
long totalDataLength = 0;
21522159
int numResponses = 0;
21532160
final long rounded = roundUp(readBlock.getLength() + offsetAlignment, bytesPerChecksum);
21542161
final long requiredLength = Math.min(rounded, blockData.getSize() - adjustedOffset);
@@ -2186,6 +2193,7 @@ private void readBlockImpl(ContainerCommandRequestProto request, RandomAccessFil
21862193
totalDataLength += dataLength;
21872194
numResponses++;
21882195
}
2196+
return totalDataLength;
21892197
}
21902198

21912199
static List<ByteString> getChecksums(long blockOffset, int readLength, int bytesPerChunk, int bytesPerChecksum,

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/ChunkUtils.java

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -233,11 +233,8 @@ private static void readData(File file, long offset, long len,
233233
}
234234

235235
// Increment volumeIO stats here.
236-
long endTime = Time.monotonicNow();
237236
if (volume != null) {
238-
volume.getVolumeIOStats().incReadTime(endTime - startTime);
239-
volume.getVolumeIOStats().incReadOpCount();
240-
volume.getVolumeIOStats().incReadBytes(bytesRead);
237+
volume.getVolumeIOStats().recordReadOperation(startTime, bytesRead);
241238
}
242239

243240
LOG.debug("Read {} bytes starting at offset {} from {}",

hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java

Lines changed: 148 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,13 +30,16 @@
3030
import static org.apache.hadoop.ozone.container.checksum.ContainerMerkleTreeTestUtils.verifyAllDataChecksumsMatch;
3131
import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createBlockMetaData;
3232
import static org.apache.hadoop.ozone.container.common.impl.ContainerImplTestUtils.newContainerSet;
33+
import static org.apache.ozone.test.MetricsAsserts.assertCounter;
34+
import static org.apache.ozone.test.MetricsAsserts.getMetrics;
3335
import static org.assertj.core.api.Assertions.assertThat;
3436
import static org.junit.jupiter.api.Assertions.assertEquals;
3537
import static org.junit.jupiter.api.Assertions.assertFalse;
3638
import static org.junit.jupiter.api.Assertions.assertNotEquals;
3739
import static org.junit.jupiter.api.Assertions.assertNotNull;
3840
import static org.junit.jupiter.api.Assertions.assertNull;
3941
import static org.junit.jupiter.api.Assertions.assertTrue;
42+
import static org.junit.jupiter.api.Assertions.fail;
4043
import static org.mockito.ArgumentMatchers.any;
4144
import static org.mockito.ArgumentMatchers.anyLong;
4245
import static org.mockito.ArgumentMatchers.eq;
@@ -53,6 +56,7 @@
5356
import com.google.common.collect.Sets;
5457
import java.io.File;
5558
import java.io.IOException;
59+
import java.nio.ByteBuffer;
5660
import java.nio.file.Files;
5761
import java.nio.file.Path;
5862
import java.time.Clock;
@@ -68,22 +72,30 @@
6872
import org.apache.commons.io.FileUtils;
6973
import org.apache.hadoop.conf.StorageUnit;
7074
import org.apache.hadoop.fs.FileUtil;
75+
import org.apache.hadoop.hdds.client.BlockID;
7176
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
7277
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
7378
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
7479
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
7580
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
81+
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
7682
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerDataProto.State;
7783
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType;
7884
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
7985
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
8086
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
8187
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
8288
import org.apache.hadoop.hdds.security.token.TokenVerifier;
89+
import org.apache.hadoop.hdds.utils.io.RandomAccessFileChannel;
90+
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
91+
import org.apache.hadoop.ozone.common.ChunkBuffer;
92+
import org.apache.hadoop.ozone.container.ContainerTestHelper;
8393
import org.apache.hadoop.ozone.container.checksum.ContainerChecksumTreeManager;
8494
import org.apache.hadoop.ozone.container.checksum.ContainerMerkleTreeWriter;
8595
import org.apache.hadoop.ozone.container.checksum.DNContainerOperationClient;
8696
import org.apache.hadoop.ozone.container.common.ContainerTestUtils;
97+
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
98+
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
8799
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
88100
import org.apache.hadoop.ozone.container.common.impl.ContainerImplTestUtils;
89101
import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
@@ -94,6 +106,7 @@
94106
import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender;
95107
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
96108
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
109+
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
97110
import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
98111
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
99112
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
@@ -106,6 +119,7 @@
106119
import org.apache.hadoop.util.Time;
107120
import org.apache.ozone.test.GenericTestUtils;
108121
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
122+
import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
109123
import org.junit.jupiter.api.Assertions;
110124
import org.junit.jupiter.api.BeforeEach;
111125
import org.junit.jupiter.api.Test;
@@ -938,4 +952,138 @@ private KeyValueHandler createKeyValueHandler(Path path) throws IOException {
938952

939953
return kvHandler;
940954
}
955+
956+
private static class HandlerWithVolumeSet {
957+
private final KeyValueHandler handler;
958+
private final MutableVolumeSet volumeSet;
959+
private final ContainerSet containerSet;
960+
961+
HandlerWithVolumeSet(KeyValueHandler handler, MutableVolumeSet volumeSet, ContainerSet containerSet) {
962+
this.handler = handler;
963+
this.volumeSet = volumeSet;
964+
this.containerSet = containerSet;
965+
}
966+
967+
KeyValueHandler getHandler() {
968+
return handler;
969+
}
970+
971+
MutableVolumeSet getVolumeSet() {
972+
return volumeSet;
973+
}
974+
975+
ContainerSet getContainerSet() {
976+
return containerSet;
977+
}
978+
}
979+
980+
private HandlerWithVolumeSet createKeyValueHandlerWithVolumeSet(Path path) throws IOException {
981+
ContainerMetrics.remove();
982+
final ContainerSet containerSet = newContainerSet();
983+
final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class);
984+
985+
HddsVolume hddsVolume = new HddsVolume.Builder(path.toString()).conf(conf)
986+
.clusterID(CLUSTER_ID).datanodeUuid(DATANODE_UUID)
987+
.volumeSet(volumeSet)
988+
.build();
989+
hddsVolume.format(CLUSTER_ID);
990+
hddsVolume.createWorkingDir(CLUSTER_ID, null);
991+
hddsVolume.createTmpDirs(CLUSTER_ID);
992+
when(volumeSet.getVolumesList()).thenReturn(Collections.singletonList(hddsVolume));
993+
994+
final KeyValueHandler kvHandler = ContainerTestUtils.getKeyValueHandler(conf,
995+
DATANODE_UUID, containerSet, volumeSet);
996+
kvHandler.setClusterID(CLUSTER_ID);
997+
hddsVolume.getVolumeInfoStats().unregister();
998+
hddsVolume.getVolumeIOStats().unregister();
999+
1000+
ContainerController controller = new ContainerController(containerSet,
1001+
Collections.singletonMap(ContainerType.KeyValueContainer, kvHandler));
1002+
OnDemandContainerScanner onDemandScanner = new OnDemandContainerScanner(
1003+
conf.getObject(ContainerScannerConfiguration.class), controller);
1004+
containerSet.registerOnDemandScanner(onDemandScanner);
1005+
1006+
return new HandlerWithVolumeSet(kvHandler, volumeSet, containerSet);
1007+
}
1008+
1009+
@Test
1010+
public void testReadBlockMetrics() throws Exception {
1011+
Path testDir = Files.createTempDirectory("testReadBlockMetrics");
1012+
RandomAccessFileChannel blockFile = null;
1013+
try {
1014+
conf.set(OZONE_SCM_CONTAINER_LAYOUT_KEY, ContainerLayoutVersion.FILE_PER_BLOCK.name());
1015+
HandlerWithVolumeSet handlerWithVolume = createKeyValueHandlerWithVolumeSet(testDir);
1016+
KeyValueHandler kvHandler = handlerWithVolume.getHandler();
1017+
MutableVolumeSet volumeSet = handlerWithVolume.getVolumeSet();
1018+
ContainerSet containerSet = handlerWithVolume.getContainerSet();
1019+
1020+
long containerID = ContainerTestHelper.getTestContainerID();
1021+
KeyValueContainerData containerData = new KeyValueContainerData(
1022+
containerID, ContainerLayoutVersion.FILE_PER_BLOCK,
1023+
(long) StorageUnit.GB.toBytes(1), UUID.randomUUID().toString(),
1024+
DATANODE_UUID);
1025+
KeyValueContainer container = new KeyValueContainer(containerData, conf);
1026+
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), CLUSTER_ID);
1027+
containerSet.addContainer(container);
1028+
1029+
BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
1030+
BlockData blockData = new BlockData(blockID);
1031+
ChunkInfo chunkInfo = new ChunkInfo("chunk1", 0, 1024);
1032+
blockData.addChunk(chunkInfo.getProtoBufMessage());
1033+
kvHandler.getBlockManager().putBlock(container, blockData);
1034+
1035+
ChunkBuffer data = ChunkBuffer.wrap(ByteBuffer.allocate(1024));
1036+
kvHandler.getChunkManager().writeChunk(container, blockID, chunkInfo, data,
1037+
DispatcherContext.getHandleWriteChunk());
1038+
1039+
ContainerCommandRequestProto readBlockRequest =
1040+
ContainerCommandRequestProto.newBuilder()
1041+
.setCmdType(ContainerProtos.Type.ReadBlock)
1042+
.setContainerID(containerID)
1043+
.setDatanodeUuid(DATANODE_UUID)
1044+
.setReadBlock(ContainerProtos.ReadBlockRequestProto.newBuilder()
1045+
.setBlockID(blockID.getDatanodeBlockIDProtobuf())
1046+
.setOffset(0)
1047+
.setLength(1024)
1048+
.build())
1049+
.build();
1050+
1051+
final AtomicInteger responseCount = new AtomicInteger(0);
1052+
1053+
StreamObserver<ContainerCommandResponseProto> streamObserver =
1054+
new StreamObserver<ContainerCommandResponseProto>() {
1055+
@Override
1056+
public void onNext(ContainerCommandResponseProto response) {
1057+
assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
1058+
responseCount.incrementAndGet();
1059+
}
1060+
1061+
@Override
1062+
public void onError(Throwable t) {
1063+
fail("ReadBlock failed", t);
1064+
}
1065+
1066+
@Override
1067+
public void onCompleted() {
1068+
}
1069+
};
1070+
1071+
blockFile = new RandomAccessFileChannel();
1072+
ContainerCommandResponseProto response = kvHandler.readBlock(
1073+
readBlockRequest, container, blockFile, streamObserver);
1074+
1075+
assertNull(response, "ReadBlock should return null on success");
1076+
assertTrue(responseCount.get() > 0, "Should receive at least one response");
1077+
1078+
MetricsRecordBuilder containerMetrics = getMetrics(
1079+
ContainerMetrics.STORAGE_CONTAINER_METRICS);
1080+
assertCounter("bytesReadBlock", 1024L, containerMetrics);
1081+
} finally {
1082+
if (blockFile != null) {
1083+
blockFile.close();
1084+
}
1085+
FileUtils.deleteDirectory(testDir.toFile());
1086+
ContainerMetrics.remove();
1087+
}
1088+
}
9411089
}

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -207,6 +207,7 @@ static void runTestClientServer(
207207
assertCounter("numReadChunk", 1L, containerMetrics);
208208
assertCounter("bytesWriteChunk", 1024L, containerMetrics);
209209
assertCounter("bytesReadChunk", 1024L, containerMetrics);
210+
// bytesReadBlock is tested in TestKeyValueHandler.testReadBlockMetrics
210211

211212
String sec = DFS_METRICS_PERCENTILES_INTERVALS + "s";
212213
Thread.sleep((DFS_METRICS_PERCENTILES_INTERVALS + 1) * 1000);

0 commit comments

Comments
 (0)