[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383112056
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383110349
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383113149
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,522 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.ipc.RpcCallback;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestSlowLogRecorder {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestSlowLogRecorder.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestSlowLogRecorder.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private SlowLogRecorder slowLogRecorder;
+
+  private static Configuration applySlowLogRecorderConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
+return conf;
+  }
+
+  /**
+   * confirm that for a ringbuffer of slow logs, payload on given index of 
buffer
+   * has expected elements
+   *
+   * @param i index of ringbuffer logs
+   * @param j data value that was put on index i
+   * @param slowLogPayloads list of payload retrieved from {@link 
SlowLogRecorder}
+   */
+  private void confirmPayloadParams(int i, int j, List 
slowLogPayloads) {
+Assert.assertEquals(slowLogPayloads.get(i).getClientAddress(), "client_" + 
j);
+Assert.assertEquals(slowLogPayloads.get(i).getUserName(), "userName_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getServerClass(), "class_" + j);
+  }
+
+  @Test
+  public void testOnlieSlowLogConsumption() throws Exception {
+Configuration conf = applySlowLogRecorderConf(8);
+slowLogRecorder = new SlowLogRecorder(conf);
+AdminProtos.SlowLogResponseRequest request =
+  AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads(request).size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  RpcLogDetails rpcLogDetails =
+getRpcLogDetails("userName_"

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383109014
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383110145
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383110494
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382879831
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,522 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.ipc.RpcCallback;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestSlowLogRecorder {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestSlowLogRecorder.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestSlowLogRecorder.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private SlowLogRecorder slowLogRecorder;
+
+  private static Configuration applySlowLogRecorderConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
+return conf;
+  }
+
+  /**
+   * confirm that for a ringbuffer of slow logs, payload on given index of 
buffer
+   * has expected elements
+   *
+   * @param i index of ringbuffer logs
+   * @param j data value that was put on index i
+   * @param slowLogPayloads list of payload retrieved from {@link 
SlowLogRecorder}
+   */
+  private void confirmPayloadParams(int i, int j, List 
slowLogPayloads) {
+Assert.assertEquals(slowLogPayloads.get(i).getClientAddress(), "client_" + 
j);
+Assert.assertEquals(slowLogPayloads.get(i).getUserName(), "userName_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getServerClass(), "class_" + j);
+  }
+
+  @Test
+  public void testOnlieSlowLogConsumption() throws Exception {
+Configuration conf = applySlowLogRecorderConf(8);
+slowLogRecorder = new SlowLogRecorder(conf);
+AdminProtos.SlowLogResponseRequest request =
+  AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads(request).size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  RpcLogDetails rpcLogDetails =
+getRpcLogDetails("userName_"

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382872894
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382878329
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RingBufferEnvelope.java
 ##
 @@ -0,0 +1,57 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+/**
+ * An envelope to carry payload in the slow log ring buffer that serves as 
online buffer
+ * to provide latest TooSlowLog
+ */
+@InterfaceAudience.Private
+final class RingBufferEnvelope {
+
+  private RpcLogDetails rpcLogDetails;
+
+  /**
+   * Load the Truck with {@link RpcCall}
 
 Review comment:
   nit: whats a Truck.. don't think thats a disruptor terminilogy...


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382879845
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,522 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.ipc.RpcCallback;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestSlowLogRecorder {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestSlowLogRecorder.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestSlowLogRecorder.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private SlowLogRecorder slowLogRecorder;
+
+  private static Configuration applySlowLogRecorderConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
+return conf;
+  }
+
+  /**
+   * confirm that for a ringbuffer of slow logs, payload on given index of 
buffer
+   * has expected elements
+   *
+   * @param i index of ringbuffer logs
+   * @param j data value that was put on index i
+   * @param slowLogPayloads list of payload retrieved from {@link 
SlowLogRecorder}
+   */
+  private void confirmPayloadParams(int i, int j, List 
slowLogPayloads) {
+Assert.assertEquals(slowLogPayloads.get(i).getClientAddress(), "client_" + 
j);
+Assert.assertEquals(slowLogPayloads.get(i).getUserName(), "userName_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getServerClass(), "class_" + j);
+  }
+
+  @Test
+  public void testOnlieSlowLogConsumption() throws Exception {
+Configuration conf = applySlowLogRecorderConf(8);
+slowLogRecorder = new SlowLogRecorder(conf);
+AdminProtos.SlowLogResponseRequest request =
+  AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads(request).size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  RpcLogDetails rpcLogDetails =
+getRpcLogDetails("userName_"

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382878645
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382878529
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382874132
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382872128
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382879850
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,522 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.ipc.RpcCallback;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestSlowLogRecorder {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestSlowLogRecorder.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestSlowLogRecorder.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private SlowLogRecorder slowLogRecorder;
+
+  private static Configuration applySlowLogRecorderConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
+return conf;
+  }
+
+  /**
+   * confirm that for a ringbuffer of slow logs, payload on given index of 
buffer
+   * has expected elements
+   *
+   * @param i index of ringbuffer logs
+   * @param j data value that was put on index i
+   * @param slowLogPayloads list of payload retrieved from {@link 
SlowLogRecorder}
+   */
+  private void confirmPayloadParams(int i, int j, List 
slowLogPayloads) {
+Assert.assertEquals(slowLogPayloads.get(i).getClientAddress(), "client_" + 
j);
+Assert.assertEquals(slowLogPayloads.get(i).getUserName(), "userName_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getServerClass(), "class_" + j);
+  }
+
+  @Test
+  public void testOnlieSlowLogConsumption() throws Exception {
+Configuration conf = applySlowLogRecorderConf(8);
+slowLogRecorder = new SlowLogRecorder(conf);
+AdminProtos.SlowLogResponseRequest request =
+  AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads(request).size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  RpcLogDetails rpcLogDetails =
+getRpcLogDetails("userName_"

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382878572
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382876386
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -3410,4 +3471,56 @@ public static RegionStatesCount 
toTableRegionStatesCount(
   .build();
   }
 
+  /**
+   * Convert Protobuf class
+   * {@link 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload}
+   * To client SlowLog Payload class {@link SlowLogRecord}
+   *
+   * @param slowLogPayload SlowLog Payload protobuf instance
+   * @return SlowLog Payload for client usecase
+   */
+  private static SlowLogRecord getSlowLogPayload(
 
 Review comment:
   nit: getSlowLogRecord()?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382871376
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3874,4 +3878,62 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> getSlowLogResponses(
+@Nullable final Set serverNames,
 
 Review comment:
   nit: mind doing the annotations in multiple places? (not just here)


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382873543
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + "

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382878387
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
 
 Review comment:
   endOfBatch is unused.. is that okay?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-21 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382879258
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
 ##
 @@ -0,0 +1,153 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Online SlowLog Provider Service that keeps slow RPC logs in the ring buffer.
+ * The service uses LMAX Disruptor to save slow records which are then 
consumed by
+ * a queue and based on the ring buffer size, the available records are then 
fetched
+ * from the queue in thread-safe manner.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SlowLogRecorder {
+
+  private final Disruptor disruptor;
+  private final SlowLogEventHandler slowLogEventHandler;
+  private final int eventCount;
+  private final boolean isOnlineSlowLogProviderEnabled;
+
+  private static final String SLOW_LOG_RING_BUFFER_SIZE =
+"hbase.regionserver.slowlog.ringbuffer.size";
+
+  /**
+   * Initialize disruptor with configurable ringbuffer size
+   */
+  public SlowLogRecorder(Configuration conf) {
+isOnlineSlowLogProviderEnabled = 
conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
+  HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
+
+if (!isOnlineSlowLogProviderEnabled) {
+  this.disruptor = null;
 
 Review comment:
   nit: all these are defaults.. any reason specifically null initialize them?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382256846
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -2217,6 +2227,57 @@ private static String getStringForByteString(ByteString 
bs) {
 return Bytes.toStringBinary(bs.toByteArray());
   }
 
+  /**
+   * Return SlowLogParams to maintain recent online slowlog responses
+   *
+   * @param message Message object {@link Message}
+   * @return SlowLogParams with regionName(for filter queries) and params
+   */
+  public static SlowLogParams getSlowLogParams(Message message) {
+if (message == null) {
+  return null;
+}
+if (message instanceof ScanRequest) {
+  ScanRequest scanRequest = (ScanRequest) message;
 
 Review comment:
   Are we picking specific fields for concise output rather than dumping 
everything?
   What if there is a new request type? 


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382247138
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,154 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
 
 Review comment:
   Ya, like we discussed offline, that makes it much better.. also look at the 
other refactoring comments.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382251037
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java
 ##
 @@ -0,0 +1,317 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+
+/**
+ * SlowLog payload for client
+ */
+@InterfaceAudience.Private
+final public class SlowLogRecord {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogRecord.class, (JsonSerializer)
+  (slowLogPayload, type, jsonSerializationContext) -> {
+Gson gson = new Gson();
+JsonObject jsonObj = (JsonObject) gson.toJsonTree(slowLogPayload);
+if (slowLogPayload.getMultiGetsCount() == 0) {
+  jsonObj.remove("multiGetsCount");
+}
+if (slowLogPayload.getMultiMutationsCount() == 0) {
+  jsonObj.remove("multiMutationsCount");
+}
+if (slowLogPayload.getMultiServiceCalls() == 0) {
+  jsonObj.remove("multiServiceCalls");
+}
+return jsonObj;
+  }).create();
+
+  private long startTime;
+  private int processingTime;
+  private int queueTime;
+  private long responseSize;
+  private String clientAddress;
+  private String serverClass;
+  private String methodName;
+  private String callDetails;
+  private String param;
+  // we don't want to serialize region name, it is just for the filter purpose
+  // hence avoiding deserialization
+  private transient String regionName;
+  private String userName;
+  private int multiGetsCount;
+  private int multiMutationsCount;
+  private int multiServiceCalls;
+
+  public long getStartTime() {
+return startTime;
+  }
+
+  public int getProcessingTime() {
+return processingTime;
+  }
+
+  public int getQueueTime() {
+return queueTime;
+  }
+
+  public long getResponseSize() {
+return responseSize;
+  }
+
+  public String getClientAddress() {
+return clientAddress;
+  }
+
+  public String getServerClass() {
+return serverClass;
+  }
+
+  public String getMethodName() {
+return methodName;
+  }
+
+  public String getCallDetails() {
+return callDetails;
+  }
+
+  public String getParam() {
+return param;
+  }
+
+  public String getRegionName() {
+return regionName;
+  }
+
+  public String getUserName() {
+return userName;
+  }
+
+  public int getMultiGetsCount() {
+return multiGetsCount;
+  }
+
+  public int getMultiMutationsCount() {
+return multiMutationsCount;
+  }
+
+  public int getMultiServiceCalls() {
+return multiServiceCalls;
+  }
+
+  private SlowLogRecord(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) {
+this.startTime = startTime;
+this.processingTime = processingTime;
+this.queueTime = queueTime;
+this.responseSize = responseSize;
+this.clientAddress = clientAddress;
+this.serverClass = serverClass;
+this.methodName = methodName;
+this.callDetails = callDetails;
+this.param = param;
+this.regionName = regionName;
+this.userName = userName;
+this.multiGetsCount = multiGetsCount;
+this.multiMutationsCount = multiMutationsCount;
+this.multiServiceCalls = multiServiceCalls;
+  }
+
+  public static class SlowLogPayloadBuilder {
 
 Review comment:
   Is this SlowLog"R

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382212583
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3874,4 +3876,62 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> getSlowLogResponses(
+final Set serverNames,
 
 Review comment:
   nit: Use Nullable annotations in signatures.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382211635
 
 

 ##
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
 ##
 @@ -2261,4 +2261,30 @@ boolean snapshotCleanupSwitch(final boolean on, final 
boolean synchronous)
*/
   boolean isSnapshotCleanupEnabled() throws IOException;
 
+
+  /**
+   * Retrieves online slow RPC logs from the provided list of
+   * RegionServers
+   *
+   * @param serverNames Server names to get slowlog responses from
+   * @param slowLogQueryFilter filter to be used if provided
+   * @return online slowlog response list
+   * @throws IOException if a remote or network exception occurs
+   */
+  List getSlowLogResponses(final Set serverNames,
 
 Review comment:
   nit: format to fewer lines (multiple places)


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382247687
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,167 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
 
 Review comment:
   nit: Single line: queue = synchronizedQueue(EvictingQueue...)


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382232200
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 ##
 @@ -590,6 +596,10 @@ public HRegionServer(final Configuration conf) throws 
IOException {
   this.abortRequested = false;
   this.stopped = false;
 
+  // initiate online slowlog ringbuffer only for RegionServers
+  if (!(this instanceof HMaster)) {
 
 Review comment:
   I think Nick had a valid point..didn't get your response..why not check conf 
for "hbase.regionserver.slowlog.buffer.enabled" and not initialize 
SlowLogRecorder? You have checks anyway in the callers like
   
   if (slowLogRecorder!= null) slowLogRecorder.addPayLoad()...
   
   I think that simplies the code rather than having conditionals in 
SLowLogRecorder constructor.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382216671
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java
 ##
 @@ -0,0 +1,317 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+
+/**
+ * SlowLog payload for client
+ */
+@InterfaceAudience.Private
+final public class SlowLogRecord {
+
+  private static final Gson GSON = GsonUtil.createGson()
 
 Review comment:
   nit: Add a comment why we need this?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382242034
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +517,63 @@ 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 && tooSlow) ? "TooLarge & TooSlow"
+  : (tooSlow ? "TooSlow" : "TooLarge");
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.slowLogRecorder != null) {
+  logOnlineSlowResponse(param, methodName, call,
+clientAddress, startTime, processingTime, qTime, responseSize, 
userName,
+className, responseInfo);
+}
+  }
+
+  /**
+   * Add too slow log to ringbuffer for retrieval of latest n slow logs
+   *
+   * @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 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 className ClassName of the SlowLog call
+   * @param responseInfo Base information map that is reported regardless of 
type of call
+   */
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+  String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+  String userName, String className, Map responseInfo) {
+
+try {
+
+  final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+.setCallDetails(call)
+.setClientAddress(clientAddress)
+.setMethodName(methodName)
+.setMultiGets(responseInfo.containsKey(MULTI_GETS)
 
 Review comment:
   Are these of any use for debugging? If not I suggest removing them..


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382216905
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java
 ##
 @@ -0,0 +1,317 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+
+/**
+ * SlowLog payload for client
 
 Review comment:
   nit: add more detail in the javadoc.. not clear what client is..


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382242230
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +517,63 @@ 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 && tooSlow) ? "TooLarge & TooSlow"
+  : (tooSlow ? "TooSlow" : "TooLarge");
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.slowLogRecorder != null) {
+  logOnlineSlowResponse(param, methodName, call,
+clientAddress, startTime, processingTime, qTime, responseSize, 
userName,
+className, responseInfo);
+}
+  }
+
+  /**
+   * Add too slow log to ringbuffer for retrieval of latest n slow logs
+   *
+   * @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 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 className ClassName of the SlowLog call
+   * @param responseInfo Base information map that is reported regardless of 
type of call
+   */
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+  String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+  String userName, String className, Map responseInfo) {
+
+try {
+
+  final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+.setCallDetails(call)
+.setClientAddress(clientAddress)
+.setMethodName(methodName)
+.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)
+.setProcessingTime(processingTime)
+.setQueueTime(qTime)
+.setResponseSize(responseSize)
+.setServerClass(className)
+.setStartTime(startTime)
+.setUserName(userName)
+.build();
+  this.slowLogRecorder.addSlowLogPayload(slowLogPayload, param);
 
 Review comment:
   nit: this is redundant..
   
   


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382257703
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,167 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+Pair slowLogPayloadPair = event.getPayload();
+SlowLogPayload slowLogPayload = slowLogPayloadPair.getFirst();
+Message param = slowLogPayloadPair.getSecond();
+if (!slowLogPayload.hasParam() || !slowLogPayload.hasRegionName()) {
+  SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+  slowLogPayload = slowLogPayload.toBuilder()
+.setParam(slowLogParams != null ? slowLogParams.getParams() : 
StringUtils.EMPTY)
+.setRegionName(slowLogParams != null ? slowLogParams.getRegionName() : 
StringUtils.EMPTY)
+.build();
+}
+if (LOG.isTraceEnabled()) {
+  LOG.trace("Received Slow Log Event. RingBuffer sequence: {}, 
isEndOfBatch: {}, " +
+  "Event Call: {}", sequence, endOfBatch,
+slowLogPayload.getCallDetails());
+}
+queue.add(slowLogPayload);
 
 Review comment:
   Like I mentioned elsewhere, this should not be a Queue of protobuf 
serialized stuff, instead it should be a simple container class that is 
serialized as output only when there is a request to dump it. 


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382241895
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +517,63 @@ 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 && tooSlow) ? "TooLarge & TooSlow"
+  : (tooSlow ? "TooSlow" : "TooLarge");
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.slowLogRecorder != null) {
+  logOnlineSlowResponse(param, methodName, call,
+clientAddress, startTime, processingTime, qTime, responseSize, 
userName,
+className, responseInfo);
+}
+  }
+
+  /**
+   * Add too slow log to ringbuffer for retrieval of latest n slow logs
+   *
+   * @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 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 className ClassName of the SlowLog call
+   * @param responseInfo Base information map that is reported regardless of 
type of call
+   */
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+  String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+  String userName, String className, Map responseInfo) {
+
+try {
+
+  final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+.setCallDetails(call)
 
 Review comment:
   Oh wait.. why are we putting a protobuf object in the ring buffer.. Can we 
just have a simple object that has reference to "call", "param" etc and then we 
can lazily serialize it when needed by clients?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382246097
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +517,63 @@ 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 && tooSlow) ? "TooLarge & TooSlow"
+  : (tooSlow ? "TooSlow" : "TooLarge");
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.slowLogRecorder != null) {
+  logOnlineSlowResponse(param, methodName, call,
+clientAddress, startTime, processingTime, qTime, responseSize, 
userName,
+className, responseInfo);
+}
+  }
+
+  /**
+   * Add too slow log to ringbuffer for retrieval of latest n slow logs
+   *
+   * @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 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 className ClassName of the SlowLog call
+   * @param responseInfo Base information map that is reported regardless of 
type of call
+   */
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+  String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+  String userName, String className, Map responseInfo) {
+
+try {
+
+  final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+.setCallDetails(call)
+.setClientAddress(clientAddress)
+.setMethodName(methodName)
+.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)
+.setProcessingTime(processingTime)
+.setQueueTime(qTime)
+.setResponseSize(responseSize)
+.setServerClass(className)
+.setStartTime(startTime)
+.setUserName(userName)
+.build();
+  this.slowLogRecorder.addSlowLogPayload(slowLogPayload, param);
 
 Review comment:
   Now I see why you are not putting param inside slowLogPayLoad.. why a 
protobuf inside ringbuffer? We can have the serialization penalty on reads 
(like get_slow_log calls). Don't see why it should during the writes. There 
will be far more useless writes than reads.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-20 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r382237148
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -403,13 +419,14 @@ public void setSecretManager(SecretManager secretMana
   boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime 
> -1);
   boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize 
> -1);
   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,
 
 Review comment:
   Can you refactor all of this? Most content can be inferred from the "param" 
and "call".. So the signature should be something like logResponses(param, 
call)... Too much clutter and too many variables being passed around.. If 
possible also move the metrics updation and trace logging into this method.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-16 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379951556
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,154 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
 
 Review comment:
   I think you misunderstood my comment.. My point was to get rid of lmax's 
ringbuffer implementation with and asynchronous thread-safe EvictingQueue (it 
isn't as complex as it sounds)...
   
   When I meant asynchronous, any entries added by the event producer 
(RpcServer here) is asynchronously processed by a single thread. This is easier 
to reason about than CompletableFuture.runAsync() approach you did on the 
caller.
   
   To recap my original comment, the implementation should be something like 
this I think.
   
   - We have a shared blocking queue between the producer (RpcServer) and 
consumer (the thread that serializes the input and puts it in an evicting 
queue). All the producer does is put events (too long/too big) events in this 
shared queue which automatically makes the whole thing async from a producer 
POV.
   
   - The consumer (a single thread), keeps polling entries, serializes them 
(also does logging) and puts them into a ring buffer 
   
   This would be much fewer lines of code and much more simple to reason about 
than using the lmax's ring buffer (with very similar performance for our use 
case).
   
   Thoughts?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-16 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379949388
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
 ##
 @@ -0,0 +1,89 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * SlowLog params object that contains detailed info as params and region name 
: to be used
+ * for filter purpose
+ */
+@InterfaceAudience.Private
+public class SlowLogParams {
 
 Review comment:
   Oh I see.. It looks like "slow logging" is a thing.. was not aware of the 
term..ok then..


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-16 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379949636
 
 

 ##
 File path: hbase-common/src/main/resources/hbase-default.xml
 ##
 @@ -1956,4 +1956,27 @@ possible configurations would overwhelm and obscure the 
important.
   enable this feature.
 
   
+  
+hbase.regionserver.slowlog.ringbuffer.size
+256
+
+  Default size of ringbuffer to be maintained by each RegionServer in order
+  to store online slowlog responses. This is an in-memory ring buffer of
+  requests that were judged to be too slow in addition to the 
responseTooSlow
+  logging. The in-memory representation would be complete.
+  For more details, please look into Doc Section:
+  Get Slow Response Log from shell
+
+  
+  
+hbase.regionserver.slowlog.buffer.enabled
 
 Review comment:
   The idea here is to simplify the number of knobs as much as possible. This 
makes is easier from an operator POV. I still think we should limit the number 
of parameters here.. don't see any strong reason to use two parameters here, 
fwiw.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-16 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379949787
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,65 @@ 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 && tooSlow) ? "TooLarge & TooSlow"
+  : (tooSlow ? "TooSlow" : "TooLarge");
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.slowLogRecorder != null) {
+  logOnlineSlowResponse(param, methodName, call, clientAddress, startTime,
+processingTime, qTime, responseSize, userName, className, 
responseInfo);
+}
+  }
+
+  /**
+   * Add too slow log to ringbuffer for retrieval of latest n slow logs
+   *
+   * @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 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 className ClassName of the SlowLog call
+   * @param responseInfo Base information map that is reported regardless of 
type of call
+   */
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
 
 Review comment:
   I think the implementation should be little easy to reason about.. just 
running it in a thread in a fork-join pool seems hacky.. please refer to my 
other comment.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-16 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379950358
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
 ##
 @@ -1507,4 +1507,24 @@
*/
   CompletableFuture isSnapshotCleanupEnabled();
 
+  /**
+   * Retrieves online slowlog responses from in memory ringbuffer maintained by
+   * RegionServers
+   *
+   * @param serverNames Server names to get slowlog responses from
+   * @param slowLogQueryFilter filter to be used if provided
+   * @return Online slowlog response list. The return value wrapped by a 
{@link CompletableFuture}
+   */
+  CompletableFuture> getSlowLogResponses(final 
List serverNames,
 
 Review comment:
   I think you misunderstood my comment.. what I meant was there could be 
duplicate servernames in the list and you'll end up processing them multiple 
times. If duplicates are not allowed in the 'input' servernames, it should be a 
set.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379688843
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -85,6 +90,12 @@
   protected static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
   = new CallQueueTooBigException();
 
+  private static final String MULTI_GETS = "multi.gets";
 
 Review comment:
   nit: I think these are out of place..move to one of the util classes?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379691406
 
 

 ##
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
 ##
 @@ -2262,4 +2262,30 @@ boolean snapshotCleanupSwitch(final boolean on, final 
boolean synchronous)
*/
   boolean isSnapshotCleanupEnabled() throws IOException;
 
+
+  /**
+   * Retrieves online slowlog responses from in memory ringbuffer maintained by
+   * RegionServers
+   *
+   * @param serverNames Server names to get slowlog responses from
+   * @param slowLogQueryFilter filter to be used if provided
+   * @return online slowlog response list
+   * @throws IOException if a remote or network exception occurs
+   */
+  List getSlowLogResponses(final List serverNames,
 
 Review comment:
   Like I mentioned elsewhere, I think these should getSlowRequests() and 
clearSlowRequestBuffer() in the interest of keeping the API simple and easy to 
follow..


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379693393
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
 ##
 @@ -0,0 +1,144 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Online SlowLog Provider Service that keeps slow logs in the ring buffer
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SlowLogRecorder {
 
 Review comment:
   This needs more java docs, especially around thread safety and how it works? 
Mind adding?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379693205
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
 ##
 @@ -0,0 +1,144 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Online SlowLog Provider Service that keeps slow logs in the ring buffer
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SlowLogRecorder {
+
+  private final Disruptor disruptor;
+  private final SlowLogEventHandler slowLogEventHandler;
+  private final int eventCount;
+  private final boolean isOnlineSlowLogProviderEnabled;
+
+  private static final String SLOW_LOG_RING_BUFFER_SIZE =
+"hbase.regionserver.slowlog.ringbuffer.size";
+
+  /**
+   * Initialize disruptor with configurable ringbuffer size
+   */
+  public SlowLogRecorder(Configuration conf) {
+isOnlineSlowLogProviderEnabled = 
conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
+  HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
+
+this.eventCount = conf.getInt(SLOW_LOG_RING_BUFFER_SIZE,
 
 Review comment:
   All this can be short-circuted if isOnlineSlowLogProviderEnabled == false.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379691679
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
 ##
 @@ -1507,4 +1507,24 @@
*/
   CompletableFuture isSnapshotCleanupEnabled();
 
+  /**
+   * Retrieves online slowlog responses from in memory ringbuffer maintained by
 
 Review comment:
   nit: I think inmemory ringbuffer part can be removed I think, that is server 
side implementation that client doesn't care about.


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379698328
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,312 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestSlowLogRecorder {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestSlowLogRecorder.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestSlowLogRecorder.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private SlowLogRecorder slowLogRecorder;
+
+  private static Configuration applySlowLogRecorderConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
+return conf;
+  }
+
+  private SlowLogPayload getSlowLogPayload(int i) {
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails("call_" + i)
+  .setClientAddress("client_" + i)
+  .setMethodName("method_" + i)
+  .setUserName("userName_" + i)
+  .setMultiGets(i)
+  .setMultiMutations(i)
+  .setMultiServiceCalls(i)
+  .setProcessingTime(i + 500)
+  .setQueueTime(i + 400)
+  .setResponseSize(i + 200)
+  .setStartTime(EnvironmentEdgeManager.currentTime())
+  .setServerClass("class_" + i)
+  .setParam("param_" + i)
+  .build();
+return slowLogPayload;
+  }
+
+  /**
+   * confirm that for a ringbuffer of slow logs, payload on given index of 
buffer
+   * has expected elements
+   *
+   * @param i index of ringbuffer logs
+   * @param j data value that was put on index i
+   * @param slowLogPayloads list of payload retrieved from {@link 
SlowLogRecorder}
+   */
+  private void confirmPayloadParams(int i, int j, List 
slowLogPayloads) {
+Assert.assertEquals(slowLogPayloads.get(i).getClientAddress(), "client_" + 
j);
+Assert.assertEquals(slowLogPayloads.get(i).getCallDetails(), "call_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getMethodName(), "method_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getUserName(), "userName_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getProcessingTime(), j + 500);
+Assert.assertEquals(slowLogPayloads.get(i).getQueueTime(), j + 400);
+Assert.assertEquals(slowLogPayloads.get(i).getResponseSize(), j + 200);
+Assert.assertEquals(slowLogPayloads.get(i).getServerClass(), "class_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getParam(), "param_" + j);
+  }
+
+  @Test
+  public void testOnlieSlowLogConsumption() throws Exception {
+Configuration conf = applySlowLogRecorderConf(8);
+slowLogRecorder = new SlowLogRecorder(conf);
+AdminProtos.SlowLogResponseRequest request =
+  AdminProtos.SlowLogResponseRequest.newBuilder().build();
+Assert.assertEquals(slowLogRecorder.getSlowLogPayl

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379692739
 
 

 ##
 File path: hbase-common/src/main/resources/hbase-default.xml
 ##
 @@ -1956,4 +1956,27 @@ possible configurations would overwhelm and obscure the 
important.
   enable this feature.
 
   
+  
+hbase.regionserver.slowlog.ringbuffer.size
+256
+
+  Default size of ringbuffer to be maintained by each RegionServer in order
+  to store online slowlog responses. This is an in-memory ring buffer of
+  requests that were judged to be too slow in addition to the 
responseTooSlow
+  logging. The in-memory representation would be complete.
+  For more details, please look into Doc Section:
+  Get Slow Response Log from shell
+
+  
+  
+hbase.regionserver.slowlog.buffer.enabled
 
 Review comment:
   Why two configs? if "hbase.regionserver.slowlog.ringbuffer.size" > 0, it 
means it is enabled..(in the interest of reducing the number of knobs)


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379698898
 
 

 ##
 File path: 
hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
 ##
 @@ -1157,6 +1158,18 @@ public boolean isSnapshotCleanupEnabled() {
 throw new NotImplementedException("isSnapshotCleanupEnabled not supported 
in ThriftAdmin");
   }
 
+  @Override
+  public List getSlowLogResponses(final List 
serverNames)
+  throws IOException {
+throw new NotImplementedException("getSlowLogResponses not supported in 
ThriftAdmin");
 
 Review comment:
   Ya, I had a similar question about thrift interface when I was working on 
HBASE-23330, but it looks like it is supported. So we should probably do this 
too.. it is not too much code right? (just defining the equivalent thrift 
specs).


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379691734
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
 ##
 @@ -1507,4 +1507,24 @@
*/
   CompletableFuture isSnapshotCleanupEnabled();
 
+  /**
+   * Retrieves online slowlog responses from in memory ringbuffer maintained by
+   * RegionServers
+   *
+   * @param serverNames Server names to get slowlog responses from
+   * @param slowLogQueryFilter filter to be used if provided
+   * @return Online slowlog response list. The return value wrapped by a 
{@link CompletableFuture}
+   */
+  CompletableFuture> getSlowLogResponses(final 
List serverNames,
 
 Review comment:
   nit: switch to a Set<> ?, we don't want duplicates?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379685711
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
 ##
 @@ -0,0 +1,89 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * SlowLog params object that contains detailed info as params and region name 
: to be used
+ * for filter purpose
+ */
+@InterfaceAudience.Private
+public class SlowLogParams {
 
 Review comment:
   I have this very basic question(may be discussed elsewhere but I don't see 
it)..why does everything contain"log"
   
   SlowLogParams
   SlowLogResponses
   SlowLogQueryFilter...
   
   Reading through the jira, I think we should be calling it SlowRequestBuffer 
or something like that? I don't understand the "log" part here.. reading 
through the patch, it looks like we are book-keeping this data *after* the 
logging is done, right?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379698112
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,137 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final SlowLogPayload[] slowLogPayloads;
+
+  private AtomicInteger slowLogPayloadIdx = new AtomicInteger();
+
+  private static final Lock LOCK = new ReentrantLock();
+
+  private static final int WRITE_LOCK_WAIT = 1;
+  private static final int READ_LOCK_WAIT = 1;
+
+  SlowLogEventHandler(int eventCount) {
+this.slowLogPayloads = new SlowLogPayload[eventCount];
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferTruck event, long sequence, boolean 
endOfBatch) throws Exception {
+SlowLogPayload slowLogPayload = event.getPayload();
+if (LOG.isTraceEnabled()) {
+  LOG.trace("Received Slow Log Event. RingBuffer sequence: {}, 
isEndOfBatch: {}, " +
+  "Event Call: {}", sequence, endOfBatch,
+slowLogPayload.getCallDetails());
+}
+try {
+  if (LOCK.tryLock(WRITE_LOCK_WAIT, TimeUnit.SECONDS)) {
+int index = slowLogPayloadIdx.getAndSet(
+  (slowLogPayloadIdx.get() + 1) % slowLogPayloads.length);
+slowLogPayloads[index] = slowLogPayload;
+  }
 
 Review comment:
   Also, looking at the implementation, we are always blocking on a lock, so 
all writes are pretty much serialized.. so I'd guess my suggestion won't be any 
less performant..(I think)..


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379688135
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,65 @@ 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 && tooSlow) ? "TooLarge & TooSlow"
+  : (tooSlow ? "TooSlow" : "TooLarge");
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.slowLogRecorder != null) {
+  logOnlineSlowResponse(param, methodName, call, clientAddress, startTime,
+processingTime, qTime, responseSize, userName, className, 
responseInfo);
+}
+  }
+
+  /**
+   * Add too slow log to ringbuffer for retrieval of latest n slow logs
+   *
+   * @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 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 className ClassName of the SlowLog call
+   * @param responseInfo Base information map that is reported regardless of 
type of call
+   */
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
 
 Review comment:
   Shouldn't this happen asynchronously? This adds latency to an already slow 
request (and you are serializing stuff which is even slower with larger 
requests), client doesn't care whether we log the request to the ring buffer or 
not.. 


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379697765
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,137 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final SlowLogPayload[] slowLogPayloads;
+
+  private AtomicInteger slowLogPayloadIdx = new AtomicInteger();
+
+  private static final Lock LOCK = new ReentrantLock();
+
+  private static final int WRITE_LOCK_WAIT = 1;
+  private static final int READ_LOCK_WAIT = 1;
+
+  SlowLogEventHandler(int eventCount) {
+this.slowLogPayloads = new SlowLogPayload[eventCount];
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferTruck event, long sequence, boolean 
endOfBatch) throws Exception {
+SlowLogPayload slowLogPayload = event.getPayload();
+if (LOG.isTraceEnabled()) {
+  LOG.trace("Received Slow Log Event. RingBuffer sequence: {}, 
isEndOfBatch: {}, " +
+  "Event Call: {}", sequence, endOfBatch,
+slowLogPayload.getCallDetails());
+}
+try {
+  if (LOCK.tryLock(WRITE_LOCK_WAIT, TimeUnit.SECONDS)) {
+int index = slowLogPayloadIdx.getAndSet(
+  (slowLogPayloadIdx.get() + 1) % slowLogPayloads.length);
+slowLogPayloads[index] = slowLogPayload;
+  }
 
 Review comment:
   I kind of agree with @ndimiduk  here. Its odd that we are missing events.
   
   I'm not super familiar with disruptor's implementation of ring buffer (which 
is obviously super complex and optimized for certain ingest pattern).. but I 
think we can simplify it by adding a simple thread-safe wrapper over 
EvictingQueue...
   
   The wrapper runs in a simple thread that consumes input from a shared-buffer 
to which RpcServer writes and this thread keeps polling entries and appends to 
an underlying EvictingQueue...that way 
   
   - we don't miss any events and the whole thing is asynchronous too (since 
the RpcServer just writes the entry to the shared buffer).
   - Most of the logic can be folded into the wrapper class (currently there is 
a lot of serialization stuff happening in the context of RpcServer).. this 
thread can also "log" entries.
   - Fewer lines of code.
   
   Thoughts?


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


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-14 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r379692118
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogRecord.java
 ##
 @@ -0,0 +1,317 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+import org.apache.hbase.thirdparty.com.google.gson.JsonSerializer;
+
+/**
+ * SlowLog payload for client
+ */
+@InterfaceAudience.Private
+final public class SlowLogRecord {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogRecord.class, (JsonSerializer)
+  (slowLogPayload, type, jsonSerializationContext) -> {
+Gson gson = new Gson();
+JsonObject jsonObj = (JsonObject) gson.toJsonTree(slowLogPayload);
+if (slowLogPayload.getMultiGetsCount() == 0) {
+  jsonObj.remove("multiGetsCount");
+}
+if (slowLogPayload.getMultiMutationsCount() == 0) {
+  jsonObj.remove("multiMutationsCount");
+}
+if (slowLogPayload.getMultiServiceCalls() == 0) {
+  jsonObj.remove("multiServiceCalls");
+}
+return jsonObj;
+  }).create();
+
+  private long startTime;
+  private int processingTime;
+  private int queueTime;
+  private long responseSize;
+  private String clientAddress;
+  private String serverClass;
+  private String methodName;
+  private String callDetails;
+  private String param;
+  // we don't want to serialize region name, it is just for the filter purpose
+  // hence avoiding deserialization
+  private transient String regionName;
+  private String userName;
+  private int multiGetsCount;
+  private int multiMutationsCount;
+  private int multiServiceCalls;
+
+  public long getStartTime() {
+return startTime;
+  }
+
+  public int getProcessingTime() {
+return processingTime;
+  }
+
+  public int getQueueTime() {
+return queueTime;
+  }
+
+  public long getResponseSize() {
+return responseSize;
+  }
+
+  public String getClientAddress() {
+return clientAddress;
+  }
+
+  public String getServerClass() {
+return serverClass;
+  }
+
+  public String getMethodName() {
+return methodName;
+  }
+
+  public String getCallDetails() {
+return callDetails;
+  }
+
+  public String getParam() {
+return param;
+  }
+
+  public String getRegionName() {
+return regionName;
+  }
+
+  public String getUserName() {
+return userName;
+  }
+
+  public int getMultiGetsCount() {
+return multiGetsCount;
+  }
+
+  public int getMultiMutationsCount() {
+return multiMutationsCount;
+  }
+
+  public int getMultiServiceCalls() {
+return multiServiceCalls;
+  }
+
+  private SlowLogRecord(final long startTime, final int processingTime, final 
int queueTime,
 
 Review comment:
   nit: use a builder?


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