Skip to content

Commit 3358db1

Browse files
committed
HBASE-27534 Determine too large requests by response block size rather than just cell size (#5007)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
1 parent 8809c88 commit 3358db1

File tree

13 files changed

+294
-44
lines changed

13 files changed

+294
-44
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/OnlineLogRecord.java

Lines changed: 35 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@ final public class OnlineLogRecord extends LogEntry {
5959
private final int processingTime;
6060
private final int queueTime;
6161
private final long responseSize;
62+
private final long blockBytesScanned;
6263
private final String clientAddress;
6364
private final String serverClass;
6465
private final String methodName;
@@ -88,6 +89,13 @@ public long getResponseSize() {
8889
return responseSize;
8990
}
9091

92+
/**
93+
* Return the amount of block bytes scanned to retrieve the response cells.
94+
*/
95+
public long getBlockBytesScanned() {
96+
return blockBytesScanned;
97+
}
98+
9199
public String getClientAddress() {
92100
return clientAddress;
93101
}
@@ -129,14 +137,15 @@ public int getMultiServiceCalls() {
129137
}
130138

131139
private OnlineLogRecord(final long startTime, final int processingTime, final int queueTime,
132-
final long responseSize, final String clientAddress, final String serverClass,
133-
final String methodName, final String callDetails, final String param, final String regionName,
134-
final String userName, final int multiGetsCount, final int multiMutationsCount,
135-
final int multiServiceCalls) {
140+
final long responseSize, final long blockBytesScanned, final String clientAddress,
141+
final String serverClass, final String methodName, final String callDetails, final String param,
142+
final String regionName, final String userName, final int multiGetsCount,
143+
final int multiMutationsCount, final int multiServiceCalls) {
136144
this.startTime = startTime;
137145
this.processingTime = processingTime;
138146
this.queueTime = queueTime;
139147
this.responseSize = responseSize;
148+
this.blockBytesScanned = blockBytesScanned;
140149
this.clientAddress = clientAddress;
141150
this.serverClass = serverClass;
142151
this.methodName = methodName;
@@ -154,6 +163,7 @@ public static class OnlineLogRecordBuilder {
154163
private int processingTime;
155164
private int queueTime;
156165
private long responseSize;
166+
private long blockBytesScanned;
157167
private String clientAddress;
158168
private String serverClass;
159169
private String methodName;
@@ -185,6 +195,14 @@ public OnlineLogRecordBuilder setResponseSize(long responseSize) {
185195
return this;
186196
}
187197

198+
/**
199+
* Sets the amount of block bytes scanned to retrieve the response cells.
200+
*/
201+
public OnlineLogRecordBuilder setBlockBytesScanned(long blockBytesScanned) {
202+
this.blockBytesScanned = blockBytesScanned;
203+
return this;
204+
}
205+
188206
public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
189207
this.clientAddress = clientAddress;
190208
return this;
@@ -236,9 +254,9 @@ public OnlineLogRecordBuilder setMultiServiceCalls(int multiServiceCalls) {
236254
}
237255

238256
public OnlineLogRecord build() {
239-
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize, clientAddress,
240-
serverClass, methodName, callDetails, param, regionName, userName, multiGetsCount,
241-
multiMutationsCount, multiServiceCalls);
257+
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
258+
blockBytesScanned, clientAddress, serverClass, methodName, callDetails, param, regionName,
259+
userName, multiGetsCount, multiMutationsCount, multiServiceCalls);
242260
}
243261
}
244262

@@ -256,7 +274,8 @@ public boolean equals(Object o) {
256274

257275
return new EqualsBuilder().append(startTime, that.startTime)
258276
.append(processingTime, that.processingTime).append(queueTime, that.queueTime)
259-
.append(responseSize, that.responseSize).append(multiGetsCount, that.multiGetsCount)
277+
.append(responseSize, that.responseSize).append(blockBytesScanned, that.blockBytesScanned)
278+
.append(multiGetsCount, that.multiGetsCount)
260279
.append(multiMutationsCount, that.multiMutationsCount)
261280
.append(multiServiceCalls, that.multiServiceCalls).append(clientAddress, that.clientAddress)
262281
.append(serverClass, that.serverClass).append(methodName, that.methodName)
@@ -267,9 +286,9 @@ public boolean equals(Object o) {
267286
@Override
268287
public int hashCode() {
269288
return new HashCodeBuilder(17, 37).append(startTime).append(processingTime).append(queueTime)
270-
.append(responseSize).append(clientAddress).append(serverClass).append(methodName)
271-
.append(callDetails).append(param).append(regionName).append(userName).append(multiGetsCount)
272-
.append(multiMutationsCount).append(multiServiceCalls).toHashCode();
289+
.append(responseSize).append(blockBytesScanned).append(clientAddress).append(serverClass)
290+
.append(methodName).append(callDetails).append(param).append(regionName).append(userName)
291+
.append(multiGetsCount).append(multiMutationsCount).append(multiServiceCalls).toHashCode();
273292
}
274293

275294
@Override
@@ -281,11 +300,11 @@ public String toJsonPrettyPrint() {
281300
public String toString() {
282301
return new ToStringBuilder(this).append("startTime", startTime)
283302
.append("processingTime", processingTime).append("queueTime", queueTime)
284-
.append("responseSize", responseSize).append("clientAddress", clientAddress)
285-
.append("serverClass", serverClass).append("methodName", methodName)
286-
.append("callDetails", callDetails).append("param", param).append("regionName", regionName)
287-
.append("userName", userName).append("multiGetsCount", multiGetsCount)
288-
.append("multiMutationsCount", multiMutationsCount)
303+
.append("responseSize", responseSize).append("blockBytesScanned", blockBytesScanned)
304+
.append("clientAddress", clientAddress).append("serverClass", serverClass)
305+
.append("methodName", methodName).append("callDetails", callDetails).append("param", param)
306+
.append("regionName", regionName).append("userName", userName)
307+
.append("multiGetsCount", multiGetsCount).append("multiMutationsCount", multiMutationsCount)
289308
.append("multiServiceCalls", multiServiceCalls).toString();
290309
}
291310

hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3334,6 +3334,7 @@ private static LogEntry getSlowLogRecord(final TooSlowLog.SlowLogPayload slowLog
33343334
.setParam(slowLogPayload.getParam()).setProcessingTime(slowLogPayload.getProcessingTime())
33353335
.setQueueTime(slowLogPayload.getQueueTime()).setRegionName(slowLogPayload.getRegionName())
33363336
.setResponseSize(slowLogPayload.getResponseSize())
3337+
.setBlockBytesScanned(slowLogPayload.getBlockBytesScanned())
33373338
.setServerClass(slowLogPayload.getServerClass()).setStartTime(slowLogPayload.getStartTime())
33383339
.setUserName(slowLogPayload.getUserName()).build();
33393340
return onlineLogRecord;

hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -87,6 +87,8 @@ public static void addSlowLogRecords(final List<TooSlowLog.SlowLogPayload> slowL
8787
Bytes.toBytes(slowLogPayload.getRegionName()))
8888
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("response_size"),
8989
Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize())))
90+
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("block_bytes_scanned"),
91+
Bytes.toBytes(Long.toString(slowLogPayload.getBlockBytesScanned())))
9092
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("server_class"),
9193
Bytes.toBytes(slowLogPayload.getServerClass()))
9294
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),

hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,8 @@ message SlowLogPayload {
4444
optional int32 multi_service_calls = 14 [default = 0];
4545
required Type type = 15;
4646

47+
optional int64 block_bytes_scanned = 16;
48+
4749
// SLOW_LOG is RPC call slow in nature whereas LARGE_LOG is RPC call quite large.
4850
// Majority of times, slow logs are also large logs and hence, ALL is combination of
4951
// both

hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java

Lines changed: 20 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -428,6 +428,7 @@ public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)
428428
// Use the raw request call size for now.
429429
long requestSize = call.getSize();
430430
long responseSize = result.getSerializedSize();
431+
long responseBlockSize = call.getResponseBlockSize();
431432
if (call.isClientCellBlockSupported()) {
432433
// Include the payload size in HBaseRpcController
433434
responseSize += call.getResponseCellSize();
@@ -441,20 +442,21 @@ public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)
441442
// log any RPC responses that are slower than the configured warn
442443
// response time or larger than configured warning size
443444
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
444-
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
445+
boolean tooLarge = (warnResponseSize > -1
446+
&& (responseSize > warnResponseSize || responseBlockSize > warnResponseSize));
445447
if (tooSlow || tooLarge) {
446448
final String userName = call.getRequestUserName().orElse(StringUtils.EMPTY);
447449
// when tagging, we let TooLarge trump TooSmall to keep output simple
448450
// note that large responses will often also be slow.
449451
logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
450452
tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize,
451-
userName);
453+
responseBlockSize, userName);
452454
if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) {
453455
// send logs to ring buffer owned by slowLogRecorder
454456
final String className =
455457
server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
456458
this.namedQueueRecorder.addRecord(new RpcLogDetails(call, param, status.getClient(),
457-
responseSize, className, tooSlow, tooLarge));
459+
responseSize, responseBlockSize, className, tooSlow, tooLarge));
458460
}
459461
}
460462
return new Pair<>(result, controller.cellScanner());
@@ -482,29 +484,31 @@ public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)
482484

483485
/**
484486
* Logs an RPC response to the LOG file, producing valid JSON objects for client Operations.
485-
* @param param The parameters received in the call.
486-
* @param methodName The name of the method invoked
487-
* @param call The string representation of the call
488-
* @param tooLarge To indicate if the event is tooLarge
489-
* @param tooSlow To indicate if the event is tooSlow
490-
* @param clientAddress The address of the client who made this call.
491-
* @param startTime The time that the call was initiated, in ms.
492-
* @param processingTime The duration that the call took to run, in ms.
493-
* @param qTime The duration that the call spent on the queue prior to being initiated,
494-
* in ms.
495-
* @param responseSize The size in bytes of the response buffer.
496-
* @param userName UserName of the current RPC Call
487+
* @param param The parameters received in the call.
488+
* @param methodName The name of the method invoked
489+
* @param call The string representation of the call
490+
* @param tooLarge To indicate if the event is tooLarge
491+
* @param tooSlow To indicate if the event is tooSlow
492+
* @param clientAddress The address of the client who made this call.
493+
* @param startTime The time that the call was initiated, in ms.
494+
* @param processingTime The duration that the call took to run, in ms.
495+
* @param qTime The duration that the call spent on the queue prior to being
496+
* initiated, in ms.
497+
* @param responseSize The size in bytes of the response buffer.
498+
* @param blockBytesScanned The size of block bytes scanned to retrieve the response.
499+
* @param userName UserName of the current RPC Call
497500
*/
498501
void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow,
499502
String clientAddress, long startTime, int processingTime, int qTime, long responseSize,
500-
String userName) {
503+
long blockBytesScanned, String userName) {
501504
final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
502505
// base information that is reported regardless of type of call
503506
Map<String, Object> responseInfo = new HashMap<>();
504507
responseInfo.put("starttimems", startTime);
505508
responseInfo.put("processingtimems", processingTime);
506509
responseInfo.put("queuetimems", qTime);
507510
responseInfo.put("responsesize", responseSize);
511+
responseInfo.put("blockbytesscanned", blockBytesScanned);
508512
responseInfo.put("client", clientAddress);
509513
responseInfo.put("class", className);
510514
responseInfo.put("method", methodName);

hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/RpcLogDetails.java

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,17 +35,19 @@ public class RpcLogDetails extends NamedQueuePayload {
3535
private final Message param;
3636
private final String clientAddress;
3737
private final long responseSize;
38+
private final long blockBytesScanned;
3839
private final String className;
3940
private final boolean isSlowLog;
4041
private final boolean isLargeLog;
4142

4243
public RpcLogDetails(RpcCall rpcCall, Message param, String clientAddress, long responseSize,
43-
String className, boolean isSlowLog, boolean isLargeLog) {
44+
long blockBytesScanned, String className, boolean isSlowLog, boolean isLargeLog) {
4445
super(SLOW_LOG_EVENT);
4546
this.rpcCall = rpcCall;
4647
this.param = param;
4748
this.clientAddress = clientAddress;
4849
this.responseSize = responseSize;
50+
this.blockBytesScanned = blockBytesScanned;
4951
this.className = className;
5052
this.isSlowLog = isSlowLog;
5153
this.isLargeLog = isLargeLog;
@@ -63,6 +65,10 @@ public long getResponseSize() {
6365
return responseSize;
6466
}
6567

68+
public long getBlockBytesScanned() {
69+
return blockBytesScanned;
70+
}
71+
6672
public String getClassName() {
6773
return className;
6874
}

hbase-server/src/main/java/org/apache/hadoop/hbase/namequeues/impl/SlowLogQueueService.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,7 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
116116
final RpcCall rpcCall = rpcLogDetails.getRpcCall();
117117
final String clientAddress = rpcLogDetails.getClientAddress();
118118
final long responseSize = rpcLogDetails.getResponseSize();
119+
final long blockBytesScanned = rpcLogDetails.getBlockBytesScanned();
119120
final String className = rpcLogDetails.getClassName();
120121
final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails);
121122
if (type == null) {
@@ -158,8 +159,9 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
158159
.setParam(slowLogParams != null ? slowLogParams.getParams() : StringUtils.EMPTY)
159160
.setProcessingTime(processingTime).setQueueTime(qTime)
160161
.setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY)
161-
.setResponseSize(responseSize).setServerClass(className).setStartTime(startTime).setType(type)
162-
.setUserName(userName).build();
162+
.setResponseSize(responseSize).setBlockBytesScanned(blockBytesScanned)
163+
.setServerClass(className).setStartTime(startTime).setType(type).setUserName(userName)
164+
.build();
163165
slowLogQueue.add(slowLogPayload);
164166
if (isSlowLogTableEnabled) {
165167
if (!slowLogPayload.getRegionName().startsWith("hbase:slowlog")) {

hbase-server/src/main/resources/hbase-webapps/regionserver/rsOperationDetails.jsp

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -97,6 +97,7 @@
9797
<th>Processing Time</th>
9898
<th>Queue Time</th>
9999
<th>Response Size</th>
100+
<th>Block Bytes Scanned</th>
100101
<th>Client Address</th>
101102
<th>Server Class</th>
102103
<th>Method Name</th>
@@ -115,6 +116,7 @@
115116
<td><%=r.getProcessingTime()%>ms</td>
116117
<td><%=r.getQueueTime()%>ms</td>
117118
<td><%=StringUtils.byteDesc(r.getResponseSize())%></td>
119+
<td><%=StringUtils.byteDesc(r.getBlockBytesScanned())%></td>
118120
<td><%=r.getClientAddress()%></td>
119121
<td><%=r.getServerClass()%></td>
120122
<td><%=r.getMethodName()%></td>
@@ -138,6 +140,7 @@
138140
<th>Processing Time</th>
139141
<th>Queue Time</th>
140142
<th>Response Size</th>
143+
<th>Block Bytes Scanned</th>
141144
<th>Client Address</th>
142145
<th>Server Class</th>
143146
<th>Method Name</th>
@@ -156,6 +159,7 @@
156159
<td><%=r.getProcessingTime()%>ms</td>
157160
<td><%=r.getQueueTime()%>ms</td>
158161
<td><%=StringUtils.byteDesc(r.getResponseSize())%></td>
162+
<td><%=StringUtils.byteDesc(r.getBlockBytesScanned())%></td>
159163
<td><%=r.getClientAddress()%></td>
160164
<td><%=r.getServerClass()%></td>
161165
<td><%=r.getMethodName()%></td>

hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -529,13 +529,14 @@ public void testSlowLogMixedFilters() throws Exception {
529529

530530
static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className) {
531531
RpcCall rpcCall = getRpcCall(userName);
532-
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, className, true, true);
532+
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true,
533+
true);
533534
}
534535

535536
private RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className,
536537
boolean isSlowLog, boolean isLargeLog) {
537538
RpcCall rpcCall = getRpcCall(userName);
538-
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, className, isSlowLog,
539+
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, isSlowLog,
539540
isLargeLog);
540541
}
541542

0 commit comments

Comments
 (0)