HBASE-16033 Add more details in logging of responseTooSlow/TooLarge

This commit is contained in:
Yu Li 2016-06-16 16:40:38 +08:00
parent d1de9337ef
commit 6d02f36ac7
1 changed files with 6 additions and 29 deletions

View File

@ -2235,7 +2235,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
if (tooSlow || tooLarge) { if (tooSlow || tooLarge) {
// when tagging, we let TooLarge trump TooSmall to keep output simple // when tagging, we let TooLarge trump TooSmall to keep output simple
// note that large responses will often also be slow. // note that large responses will often also be slow.
logResponse(new Object[]{param}, logResponse(param,
md.getName(), md.getName() + "(" + param.getClass().getName() + ")", md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
(tooLarge ? "TooLarge" : "TooSlow"), (tooLarge ? "TooLarge" : "TooSlow"),
status.getClient(), startTime, processingTime, qTime, status.getClient(), startTime, processingTime, qTime,
@ -2261,7 +2261,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
/** /**
* Logs an RPC response to the LOG file, producing valid JSON objects for * Logs an RPC response to the LOG file, producing valid JSON objects for
* client Operations. * client Operations.
* @param params The parameters received in the call. * @param param The parameters received in the call.
* @param methodName The name of the method invoked * @param methodName The name of the method invoked
* @param call The string representation of the call * @param call The string representation of the call
* @param tag The tag that will be used to indicate this event in the log. * @param tag The tag that will be used to indicate this event in the log.
@ -2272,7 +2272,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
* prior to being initiated, in ms. * prior to being initiated, in ms.
* @param responseSize The size in bytes of the response buffer. * @param responseSize The size in bytes of the response buffer.
*/ */
void logResponse(Object[] params, String methodName, String call, String tag, void logResponse(Message param, String methodName, String call, String tag,
String clientAddress, long startTime, int processingTime, int qTime, String clientAddress, long startTime, int processingTime, int qTime,
long responseSize) long responseSize)
throws IOException { throws IOException {
@ -2285,32 +2285,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
responseInfo.put("client", clientAddress); responseInfo.put("client", clientAddress);
responseInfo.put("class", server == null? "": server.getClass().getSimpleName()); responseInfo.put("class", server == null? "": server.getClass().getSimpleName());
responseInfo.put("method", methodName); responseInfo.put("method", methodName);
if (params.length == 2 && server instanceof HRegionServer && responseInfo.put("call", call);
params[0] instanceof byte[] && responseInfo.put("param", ProtobufUtil.getShortTextFormat(param));
params[1] instanceof Operation) { LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
// if the slow process is a query, we want to log its table as well
// as its own fingerprint
TableName tableName = TableName.valueOf(
HRegionInfo.parseRegionName((byte[]) params[0])[0]);
responseInfo.put("table", tableName.getNameAsString());
// annotate the response map with operation details
responseInfo.putAll(((Operation) params[1]).toMap());
// report to the log file
LOG.warn("(operation" + tag + "): " +
MAPPER.writeValueAsString(responseInfo));
} else if (params.length == 1 && server instanceof HRegionServer &&
params[0] instanceof Operation) {
// annotate the response map with operation details
responseInfo.putAll(((Operation) params[0]).toMap());
// report to the log file
LOG.warn("(operation" + tag + "): " +
MAPPER.writeValueAsString(responseInfo));
} else {
// can't get JSON details, so just report call.toString() along with
// a more generic tag.
responseInfo.put("call", call);
LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
}
} }
/** Stops the service. No new calls will be handled after this is called. */ /** Stops the service. No new calls will be handled after this is called. */