From b5d78583fc0d3961e887fb667e3a4ae22bff4af2 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Mon, 23 Oct 2023 14:36:20 -0400 Subject: [PATCH] set request and response sizes in NettyRpcDuplexHandler --- .../java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java index fe32189f81bd..acd419c8d573 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java @@ -110,6 +110,7 @@ private void writeRequest(ChannelHandlerContext ctx, Call call, ChannelPromise p } else { ctx.write(buf, promise); } + call.callStats.setRequestSizeBytes(totalSize); } } @@ -162,6 +163,7 @@ private void readResponse(ChannelHandlerContext ctx, ByteBuf buf) throws IOExcep } return; } + call.callStats.setResponseSizeBytes(totalSize); if (remoteExc != null) { call.setException(remoteExc); return;