Repository: hbase Updated Branches: refs/heads/master d1de9337e -> 6d02f36ac
HBASE-16033 Add more details in logging of responseTooSlow/TooLarge Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6d02f36a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6d02f36a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6d02f36a Branch: refs/heads/master Commit: 6d02f36ac78426f178b977566c170fb2e4b3503a Parents: d1de933 Author: Yu Li <l...@apache.org> Authored: Thu Jun 16 16:40:38 2016 +0800 Committer: Yu Li <l...@apache.org> Committed: Thu Jun 16 16:42:10 2016 +0800 ---------------------------------------------------------------------- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 35 ++++---------------- 1 file changed, 6 insertions(+), 29 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/6d02f36a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index c7f5a10..ad88e8e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -2235,7 +2235,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { if (tooSlow || tooLarge) { // when tagging, we let TooLarge trump TooSmall to keep output simple // note that large responses will often also be slow. - logResponse(new Object[]{param}, + logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")", (tooLarge ? "TooLarge" : "TooSlow"), 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 * 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 call The string representation of the call * @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. * @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, long responseSize) throws IOException { @@ -2285,32 +2285,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { responseInfo.put("client", clientAddress); responseInfo.put("class", server == null? "": server.getClass().getSimpleName()); responseInfo.put("method", methodName); - if (params.length == 2 && server instanceof HRegionServer && - params[0] instanceof byte[] && - params[1] instanceof Operation) { - // 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)); - } + responseInfo.put("call", call); + responseInfo.put("param", ProtobufUtil.getShortTextFormat(param)); + LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo)); } /** Stops the service. No new calls will be handled after this is called. */