Skip to content

Commit

Permalink
HubSpot Backport: HBASE-27534 Determine too large requests by respons…
Browse files Browse the repository at this point in the history
…e block size rather than just cell size (apache#5007)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
  • Loading branch information
bbeaudreault committed Feb 28, 2023
1 parent 047cd16 commit 6ba8b32
Show file tree
Hide file tree
Showing 13 changed files with 310 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ final public class OnlineLogRecord extends LogEntry {
private final int processingTime;
private final int queueTime;
private final long responseSize;
private final long blockBytesScanned;
private final String clientAddress;
private final String serverClass;
private final String methodName;
Expand Down Expand Up @@ -88,6 +89,13 @@ public long getResponseSize() {
return responseSize;
}

/**
* Return the amount of block bytes scanned to retrieve the response cells.
*/
public long getBlockBytesScanned() {
return blockBytesScanned;
}

public String getClientAddress() {
return clientAddress;
}
Expand Down Expand Up @@ -129,14 +137,15 @@ public int getMultiServiceCalls() {
}

private OnlineLogRecord(final long startTime, final int processingTime, final int queueTime,
final long responseSize, final String clientAddress, final String serverClass,
final String methodName, final String callDetails, final String param, final String regionName,
final String userName, final int multiGetsCount, final int multiMutationsCount,
final int multiServiceCalls) {
final long responseSize, final long blockBytesScanned, final String clientAddress,
final String serverClass, final String methodName, final String callDetails, final String param,
final String regionName, final String userName, final int multiGetsCount,
final int multiMutationsCount, final int multiServiceCalls) {
this.startTime = startTime;
this.processingTime = processingTime;
this.queueTime = queueTime;
this.responseSize = responseSize;
this.blockBytesScanned = blockBytesScanned;
this.clientAddress = clientAddress;
this.serverClass = serverClass;
this.methodName = methodName;
Expand All @@ -154,6 +163,7 @@ public static class OnlineLogRecordBuilder {
private int processingTime;
private int queueTime;
private long responseSize;
private long blockBytesScanned;
private String clientAddress;
private String serverClass;
private String methodName;
Expand Down Expand Up @@ -185,6 +195,14 @@ public OnlineLogRecordBuilder setResponseSize(long responseSize) {
return this;
}

/**
* Sets the amount of block bytes scanned to retrieve the response cells.
*/
public OnlineLogRecordBuilder setBlockBytesScanned(long blockBytesScanned) {
this.blockBytesScanned = blockBytesScanned;
return this;
}

public OnlineLogRecordBuilder setClientAddress(String clientAddress) {
this.clientAddress = clientAddress;
return this;
Expand Down Expand Up @@ -236,9 +254,9 @@ public OnlineLogRecordBuilder setMultiServiceCalls(int multiServiceCalls) {
}

public OnlineLogRecord build() {
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize, clientAddress,
serverClass, methodName, callDetails, param, regionName, userName, multiGetsCount,
multiMutationsCount, multiServiceCalls);
return new OnlineLogRecord(startTime, processingTime, queueTime, responseSize,
blockBytesScanned, clientAddress, serverClass, methodName, callDetails, param, regionName,
userName, multiGetsCount, multiMutationsCount, multiServiceCalls);
}
}

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

return new EqualsBuilder().append(startTime, that.startTime)
.append(processingTime, that.processingTime).append(queueTime, that.queueTime)
.append(responseSize, that.responseSize).append(multiGetsCount, that.multiGetsCount)
.append(responseSize, that.responseSize).append(blockBytesScanned, that.blockBytesScanned)
.append(multiGetsCount, that.multiGetsCount)
.append(multiMutationsCount, that.multiMutationsCount)
.append(multiServiceCalls, that.multiServiceCalls).append(clientAddress, that.clientAddress)
.append(serverClass, that.serverClass).append(methodName, that.methodName)
Expand All @@ -267,9 +286,9 @@ public boolean equals(Object o) {
@Override
public int hashCode() {
return new HashCodeBuilder(17, 37).append(startTime).append(processingTime).append(queueTime)
.append(responseSize).append(clientAddress).append(serverClass).append(methodName)
.append(callDetails).append(param).append(regionName).append(userName).append(multiGetsCount)
.append(multiMutationsCount).append(multiServiceCalls).toHashCode();
.append(responseSize).append(blockBytesScanned).append(clientAddress).append(serverClass)
.append(methodName).append(callDetails).append(param).append(regionName).append(userName)
.append(multiGetsCount).append(multiMutationsCount).append(multiServiceCalls).toHashCode();
}

@Override
Expand All @@ -281,11 +300,11 @@ public String toJsonPrettyPrint() {
public String toString() {
return new ToStringBuilder(this).append("startTime", startTime)
.append("processingTime", processingTime).append("queueTime", queueTime)
.append("responseSize", responseSize).append("clientAddress", clientAddress)
.append("serverClass", serverClass).append("methodName", methodName)
.append("callDetails", callDetails).append("param", param).append("regionName", regionName)
.append("userName", userName).append("multiGetsCount", multiGetsCount)
.append("multiMutationsCount", multiMutationsCount)
.append("responseSize", responseSize).append("blockBytesScanned", blockBytesScanned)
.append("clientAddress", clientAddress).append("serverClass", serverClass)
.append("methodName", methodName).append("callDetails", callDetails).append("param", param)
.append("regionName", regionName).append("userName", userName)
.append("multiGetsCount", multiGetsCount).append("multiMutationsCount", multiMutationsCount)
.append("multiServiceCalls", multiServiceCalls).toString();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3306,6 +3306,7 @@ private static LogEntry getSlowLogRecord(final TooSlowLog.SlowLogPayload slowLog
.setParam(slowLogPayload.getParam()).setProcessingTime(slowLogPayload.getProcessingTime())
.setQueueTime(slowLogPayload.getQueueTime()).setRegionName(slowLogPayload.getRegionName())
.setResponseSize(slowLogPayload.getResponseSize())
.setBlockBytesScanned(slowLogPayload.getBlockBytesScanned())
.setServerClass(slowLogPayload.getServerClass()).setStartTime(slowLogPayload.getStartTime())
.setUserName(slowLogPayload.getUserName()).build();
return onlineLogRecord;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,8 @@ public static void addSlowLogRecords(final List<TooSlowLog.SlowLogPayload> slowL
Bytes.toBytes(slowLogPayload.getRegionName()))
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("response_size"),
Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize())))
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("block_bytes_scanned"),
Bytes.toBytes(Long.toString(slowLogPayload.getBlockBytesScanned())))
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("server_class"),
Bytes.toBytes(slowLogPayload.getServerClass()))
.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
Expand Down
2 changes: 2 additions & 0 deletions hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@ message SlowLogPayload {
optional int32 multi_service_calls = 14 [default = 0];
required Type type = 15;

optional int64 block_bytes_scanned = 16;

// SLOW_LOG is RPC call slow in nature whereas LARGE_LOG is RPC call quite large.
// Majority of times, slow logs are also large logs and hence, ALL is combination of
// both
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -428,6 +428,7 @@ public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)
// Use the raw request call size for now.
long requestSize = call.getSize();
long responseSize = result.getSerializedSize();
long responseBlockSize = call.getResponseBlockSize();
if (call.isClientCellBlockSupported()) {
// Include the payload size in HBaseRpcController
responseSize += call.getResponseCellSize();
Expand All @@ -441,20 +442,21 @@ public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)
// log any RPC responses that are slower than the configured warn
// response time or larger than configured warning size
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
boolean tooLarge = (warnResponseSize > -1
&& (responseSize > warnResponseSize || responseBlockSize > warnResponseSize));
if (tooSlow || tooLarge) {
final String userName = call.getRequestUserName().orElse(StringUtils.EMPTY);
// when tagging, we let TooLarge trump TooSmall to keep output simple
// note that large responses will often also be slow.
logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize,
userName);
responseBlockSize, userName);
if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) {
// send logs to ring buffer owned by slowLogRecorder
final String className =
server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
this.namedQueueRecorder.addRecord(new RpcLogDetails(call, param, status.getClient(),
responseSize, className, tooSlow, tooLarge));
responseSize, responseBlockSize, className, tooSlow, tooLarge));
}
}
return new Pair<>(result, controller.cellScanner());
Expand Down Expand Up @@ -482,29 +484,31 @@ public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)

/**
* Logs an RPC response to the LOG file, producing valid JSON objects for client Operations.
* @param param The parameters received in the call.
* @param methodName The name of the method invoked
* @param call The string representation of the call
* @param tooLarge To indicate if the event is tooLarge
* @param tooSlow To indicate if the event is tooSlow
* @param clientAddress The address of the client who made this call.
* @param startTime The time that the call was initiated, in ms.
* @param processingTime The duration that the call took to run, in ms.
* @param qTime The duration that the call spent on the queue prior to being initiated,
* in ms.
* @param responseSize The size in bytes of the response buffer.
* @param userName UserName of the current RPC Call
* @param param The parameters received in the call.
* @param methodName The name of the method invoked
* @param call The string representation of the call
* @param tooLarge To indicate if the event is tooLarge
* @param tooSlow To indicate if the event is tooSlow
* @param clientAddress The address of the client who made this call.
* @param startTime The time that the call was initiated, in ms.
* @param processingTime The duration that the call took to run, in ms.
* @param qTime The duration that the call spent on the queue prior to being
* initiated, in ms.
* @param responseSize The size in bytes of the response buffer.
* @param blockBytesScanned The size of block bytes scanned to retrieve the response.
* @param userName UserName of the current RPC Call
*/
void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow,
String clientAddress, long startTime, int processingTime, int qTime, long responseSize,
String userName) {
long blockBytesScanned, String userName) {
final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();
// base information that is reported regardless of type of call
Map<String, Object> responseInfo = new HashMap<>();
responseInfo.put("starttimems", startTime);
responseInfo.put("processingtimems", processingTime);
responseInfo.put("queuetimems", qTime);
responseInfo.put("responsesize", responseSize);
responseInfo.put("blockbytesscanned", blockBytesScanned);
responseInfo.put("client", clientAddress);
responseInfo.put("class", className);
responseInfo.put("method", methodName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,17 +35,19 @@ public class RpcLogDetails extends NamedQueuePayload {
private final Message param;
private final String clientAddress;
private final long responseSize;
private final long blockBytesScanned;
private final String className;
private final boolean isSlowLog;
private final boolean isLargeLog;

public RpcLogDetails(RpcCall rpcCall, Message param, String clientAddress, long responseSize,
String className, boolean isSlowLog, boolean isLargeLog) {
long blockBytesScanned, String className, boolean isSlowLog, boolean isLargeLog) {
super(SLOW_LOG_EVENT);
this.rpcCall = rpcCall;
this.param = param;
this.clientAddress = clientAddress;
this.responseSize = responseSize;
this.blockBytesScanned = blockBytesScanned;
this.className = className;
this.isSlowLog = isSlowLog;
this.isLargeLog = isLargeLog;
Expand All @@ -63,6 +65,10 @@ public long getResponseSize() {
return responseSize;
}

public long getBlockBytesScanned() {
return blockBytesScanned;
}

public String getClassName() {
return className;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,7 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
final RpcCall rpcCall = rpcLogDetails.getRpcCall();
final String clientAddress = rpcLogDetails.getClientAddress();
final long responseSize = rpcLogDetails.getResponseSize();
final long blockBytesScanned = rpcLogDetails.getBlockBytesScanned();
final String className = rpcLogDetails.getClassName();
final TooSlowLog.SlowLogPayload.Type type = getLogType(rpcLogDetails);
if (type == null) {
Expand Down Expand Up @@ -157,8 +158,9 @@ public void consumeEventFromDisruptor(NamedQueuePayload namedQueuePayload) {
.setParam(slowLogParams != null ? slowLogParams.getParams() : StringUtils.EMPTY)
.setProcessingTime(processingTime).setQueueTime(qTime)
.setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : StringUtils.EMPTY)
.setResponseSize(responseSize).setServerClass(className).setStartTime(startTime).setType(type)
.setUserName(userName).build();
.setResponseSize(responseSize).setBlockBytesScanned(blockBytesScanned)
.setServerClass(className).setStartTime(startTime).setType(type).setUserName(userName)
.build();
slowLogQueue.add(slowLogPayload);
if (isSlowLogTableEnabled) {
if (!slowLogPayload.getRegionName().startsWith("hbase:slowlog")) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,7 @@
<th>Processing Time</th>
<th>Queue Time</th>
<th>Response Size</th>
<th>Block Bytes Scanned</th>
<th>Client Address</th>
<th>Server Class</th>
<th>Method Name</th>
Expand All @@ -115,6 +116,7 @@
<td><%=r.getProcessingTime()%>ms</td>
<td><%=r.getQueueTime()%>ms</td>
<td><%=StringUtils.byteDesc(r.getResponseSize())%></td>
<td><%=StringUtils.byteDesc(r.getBlockBytesScanned())%></td>
<td><%=r.getClientAddress()%></td>
<td><%=r.getServerClass()%></td>
<td><%=r.getMethodName()%></td>
Expand All @@ -138,6 +140,7 @@
<th>Processing Time</th>
<th>Queue Time</th>
<th>Response Size</th>
<th>Block Bytes Scanned</th>
<th>Client Address</th>
<th>Server Class</th>
<th>Method Name</th>
Expand All @@ -156,6 +159,7 @@
<td><%=r.getProcessingTime()%>ms</td>
<td><%=r.getQueueTime()%>ms</td>
<td><%=StringUtils.byteDesc(r.getResponseSize())%></td>
<td><%=StringUtils.byteDesc(r.getBlockBytesScanned())%></td>
<td><%=r.getClientAddress()%></td>
<td><%=r.getServerClass()%></td>
<td><%=r.getMethodName()%></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -529,13 +529,14 @@ public void testSlowLogMixedFilters() throws Exception {

static RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className) {
RpcCall rpcCall = getRpcCall(userName);
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, className, true, true);
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, true,
true);
}

private RpcLogDetails getRpcLogDetails(String userName, String clientAddress, String className,
boolean isSlowLog, boolean isLargeLog) {
RpcCall rpcCall = getRpcCall(userName);
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, className, isSlowLog,
return new RpcLogDetails(rpcCall, rpcCall.getParam(), clientAddress, 0, 0, className, isSlowLog,
isLargeLog);
}

Expand Down
Loading

0 comments on commit 6ba8b32

Please sign in to comment.