ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r377238105
 
 

 ##########
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##########
 @@ -496,11 +518,51 @@ void logResponse(Message param, String methodName, 
String call, String tag,
           }
         }
       }
-      responseInfo.put("multi.gets", numGets);
-      responseInfo.put("multi.mutations", numMutations);
-      responseInfo.put("multi.servicecalls", numServiceCalls);
+      responseInfo.put(MULTI_GETS, numGets);
+      responseInfo.put(MULTI_MUTATIONS, numMutations);
+      responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);
     }
+    final String tag = tooLarge ? "TooLarge" : "TooSlow";
     LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+    if (tooSlow && this.onlineSlowLogProvider != null
+        && !call.startsWith(GET_SLOW_LOG_RESPONSES)
+        && !call.startsWith(CLEAR_SLOW_LOGS_RESPONSES)) {
+      logOnlineSlowResponse(param, methodName, call, clientAddress, startTime,
+        processingTime, qTime, responseSize, userName, className, 
responseInfo);
+    }
+  }
+
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+      String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+      String userName, String className, Map<String, Object> responseInfo) {
+    // add too slow log to ringbuffer for retrieval of latest n slow logs
+
+    try {
+      final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+
+      final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+        .setStartTime(startTime)
+        .setProcessingTime(processingTime)
+        .setQueueTime(qTime)
+        .setResponseSize(responseSize)
+        .setClientAddress(clientAddress)
+        .setServerClass(className)
+        .setMethodName(methodName)
+        .setCallDetails(call)
+        .setUserName(userName)
+        .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : 
StringUtils.EMPTY)
+        .setParam(slowLogParams != null ? slowLogParams.getParams() : 
StringUtils.EMPTY)
+        .setMultiGets(responseInfo.containsKey(MULTI_GETS)
+          ? (int) responseInfo.get(MULTI_GETS) : 0)
+        .setMultiMutations(responseInfo.containsKey(MULTI_MUTATIONS)
+          ? (int) responseInfo.get(MULTI_MUTATIONS) : 0)
+        .setMultiServiceCalls(responseInfo.containsKey(MULTI_SERVICE_CALLS)
+          ? (int) responseInfo.get(MULTI_SERVICE_CALLS) : 0)
+        .build();
+      this.onlineSlowLogProvider.addSlowLogPayload(slowLogPayload);
+    } catch (Exception e) {
+      LOG.debug("Error while adding slowlog response to ringbuffer", e);
 
 Review comment:
   If an operator has enabled this feature, I think they'd want to know if it's 
somehow not working. How about logging at warning level instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to