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

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

 ##
 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");
+  }
+
+  @Override
+  public List clearSlowLogResponses(final List 
serverNames)
+  throws IOException {
+throw new NotImplementedException("clearSlowLogsResponses not supported in 
ThriftAdmin");
 
 Review comment:
   Hey @saintstack -- we are not maintaining parity with the thrift API, boss?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: src/main/asciidoc/_chapters/hbase-default.adoc
 ##
 @@ -2207,3 +2207,41 @@ The percent of region server RPC threads failed to 
abort RS.
 .Default
 `-1`
 
+
+[[hbase.regionserver.slowlog.ringbuffer.size]]
+*`hbase.regionserver.slowlog.ringbuffer.size`*::
++
+.Description
+
+  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
+
+
++
+.Default
+`256`
+
+
+
+[[hbase.regionserver.slowlog.buffer.enabled]]
+*`hbase.regionserver.slowlog.buffer.enabled`*::
++
+.Description
+
+  Indicates whether RegionServers have ring buffer running for storing
+  Online Slow logs in FIFO manner with limited entries. The size of
+  the ring buffer is indicated by config: 
hbase.regionserver.slowlog.ringbuffer.size
+  The default value is false, turn this on and get latest slowlog
+  responses with complete data.
+  For more details, please look into Doc Section:
 
 Review comment:
   TODO?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-shell/src/main/ruby/hbase/admin.rb
 ##
 @@ -1453,6 +1453,97 @@ def list_decommissioned_regionservers
   @admin.listDecommissionedRegionServers
 end
 
+
#--
+# Retrieve SlowLog Responses from RegionServers
+def get_slowlog_responses(server_names, args)
+  unless server_names.is_a?(Array) || server_names.is_a?(String)
+raise(ArgumentError,
+  "#{server_names.class} of #{server_names.inspect} is not of 
Array/String type")
+  end
+  if server_names == '*'
+server_names = getServerNames([], true)
+  else
+server_names_list = get_server_names(server_names)
+server_names = getServerNames(server_names_list, false)
+  end
+  slow_log_responses = @admin.getSlowLogResponses(server_names)
+  filtered_log_responses = filter_slow_responses(args,
+ slow_log_responses)
+  puts 'Retrieved SlowLog Responses from RegionServers.'
+  if args.empty?
+puts slow_log_responses.to_s
+  else
+puts filtered_log_responses.to_s
+  end
+end
+
+def filter_slow_responses(args, slow_log_responses)
+  filtered_log_responses = java.util.ArrayList.new
 
 Review comment:
   See my comment on JIRA.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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:
   I think we should support it in the thrift interface... we support a thrift 
interface, don't we? Or is it deprecated? I don't know the current state of 
thrift.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
 ##
 @@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+
+syntax = "proto2";
+
+// This file contains protocol buffers that are used for Online TooSlowLogs
+// To be used as Ring Buffer payload
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
+option java_outer_classname = "TooSlowLog";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message SlowLogPayload {
+required int64 start_time = 1;
+required int32 processing_time = 2;
+required int32 queue_time = 3;
+required int64 response_size = 4;
+required string client_address = 5;
+required string server_class = 6;
+required string method_name = 7;
+required string call_details = 8;
+required string param = 9;
+required string user_name = 10;
 
 Review comment:
   Okay, I miss-understood how filtering is to work. See my recent comment on 
JIRA for thoughts on pushing the filtering server-side.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,150 @@
+/*
+ *
+ * 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_SEC = 1;
+  private static final int READ_LOCK_WAIT_SEC = 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(RingBufferEnvelope 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_SEC, TimeUnit.SECONDS)) {
+try {
+  final int index =
+slowLogPayloadIdx.getAndUpdate(val -> (val + 1) % 
slowLogPayloads.length);
+  slowLogPayloads[index] = slowLogPayload;
+} finally {
+  LOCK.unlock();
+}
+  } else {
+LOG.warn("Failed to acquire write lock while processing SlowLogEvent 
from RingBuffer");
+  }
+} catch (InterruptedException e) {
+  LOG.warn("Failed to acquire write lock while processing SlowLogEvent 
from RingBuffer", e);
+}
+  }
+
+  /**
+   * Cleans up slow log payloads
+   *
+   * @return true if slow log payloads are cleaned up, false otherwise
+   */
+  boolean clearSlowLogs() {
+if (LOG.isDebugEnabled()) {
+  LOG.debug("Received request to clean up online slowlog buffer..");
+}
+boolean isCleanedUp = false;
+try {
+  if (LOCK.tryLock(WRITE_LOCK_WAIT_SEC, TimeUnit.SECONDS)) {
+try {
+  Arrays.fill(slowLogPayloads, null);
+  slowLogPayloadIdx.set(0);
+  isCleanedUp = true;
+} finally {
+  LOCK.unlock();
+}
+  } else {
+LOG.warn("Failed to acquire write lock while clearing slow logs");
+  }
+} catch (InterruptedException e) {
+  LOG.warn("Failed to acquire write lock while clearing slow logs", e);
+}
+return isCleanedUp;
+  }
+
+  /**
+   * Retrieve list of slow log payloads
+   *
+   * @return list of slow log payloads
+   */
+  List getSlowLogPayloads() {
+List slowLogPayloadList = null;
+try {
+  if (LOCK.tryLock(READ_LOCK_WAIT_SEC, TimeUnit.SECONDS)) {
+try {
+  slowLogPayloadList = 

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

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

 ##
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
 ##
 @@ -1519,6 +1519,15 @@
   "hbase.master.executor.logreplayops.threads";
   public static final int MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT = 10;
 
+  public static final String SLOW_LOG_RING_BUFFER_SIZE =
 
 Review comment:
   Yes, but adding fields here in `HConstants`, part of our public api, makes 
maintaining these things more painful. I thought we had some developer guidance 
around this, but I'm not finding it at the moment. Maybe someone else here 
remembers those details.
   
   I actually prefer that tests not use these contestants at all, but rather 
write out the full configuration string values. That way it's easier to notice 
when you accidentally break part of your public interface -- your tests that 
depend on that interface start to fail.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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:
   Logging is better than nothing; that satisfies my review query. I think the 
idea of a best-effort recording of observed events is a strange approach for a 
database to take. Updating a metric would also be useful for providing durable, 
light-weight account of these activities.
   
   As for logging to HDFS files, I'm concerned that will cause more overhead 
and harm than good. We can take it up on the sub-task, but I'm currently 
thinking a better place would be a new system table.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,266 @@
+/*
+ *
+ * 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.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(HConstants.SLOW_LOG_RING_BUFFER_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);
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  SlowLogPayload 

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

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,266 @@
+/*
+ *
+ * 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.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(HConstants.SLOW_LOG_RING_BUFFER_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);
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  SlowLogPayload 

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

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.class, (JsonSerializer)
+  (slowLogPayload, type, jsonSerializationContext) -> {
 
 Review comment:
   I didn't realize we had switched internal serialization over to GSON. I'll 
have to study that API.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,51 @@ void logResponse(Message param, String methodName, 
String call, String tag,
   }
 }
   }
-  responseInfo.put("multi.gets", numGets);
-  responseInfo.put("multi.mutations", numMutations);
-  responseInfo.put("multi.servicecalls", numServiceCalls);
+  responseInfo.put(MULTI_GETS, numGets);
+  responseInfo.put(MULTI_MUTATIONS, numMutations);
+  responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 
 Review comment:
   Yeah I think the existing logic is broken too. Why should we not recognize 
an RPC as both too large and too small?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  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 SlowLogPayload(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 {
+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;
+private String regionName;
+

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

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +515,44 @@ 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_SERVICECALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (!tooLarge && this.onlineSlowLogProvider != null
 
 Review comment:
   Since you're doing all this good work to get the `slowResponse` log support, 
we should consider following it with support for `largeResponse` messages as 
well? Seem like most of the hard stuff will be resolved by the first feature. 
Maybe follow-on JIRA.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
 ##
 @@ -426,4 +437,55 @@ public void testAsyncTimeout() throws IOException {
   rpcServer.stop();
 }
   }
+
+  @Test
 
 Review comment:
   Yeah with a void method you're stuck looking for side-effects. Even with a 
non-void method, you should still be looking for side-effects of the call...


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: src/main/asciidoc/_chapters/hbase-default.adoc
 ##
 @@ -2207,3 +2207,39 @@ The percent of region server RPC threads failed to 
abort RS.
 .Default
 `-1`
 
+
+[[hbase.regionserver.slowlog.ringbuffer.size]]
+*`hbase.regionserver.slowlog.ringbuffer.size`*::
++
+.Description
+
+  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: https://issues.apache.org/jira/browse/HBASE-22978
 
 Review comment:
   Instead of pointing off to a JIRA, how about refer to your new "Get Slow 
Response Log from shell" section in the book?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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;
+  }
+} finally {
+  LOCK.unlock();
+}
+  }
+
+  /**
+   * Cleans up slow log payloads
+   *
+   * @return true if slow log payloads are cleaned up, false otherwise
+   */
+  boolean clearSlowLogs() {
+if (LOG.isDebugEnabled()) {
+  LOG.debug("Received request to clean up online slowlog buffer..");
+}
+boolean isCleanedUp = true;
 
 Review comment:
   You'll return `true` in the event that the lock is never acquired.
   
   Start pessimistic with `isCleanedUp = false` and set it `true` only after 
you've successfully done so.
   
   I think this method should look something more like
   
   ```java
   boolean isCleanedUp = false;
   try {
 if (LOCK.tryLock(WRITE_LOCK_WAIT, TimeUnit.SECONDS)) {
   try {
 Arrays.fill(slowLogPayloads, null);
 slowLogPayloadIdx.set(0);
 isCleanedUp = true;
   } finally {
 LOCK.unlock();
   }
 } else {
   LOG.warn("Failed to acquire write lock while clearing slow logs.");
 }
   } catch (InterruptedException e) {
 LOG.warn("Failed to acquire write lock while clearing slow logs.", e);
 Thread.currentThread().interrupt();
   }
   return isCleanedUp;
   ```


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.
 

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

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestOnlineSlowLogProvider.java
 ##
 @@ -0,0 +1,307 @@
+/*
+ *
+ * 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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestOnlineSlowLogProvider {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestOnlineSlowLogProvider.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestOnlineSlowLogProvider.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private Configuration getOnlineSlowLogConf(int eventSize) {
 
 Review comment:
   A `get` method that modifies something under the covers? How about 
"applyOnlineSlowLogConf"?
   
   Also, an instance method that modifies static state leads to flakey tests 
and limits parallelism. Please refactor.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-shell/src/main/ruby/hbase/admin.rb
 ##
 @@ -1453,6 +1453,97 @@ def list_decommissioned_regionservers
   @admin.listDecommissionedRegionServers
 end
 
+
#--
+# Retrieve SlowLog Responses from RegionServers
+def get_slowlog_responses(server_names, args)
+  unless server_names.is_a?(Array) || server_names.is_a?(String)
+raise(ArgumentError,
+  "#{server_names.class} of #{server_names.inspect} is not of 
Array/String type")
+  end
+  if server_names == '*'
+server_names = getServerNames([], true)
+  else
+server_names_list = get_server_names(server_names)
+server_names = getServerNames(server_names_list, false)
+  end
+  slow_log_responses = @admin.getSlowLogResponses(server_names)
+  filtered_log_responses = filter_slow_responses(args,
+ slow_log_responses)
+  puts 'Retrieved SlowLog Responses from RegionServers.'
+  if args.empty?
+puts slow_log_responses.to_s
+  else
+puts filtered_log_responses.to_s
+  end
+end
+
+def filter_slow_responses(args, slow_log_responses)
+  filtered_log_responses = java.util.ArrayList.new
+  slow_log_responses.each do |slow_log|
+if args.key? 'REGION_NAME'
+  region_name = args['REGION_NAME']
+  unless region_name.nil?
+if region_name.eql?(slow_log.getRegionName)
+  filtered_log_responses.add(slow_log)
+  next
+end
+  end
+end
+if args.key? 'TABLE_NAME'
+  table_name = args['TABLE_NAME']
+  unless table_name.nil?
+if slow_log.getRegionName.start_with?(table_name)
+  filtered_log_responses.add(slow_log)
+  next
+end
+  end
+end
+if args.key? 'CLIENT_IP'
+  client_ip = args['CLIENT_IP']
+  unless client_ip.nil?
+if client_ip.eql?(slow_log.getClientAddress)
+  filtered_log_responses.add(slow_log)
+  next
+end
+  end
+end
+if args.key? 'USER'
+  user = args['USER']
+  unless user.nil?
+if user.eql?(slow_log.getUserName)
+  filtered_log_responses.add(slow_log)
+  next
+end
+  end
+end
+  end
+  filtered_log_responses
+end
+
+
#--
+# Clears SlowLog Responses from RegionServers
+def clear_slowlog_responses(server_names)
+  unless server_names.nil? || server_names.is_a?(Array) || 
server_names.is_a?(String)
+raise(ArgumentError,
+  "#{server_names.class} of #{server_names.inspect} is not of 
correct type")
+  end
+  if server_names.nil?
+server_names = getServerNames([], true)
+  else
+server_names_list = get_server_names(server_names)
+server_names = getServerNames(server_names_list, false)
+  end
+  clear_log_responses = @admin.clearSlowLogResponses(server_names)
+  clear_log_success_count = 0
+  clear_log_responses.each do |response|
+if response
+  clear_log_success_count += 1
+end
+  end
+  puts 'Cleared Slowlog responses from ' \
+   "#{clear_log_success_count}/#{clear_log_responses.size} 
RegionServers"
 
 Review comment:
   Nice summary. If this isn't 100%, where does the operator go to find out 
more? I guess the RS logs, but of which host(s)? We assume all our operators 
have a log aggregation tool?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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);
 
 Review comment:
   Calling `get()` from within `getAndSet()` defeats the atomicity of an 
`AtomicInteger`, yeah? I think you meant to write
   
   ```java
   final int index = slowLogPayloadIdx.getAndUpdate(val -> (val + 1) % 
slowLogPayloads.length);
   ```
   


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestOnlineSlowLogProvider.java
 ##
 @@ -0,0 +1,307 @@
+/*
+ *
+ * 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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestOnlineSlowLogProvider {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestOnlineSlowLogProvider.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestOnlineSlowLogProvider.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private Configuration getOnlineSlowLogConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt(HConstants.SLOW_LOG_RING_BUFFER_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 
OnlineSlowLogProvider}
+   */
+  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 = getOnlineSlowLogConf(8);
+onlineSlowLogProvider = new OnlineSlowLogProvider(conf);
+Assert.assertEquals(onlineSlowLogProvider.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 

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

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestOnlineSlowLogProvider.java
 ##
 @@ -0,0 +1,307 @@
+/*
+ *
+ * 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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestOnlineSlowLogProvider {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestOnlineSlowLogProvider.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestOnlineSlowLogProvider.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private Configuration getOnlineSlowLogConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt(HConstants.SLOW_LOG_RING_BUFFER_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 
OnlineSlowLogProvider}
+   */
+  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 = getOnlineSlowLogConf(8);
+onlineSlowLogProvider = new OnlineSlowLogProvider(conf);
+Assert.assertEquals(onlineSlowLogProvider.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 

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

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

 ##
 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;
+  }
+} finally {
+  LOCK.unlock();
+}
+  }
+
+  /**
+   * Cleans up slow log payloads
+   *
+   * @return true if slow log payloads are cleaned up, false otherwise
+   */
+  boolean clearSlowLogs() {
+if (LOG.isDebugEnabled()) {
+  LOG.debug("Received request to clean up online slowlog buffer..");
+}
+boolean isCleanedUp = true;
+try {
+  if (LOCK.tryLock(WRITE_LOCK_WAIT, TimeUnit.SECONDS)) {
+for (int i = 0; i < slowLogPayloads.length; i++) {
+  slowLogPayloads[i] = null;
+}
+slowLogPayloadIdx.set(0);
+  }
+} catch (InterruptedException e) {
+  isCleanedUp = false;
+  LOG.error("Error while cleaning up slow logs buffer...", e);
+} finally {
+  LOCK.unlock();
+}
+return isCleanedUp;
+  }
+
+  /**
+   * Retrieve list of slow log payloads
+   *
+   * @return list of slow log payloads
+   */
+  List getSlowLogPayloads() {
+List slowLogPayloadList = null;
+try {
+  if (LOCK.tryLock(READ_LOCK_WAIT, TimeUnit.SECONDS)) {
+slowLogPayloadList = Arrays.stream(slowLogPayloads)
+  .filter(Objects::nonNull)
+  .collect(Collectors.toList());
+  }
+} catch (InterruptedException e) {
+  LOG.error("Error while reading slow logs buffer...", e);
 
 Review comment:
   Is swallowing the interrupt the right thing to do? Someone is requesting 
this thread terminate...


This is an 

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

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestOnlineSlowLogProvider.java
 ##
 @@ -0,0 +1,307 @@
+/*
+ *
+ * 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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestOnlineSlowLogProvider {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestOnlineSlowLogProvider.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestOnlineSlowLogProvider.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private Configuration getOnlineSlowLogConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt(HConstants.SLOW_LOG_RING_BUFFER_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 
OnlineSlowLogProvider}
+   */
+  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 = getOnlineSlowLogConf(8);
+onlineSlowLogProvider = new OnlineSlowLogProvider(conf);
+Assert.assertEquals(onlineSlowLogProvider.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 

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

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -215,6 +224,17 @@
*/
   private RSRpcServices rsRpcServices;
 
+
+  /**
+   * Use to add online slowlog responses
+   */
+  private OnlineSlowLogProvider onlineSlowLogProvider;
 
 Review comment:
   Oh I see... yeah this is rough.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-shell/src/main/ruby/shell/commands/get_slowlog_responses.rb
 ##
 @@ -0,0 +1,65 @@
+#
+#
+# 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.
+
+# Retrieve latest slowlog responses maintained in memory by RegionServers
+
+module Shell
+  module Commands
+# Retrieve latest slowlog responses
+class GetSlowlogResponses < Command
+  def help
+<<-EOF
+Retrieve latest SlowLog Responses maintained by each or specific RegionServers.
+Specify '*' to include all RS otherwise array of server names for specific
+RS. A server name is the host, port plus startcode of a RegionServer.
+e.g.: host187.example.com,60020,1289493121758 (find servername in
+master ui or when you do detailed status in shell)
+
+Provide optional filter parameters as Hash
+
+Examples:
 
 Review comment:
   Nice docs. Examples really help.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-shell/src/main/ruby/hbase/admin.rb
 ##
 @@ -1527,6 +1618,16 @@ def stop_master
 def stop_regionserver(hostport)
   @admin.stopRegionServer(hostport)
 end
+
+
#--
+# Get list of server names
+def get_server_names(server_names)
 
 Review comment:
   nit: this is more of a `_to_java` kind of method. Reconsider the name.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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;
+  }
+} finally {
+  LOCK.unlock();
+}
+  }
+
+  /**
+   * Cleans up slow log payloads
+   *
+   * @return true if slow log payloads are cleaned up, false otherwise
+   */
+  boolean clearSlowLogs() {
+if (LOG.isDebugEnabled()) {
+  LOG.debug("Received request to clean up online slowlog buffer..");
+}
+boolean isCleanedUp = true;
+try {
+  if (LOCK.tryLock(WRITE_LOCK_WAIT, TimeUnit.SECONDS)) {
+for (int i = 0; i < slowLogPayloads.length; i++) {
+  slowLogPayloads[i] = null;
+}
+slowLogPayloadIdx.set(0);
+  }
 
 Review comment:
   same lack of `else` clause.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestOnlineSlowLogProvider.java
 ##
 @@ -0,0 +1,307 @@
+/*
+ *
+ * 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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestOnlineSlowLogProvider {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestOnlineSlowLogProvider.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestOnlineSlowLogProvider.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private Configuration getOnlineSlowLogConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt(HConstants.SLOW_LOG_RING_BUFFER_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 
OnlineSlowLogProvider}
+   */
+  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 = getOnlineSlowLogConf(8);
+onlineSlowLogProvider = new OnlineSlowLogProvider(conf);
+Assert.assertEquals(onlineSlowLogProvider.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 

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

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

 ##
 File path: hbase-shell/src/main/ruby/hbase/admin.rb
 ##
 @@ -1453,6 +1453,97 @@ def list_decommissioned_regionservers
   @admin.listDecommissionedRegionServers
 end
 
+
#--
+# Retrieve SlowLog Responses from RegionServers
+def get_slowlog_responses(server_names, args)
+  unless server_names.is_a?(Array) || server_names.is_a?(String)
+raise(ArgumentError,
+  "#{server_names.class} of #{server_names.inspect} is not of 
Array/String type")
+  end
+  if server_names == '*'
+server_names = getServerNames([], true)
+  else
+server_names_list = get_server_names(server_names)
+server_names = getServerNames(server_names_list, false)
+  end
+  slow_log_responses = @admin.getSlowLogResponses(server_names)
+  filtered_log_responses = filter_slow_responses(args,
+ slow_log_responses)
+  puts 'Retrieved SlowLog Responses from RegionServers.'
+  if args.empty?
+puts slow_log_responses.to_s
+  else
+puts filtered_log_responses.to_s
+  end
+end
+
+def filter_slow_responses(args, slow_log_responses)
+  filtered_log_responses = java.util.ArrayList.new
 
 Review comment:
   Oh, here's where your filters get applied. Please perform filtering on the 
server-side, where it can be distributed across all the region servers 
involved, and avoid the network and client overhead.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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:
   Why not? We adding thrift serialization support in a separate patch?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-shell/src/main/ruby/hbase/admin.rb
 ##
 @@ -1453,6 +1453,97 @@ def list_decommissioned_regionservers
   @admin.listDecommissionedRegionServers
 end
 
+
#--
+# Retrieve SlowLog Responses from RegionServers
+def get_slowlog_responses(server_names, args)
+  unless server_names.is_a?(Array) || server_names.is_a?(String)
+raise(ArgumentError,
+  "#{server_names.class} of #{server_names.inspect} is not of 
Array/String type")
+  end
+  if server_names == '*'
+server_names = getServerNames([], true)
+  else
+server_names_list = get_server_names(server_names)
+server_names = getServerNames(server_names_list, false)
+  end
+  slow_log_responses = @admin.getSlowLogResponses(server_names)
+  filtered_log_responses = filter_slow_responses(args,
+ slow_log_responses)
+  puts 'Retrieved SlowLog Responses from RegionServers.'
 
 Review comment:
   nit: What's with the ellipsis everywhere? Here and in log messages, they 
don't add anything for the user.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RingBufferTruck.java
 ##
 @@ -0,0 +1,63 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * A truck to carry payload in the slow log ring buffer that serves as online 
buffer
+ * to provide latest TooSlowLog
+ */
+@InterfaceAudience.Private
+final class RingBufferTruck {
 
 Review comment:
   "Truck" is a bit of an odd name, yeah? What about "Envelope"?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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;
+  }
+} finally {
+  LOCK.unlock();
+}
+  }
+
+  /**
+   * Cleans up slow log payloads
+   *
+   * @return true if slow log payloads are cleaned up, false otherwise
+   */
+  boolean clearSlowLogs() {
+if (LOG.isDebugEnabled()) {
+  LOG.debug("Received request to clean up online slowlog buffer..");
+}
+boolean isCleanedUp = true;
+try {
+  if (LOCK.tryLock(WRITE_LOCK_WAIT, TimeUnit.SECONDS)) {
+for (int i = 0; i < slowLogPayloads.length; i++) {
+  slowLogPayloads[i] = null;
+}
+slowLogPayloadIdx.set(0);
+  }
+} catch (InterruptedException e) {
+  isCleanedUp = false;
+  LOG.error("Error while cleaning up slow logs buffer...", e);
+} finally {
+  LOCK.unlock();
+}
+return isCleanedUp;
+  }
+
+  /**
+   * Retrieve list of slow log payloads
+   *
+   * @return list of slow log payloads
+   */
+  List getSlowLogPayloads() {
+List slowLogPayloadList = null;
+try {
+  if (LOCK.tryLock(READ_LOCK_WAIT, TimeUnit.SECONDS)) {
+slowLogPayloadList = Arrays.stream(slowLogPayloads)
+  .filter(Objects::nonNull)
+  .collect(Collectors.toList());
+  }
 
 Review comment:
   same lack of `else` clause.


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 

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

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

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestOnlineSlowLogProvider.java
 ##
 @@ -0,0 +1,307 @@
+/*
+ *
+ * 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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestOnlineSlowLogProvider {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestOnlineSlowLogProvider.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestOnlineSlowLogProvider.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private Configuration getOnlineSlowLogConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt(HConstants.SLOW_LOG_RING_BUFFER_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 
OnlineSlowLogProvider}
+   */
+  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 = getOnlineSlowLogConf(8);
+onlineSlowLogProvider = new OnlineSlowLogProvider(conf);
+Assert.assertEquals(onlineSlowLogProvider.getSlowLogPayloads().size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 

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

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

 ##
 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;
+  }
+} finally {
+  LOCK.unlock();
 
 Review comment:
   What happens when `unlock` is called by `tryLock()` returned `false`? Is it 
a no-op or does it throw `IllegalMonitorStateException` ?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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:
   what about the `false` condition? Just ignore the event? Seems like the 
wrong thing to do.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/OnlineSlowLogProvider.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.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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Online SlowLog Provider Service
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OnlineSlowLogProvider {
 
 Review comment:
   This class doesn't provide SlowLogs, and I'm unclear on what makes it 
online. How about naming it `SlowLogLogger` or `SlowLogRecorder` or 
`SlowLogHandlingService` ? I guess others don't like the name "Service..."


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,51 @@ void logResponse(Message param, String methodName, 
String call, String tag,
   }
 }
   }
-  responseInfo.put("multi.gets", numGets);
-  responseInfo.put("multi.mutations", numMutations);
-  responseInfo.put("multi.servicecalls", numServiceCalls);
+  responseInfo.put(MULTI_GETS, numGets);
+  responseInfo.put(MULTI_MUTATIONS, numMutations);
+  responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.onlineSlowLogProvider != null
+&& !call.startsWith(GET_SLOW_LOG_RESPONSES)
 
 Review comment:
   Why exclude these? It's quite possible that these RPCs could be taking a 
long time...


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
 ##
 @@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+
+syntax = "proto2";
+
+// This file contains protocol buffers that are used for Online TooSlowLogs
+// To be used as Ring Buffer payload
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
+option java_outer_classname = "TooSlowLog";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message SlowLogPayload {
+required int64 start_time = 1;
+required int32 processing_time = 2;
+required int32 queue_time = 3;
+required int64 response_size = 4;
+required string client_address = 5;
+required string server_class = 6;
+required string method_name = 7;
+required string call_details = 8;
+required string param = 9;
+required string user_name = 10;
 
 Review comment:
   So this POJO is used both for storing/serializing values and also as the 
query rpc filter params list? Maybe those two concerns should have two 
different payload objects...


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,51 @@ void logResponse(Message param, String methodName, 
String call, String tag,
   }
 }
   }
-  responseInfo.put("multi.gets", numGets);
-  responseInfo.put("multi.mutations", numMutations);
-  responseInfo.put("multi.servicecalls", numServiceCalls);
+  responseInfo.put(MULTI_GETS, numGets);
+  responseInfo.put(MULTI_MUTATIONS, numMutations);
+  responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.onlineSlowLogProvider != null
+&& !call.startsWith(GET_SLOW_LOG_RESPONSES)
+&& !call.startsWith(CLEAR_SLOW_LOGS_RESPONSES)) {
+  logOnlineSlowResponse(param, methodName, call, clientAddress, startTime,
+processingTime, qTime, responseSize, userName, className, 
responseInfo);
+}
+  }
+
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+  String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+  String userName, String className, Map responseInfo) {
+// add too slow log to ringbuffer for retrieval of latest n slow logs
+
+try {
+  final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+
+  final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+.setStartTime(startTime)
 
 Review comment:
   same comment re: ordering these alphabetically so that it's easier to spot a 
missed field.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  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 SlowLogPayload(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 {
+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;
+private String regionName;
+

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

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3891,69 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> getSlowLogResponses(
+  List serverNames) {
+CompletableFuture> slowLoadPayloads =
+  CompletableFuture.completedFuture(Collections.emptyList());
+if (CollectionUtils.isEmpty(serverNames)) {
+  return slowLoadPayloads;
+}
+for (ServerName serverName : serverNames) {
+  CompletableFuture> slowLogPayloadFromServer =
+getSlowLogResponseFromServer(serverName);
+  slowLoadPayloads = 
slowLoadPayloads.thenCombine(slowLogPayloadFromServer, (l1, l2) -> {
+List finalSlowLogPayloads = new LinkedList<>();
+finalSlowLogPayloads.addAll(l1);
+finalSlowLogPayloads.addAll(l2);
+return finalSlowLogPayloads;
+  });
+}
+return slowLoadPayloads;
+  }
+
+  private CompletableFuture> getSlowLogResponseFromServer(
+  final ServerName serverName) {
+return this.>newAdminCaller()
+  .action((controller, stub) -> this
+.adminCall(
+  controller, stub, RequestConverter.buildSlowLogResponseRequest(),
+  AdminService.Interface::getSlowLogResponses,
+  ProtobufUtil::toSlowLogPayloads))
+  .serverName(serverName).call();
+  }
+
+  @Override
+  public CompletableFuture> 
clearSlowLogResponses(List serverNames) {
+List> clearSlowLogResponseList = new 
ArrayList<>();
+if (CollectionUtils.isNotEmpty(serverNames)) {
+  for (ServerName serverName : serverNames) {
 
 Review comment:
   nit: you can probably collapse this for-loop and `convertToFutureOfList` 
into a single `stream()` incantation that will be easier to read.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,51 @@ void logResponse(Message param, String methodName, 
String call, String tag,
   }
 }
   }
-  responseInfo.put("multi.gets", numGets);
-  responseInfo.put("multi.mutations", numMutations);
-  responseInfo.put("multi.servicecalls", numServiceCalls);
+  responseInfo.put(MULTI_GETS, numGets);
+  responseInfo.put(MULTI_MUTATIONS, numMutations);
+  responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (tooSlow && this.onlineSlowLogProvider != null
+&& !call.startsWith(GET_SLOW_LOG_RESPONSES)
+&& !call.startsWith(CLEAR_SLOW_LOGS_RESPONSES)) {
+  logOnlineSlowResponse(param, methodName, call, clientAddress, startTime,
+processingTime, qTime, responseSize, userName, className, 
responseInfo);
+}
+  }
+
+  private void logOnlineSlowResponse(Message param, String methodName, String 
call,
+  String clientAddress, long startTime, int processingTime, int qTime, 
long responseSize,
+  String userName, String className, Map responseInfo) {
+// add too slow log to ringbuffer for retrieval of latest n slow logs
 
 Review comment:
   Might as well make this a javadoc comment on the 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  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 SlowLogPayload(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 {
+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;
+private String regionName;
+

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

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

 ##
 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;
 
 Review comment:
   These are quantities of a unit of time; include that unit in the name. 
`WRITE_LOCK_WAIT_SEC`


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3891,69 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> getSlowLogResponses(
+  List serverNames) {
+CompletableFuture> slowLoadPayloads =
+  CompletableFuture.completedFuture(Collections.emptyList());
+if (CollectionUtils.isEmpty(serverNames)) {
+  return slowLoadPayloads;
+}
+for (ServerName serverName : serverNames) {
+  CompletableFuture> slowLogPayloadFromServer =
+getSlowLogResponseFromServer(serverName);
+  slowLoadPayloads = 
slowLoadPayloads.thenCombine(slowLogPayloadFromServer, (l1, l2) -> {
+List finalSlowLogPayloads = new LinkedList<>();
 
 Review comment:
   nit: For larger clusters, you can save yourself a lot of extra allocations 
by using a stream-based approach, something like
   
   ```java
   if (CollectionUtils.isEmpty(serverNames)) {
 return CompletableFuture.completedFuture(Collections.emptyList());
   }
   
   return CompletableFuture.supplyAsync(() -> {
return serverNames.stream()
   .map(this::getSlowLogResponseFromServer)
   .map(CompletableFuture::join)
   .flatMap(List::stream)
   .collect(Collectors.toList());
   });
   ```
   
   Actually, what I think you really want is a solution that won't fall over in 
the face of a very large cluster with lots of slow results. Such a solution 
would involve a user-provided limit on the number of results returned, 
partitioning the server list in batches of size N, processing a batch, and 
short-circuiting the return when a result-count limit is reached. So, to 
support this use case, you'll need to add a limit parameter to the methods (and 
maybe a configuration point providing a default limit) and then implement 
pagination (which I think can also be implemented as a stream over partitions 
plus `Stream.limit`).


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
 ##
 @@ -1519,6 +1519,15 @@
   "hbase.master.executor.logreplayops.threads";
   public static final int MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT = 10;
 
+  public static final String SLOW_LOG_RING_BUFFER_SIZE =
 
 Review comment:
   These constants are consumed in just one place, so they should be moved to 
`OnlineSlowLogProvider`, the class that they influence.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +518,51 @@ void logResponse(Message param, String methodName, 
String call, String tag,
   }
 }
   }
-  responseInfo.put("multi.gets", numGets);
-  responseInfo.put("multi.mutations", numMutations);
-  responseInfo.put("multi.servicecalls", numServiceCalls);
+  responseInfo.put(MULTI_GETS, numGets);
+  responseInfo.put(MULTI_MUTATIONS, numMutations);
+  responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 
 Review comment:
   Again, "too large" and "too slow" are not mutually exclusive. Could be a 
message is logged with both tags, 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -3368,4 +3429,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 SlowLogPayload}
+   *
+   * @param slowLogPayload SlowLog Payload protobuf instance
+   * @return SlowLog Payload for client usecase
+   */
+  private static SlowLogPayload getSlowLogPayload(
+  final TooSlowLog.SlowLogPayload slowLogPayload) {
+SlowLogPayload clientSlowLogPayload = new 
SlowLogPayload.SlowLogPayloadBuilder()
+  .setStartTime(slowLogPayload.getStartTime())
 
 Review comment:
   nit: maybe order these alphabetically so it's easier to spot a missing field.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 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 agree that things are a bit of a mess currently. I was hoping that, with a 
little prodding, you might see an obvious better way ;)
   
   I agree that checking the instance type is better than `getProcessName`.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

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


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


With regards,
Apache Git Services


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

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

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.class, (JsonSerializer)
+  (slowLogPayload, type, jsonSerializationContext) -> {
 
 Review comment:
   I don't know Gson API; I'm more familiar with Jackson, which has lots of 
little annotations for easily tweaking the details of serialization.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/RingBufferTruck.java
 ##
 @@ -0,0 +1,63 @@
+/*
+ *
+ * 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.yetus.audience.InterfaceAudience;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * A truck to carry payload in the slow log ring buffer that serves as online 
buffer
+ * to provide latest TooSlowLog
+ */
+@InterfaceAudience.Private
+final class RingBufferTruck {
+
+  private SlowLogPayload slowLogPayload;
+
+  /**
+   * Load the Truck with {@link SlowLogPayload}
+   *
+   * @param slowLogPayload Payload to load Truck with
+   */
+  public void load(SlowLogPayload slowLogPayload) {
+this.slowLogPayload = slowLogPayload;
+  }
+
+  /**
+   * Retrieve current payload {@link SlowLogPayload} available on Truck and
+   * free up the Truck
+   *
+   * @return Retrieve available payload
+   */
+  public SlowLogPayload getPayload() {
+final SlowLogPayload slowLogPayload = this.slowLogPayload;
+this.slowLogPayload = null;
+return slowLogPayload;
+  }
+
+  /**
+   * To clean up the payload, clear payload reference
+   */
+  public void clearPayload() {
 
 Review comment:
   unused.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

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

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/OnlineSlowLogProvider.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.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.TooSlowLog.SlowLogPayload;
+
+/**
+ * Online SlowLog Provider Service
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OnlineSlowLogProvider {
+
+  private final Disruptor disruptor;
+  private final SlowLogEventHandler slowLogEventHandler;
+  private final int eventCount;
+  private final boolean isOnlineSlowLogProviderEnabled;
+
+  /**
+   * Initialize disruptor with configurable ringbuffer size
+   */
+  public OnlineSlowLogProvider(Configuration conf) {
+isOnlineSlowLogProviderEnabled = 
conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
+  HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
+
+this.eventCount = conf.getInt(HConstants.SLOW_LOG_RING_BUFFER_SIZE,
+  HConstants.DEFAULT_SLOW_LOG_RING_BUFFER_SIZE);
+
+// This is the 'writer' -- a single threaded executor. This single thread 
consumes what is
+// put on the ringbuffer.
+final String hostingThreadName = Thread.currentThread().getName();
+
+// disruptor initialization with BlockingWaitStrategy
+this.disruptor = new Disruptor<>(RingBufferTruck::new,
+  getEventCount(),
+  Threads.newDaemonThreadFactory(hostingThreadName + ".append"),
 
 Review comment:
   Maybe include something to the effect of ".slowlog.append" so that it's not 
confused as a data-path append thread.
   
   I'm curious what the parent thread name ends up being... 
`"RS:0;localhost:"`?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371445755
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3890,72 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> 
getSlowLogResponses(List serverNames)
+  throws IOException {
+CompletableFuture> slowLoadPayloads = null;
+if (CollectionUtils.isNotEmpty(serverNames)) {
+  for (ServerName serverName : serverNames) {
+CompletableFuture> slowLogPayloadFromServer =
+  getSlowLogResponseFromServer(serverName);
+if (slowLoadPayloads == null) {
+  slowLoadPayloads = slowLogPayloadFromServer;
+  continue;
+}
+slowLoadPayloads = 
slowLoadPayloads.thenCombine(slowLogPayloadFromServer, (l1, l2) -> {
+  List finalSlowLogPayloads = new ArrayList<>();
 
 Review comment:
   We have no clue what the length of these lists will be? How do we expect the 
caller to consume the results? Should this be a `LinkedList` instead?


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


With regards,
Apache Git Services


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

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371447304
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3890,72 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> 
getSlowLogResponses(List serverNames)
+  throws IOException {
+CompletableFuture> slowLoadPayloads = null;
 
 Review comment:
   Should the default case instead return a `completedFuture` over an empty 
list?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371433786
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -440,25 +463,28 @@ public void setSecretManager(SecretManager secretMana
* @param param The parameters received in the call.
* @param methodName The name of the method invoked
* @param call The string representation of the call
-   * @param tag  The tag that will be used to indicate this event in the log.
+   * @param tooLarge To indicate if the event is tooLarge or tooSlow
 
 Review comment:
   An event that has `tooLarge=false` does not imply to me that `tooSlow=true`. 
These are independent concepts that should not be mashed into a single 
parameter. The previous parameter "tag" was simply a log annotation, not a 
decision criteria. Here you've replaced it with a decision condition that lacks 
expression. If you want to pursue this change, please include parameters for 
both boolean flags, or implement something like a bit-mask that allows for 
multiple states to be expressed simultaneously.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371427340
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -85,6 +90,10 @@
   protected static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
   = new CallQueueTooBigException();
 
+  private static final String MULTI_GETS = "multi.gets";
+  private static final String MULTI_MUTATIONS = "multi.mutations";
+  private static final String MULTI_SERVICECALLS = "multi.servicecalls";
 
 Review comment:
   "MULTI_SERVICE_CALLS," "multi.service_calls".


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371431623
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -215,6 +224,17 @@
*/
   private RSRpcServices rsRpcServices;
 
+
+  /**
+   * Use to add online slowlog responses
+   */
+  private OnlineSlowLogProvider onlineSlowLogProvider;
 
 Review comment:
   This cannot be final? :(


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371457333
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 ##
 @@ -3818,6 +3828,40 @@ public ExecuteProceduresResponse 
executeProcedures(RpcController controller,
 }
   }
 
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public SlowLogResponses getSlowLogResponses(final RpcController controller,
+  final SlowLogResponseRequest request) throws ServiceException {
 
 Review comment:
   > `throws ServiceException`
   
   No it doesn't.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371439546
 
 

 ##
 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:
   Always instantiate this field? Even when the configuration is disabled?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371448948
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
 ##
 @@ -1507,4 +1507,26 @@
*/
   CompletableFuture isSnapshotCleanupEnabled();
 
+  /**
+   * Retrieves online slowlog responses from in memory ringbuffer maintained by
+   * RegionServers
+   *
+   * @param serverNames Server names to get slowlog responses from
+   * @return Online slowlog response list. The return value wrapped by a 
{@link CompletableFuture}
+   * @throws IOException if a remote or network exception occurs
 
 Review comment:
   Does 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371458942
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 ##
 @@ -3818,6 +3828,40 @@ public ExecuteProceduresResponse 
executeProcedures(RpcController controller,
 }
   }
 
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public SlowLogResponses getSlowLogResponses(final RpcController controller,
+  final SlowLogResponseRequest request) throws ServiceException {
+final OnlineSlowLogProvider onlineSlowLogProvider =
+  this.regionServer.getOnlineSlowLogProvider();
+List slowLogPayloads;
+if (onlineSlowLogProvider != null) {
+  slowLogPayloads = onlineSlowLogProvider.getSlowLogPayloads();
+} else {
+  slowLogPayloads = new ArrayList<>();
+}
+SlowLogResponses slowLogResponses = SlowLogResponses.newBuilder()
+  .addAllSlowLogPayloads(slowLogPayloads)
+  .build();
+return slowLogResponses;
+  }
+
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public ClearSlowLogResponses clearSlowLogsResponses(final RpcController 
controller,
+  final ClearSlowLogResponseRequest request) throws ServiceException {
+final OnlineSlowLogProvider onlineSlowLogProvider =
+  this.regionServer.getOnlineSlowLogProvider();
+boolean slowLogsCleaned = false;
 
 Review comment:
   You have here an ideal use of `Optional`; please use it.
   
   ```
   final boolean slowLogsCleaned = 
Optional.ofNullable(onlineSlowLogProvider)
 .map(OnlineSlowLogProvider::clearSlowLogPayloads)
 .orElse(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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371453733
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  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 SlowLogPayload(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 {
+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;
+private String regionName;
+

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

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371426646
 
 

 ##
 File path: hbase-protocol-shaded/src/main/protobuf/TooSlowLog.proto
 ##
 @@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+
+syntax = "proto2";
+
+// This file contains protocol buffers that are used for Online TooSlowLogs
+// To be used as Ring Buffer payload
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
+option java_outer_classname = "TooSlowLog";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message SlowLogPayload {
+required int64 start_time = 1;
+required int32 processing_time = 2;
+required int32 queue_time = 3;
+required int64 response_size = 4;
+required string client_address = 5;
+required string server_class = 6;
+required string method_name = 7;
+required string call_details = 8;
+required string param = 9;
+required string user_name = 10;
 
 Review comment:
   What is recorded when there is no user authentication enabled in the cluster 
configuration? Just `hbase`? Should this field be made optional so as to save 
the space?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371457420
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 ##
 @@ -3818,6 +3828,40 @@ public ExecuteProceduresResponse 
executeProcedures(RpcController controller,
 }
   }
 
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public SlowLogResponses getSlowLogResponses(final RpcController controller,
+  final SlowLogResponseRequest request) throws ServiceException {
+final OnlineSlowLogProvider onlineSlowLogProvider =
+  this.regionServer.getOnlineSlowLogProvider();
+List slowLogPayloads;
+if (onlineSlowLogProvider != null) {
+  slowLogPayloads = onlineSlowLogProvider.getSlowLogPayloads();
+} else {
+  slowLogPayloads = new ArrayList<>();
+}
+SlowLogResponses slowLogResponses = SlowLogResponses.newBuilder()
+  .addAllSlowLogPayloads(slowLogPayloads)
+  .build();
+return slowLogResponses;
+  }
+
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public ClearSlowLogResponses clearSlowLogsResponses(final RpcController 
controller,
+  final ClearSlowLogResponseRequest request) throws ServiceException {
 
 Review comment:
   > `throws ServiceException`
   
   No it doesn't.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371431837
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -215,6 +224,17 @@
*/
   private RSRpcServices rsRpcServices;
 
+
+  /**
+   * Use to add online slowlog responses
+   */
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private final boolean isOnlineSlowLogProviderEnabled;
+
 
 Review comment:
   nit: trim all these extra newlines between conceptually related fields.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371450989
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
 ##
 @@ -0,0 +1,55 @@
+/**
+ *
+ * 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.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 {
+
+  private String regionName;
+  private String params;
+
+  public SlowLogParams(String regionName, String params) {
+this.regionName = regionName;
+this.params = params;
+  }
+
+  public String getRegionName() {
+return regionName;
+  }
+
+  public String getParams() {
+return params;
+  }
+
+  @Override
+  public String toString() {
+final StringBuilder sb = new StringBuilder("SlowLogParams{");
+sb.append("regionName='").append(regionName).append('\'');
+sb.append(", params='").append(params).append('\'');
+sb.append('}');
+return sb.toString();
+  }
+}
 
 Review comment:
   Do you expect this class to be used in a collection object? If so, it should 
include `hashCode` and `equals` implementations... which are also easily built 
with Apache Commons Lang3 Builder helpers.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371450271
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
 ##
 @@ -0,0 +1,55 @@
+/**
+ *
+ * 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.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 {
+
+  private String regionName;
+  private String params;
+
+  public SlowLogParams(String regionName, String params) {
+this.regionName = regionName;
+this.params = params;
+  }
+
+  public String getRegionName() {
+return regionName;
+  }
+
+  public String getParams() {
+return params;
+  }
+
+  @Override
+  public String toString() {
+final StringBuilder sb = new StringBuilder("SlowLogParams{");
 
 Review comment:
   Please use `ToStringBuilder` with `ToStringStyle.SHORT_PREFIX_STYLE`.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371460647
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/DisruptorExceptionHandler.java
 ##
 @@ -0,0 +1,53 @@
+/**
+ *
+ * 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.ExceptionHandler;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Exception Handler for Online Slow Log Ring Buffer
+ */
+@InterfaceAudience.Private
+class DisruptorExceptionHandler implements ExceptionHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DisruptorExceptionHandler.class);
+
+  @Override
+  public void handleEventException(Throwable ex, long sequence, 
RingBufferTruck event) {
+LOG.error("Sequence={}, event={}", sequence, event, ex);
+throw new RuntimeException(ex);
+  }
+
+  @Override
+  public void handleOnStartException(Throwable ex) {
+LOG.error("Disruptor onStartException: {}", ex.toString(), ex);
 
 Review comment:
   1. Don't log the exception message and then log the exception, that's 
redundant.
   1. Re-throwing this exception means a higher lays is going to deal with the 
failure, so this log message provides no value.
   1. Re-throwing means the RS will halt under this condition. Is this what you 
want?
   1. The same questions apply to all of these handlers.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371439669
 
 

 ##
 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
 
 Review comment:
   nit: this comment isn't helpful.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371438102
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +522,44 @@ 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_SERVICECALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (isOnlineSlowLogProviderEnabled && !tooLarge && 
this.onlineSlowLogProvider != null
+&& !call.startsWith("GetSlowLogResponses")
+&& !call.startsWith("ClearSlowLogsResponses")) {
+  // add too slow log to ringbuffer for retrieval of latest n slow logs
+
+  try {
+final SlowLogParams slowLogParams = 
ProtobufUtil.getSlowLogParams(param);
+
+final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setStartTime(startTime)
+  .setProcessingTime(processingTime)
+  .setQueueTime(qTime)
+  .setResponseSize(responseSize)
+  .setClientAddress(clientAddress)
+  .setServerClass(className)
+  .setMethodName(methodName)
+  .setCallDetails(call)
+  .setUserName(userName)
+  .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() 
: StringUtils.EMPTY)
 
 Review comment:
   What's the point of making these `required` in the protobuf definition if 
you're just going to shove empty strings in? When a value isn't available, make 
them `optional` and leave them unset.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371436457
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +522,44 @@ 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_SERVICECALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (isOnlineSlowLogProviderEnabled && !tooLarge && 
this.onlineSlowLogProvider != null
+&& !call.startsWith("GetSlowLogResponses")
+&& !call.startsWith("ClearSlowLogsResponses")) {
+  // add too slow log to ringbuffer for retrieval of latest n slow logs
+
+  try {
+final SlowLogParams slowLogParams = 
ProtobufUtil.getSlowLogParams(param);
+
+final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setStartTime(startTime)
+  .setProcessingTime(processingTime)
+  .setQueueTime(qTime)
+  .setResponseSize(responseSize)
+  .setClientAddress(clientAddress)
+  .setServerClass(className)
+  .setMethodName(methodName)
+  .setCallDetails(call)
+  .setUserName(userName)
+  .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() 
: StringUtils.EMPTY)
+  .setParam(slowLogParams != null ? slowLogParams.getParams() : 
StringUtils.EMPTY)
+  .setMultiGets(responseInfo.containsKey(MULTI_GETS)
+? (int) responseInfo.get(MULTI_GETS) : 0)
+  .setMultiMutations(responseInfo.containsKey(MULTI_MUTATIONS)
+? (int) responseInfo.get(MULTI_MUTATIONS) : 0)
+  .setMultiServiceCalls(responseInfo.containsKey(MULTI_SERVICECALLS)
+? (int) responseInfo.get(MULTI_SERVICECALLS) : 0)
+  .build();
+this.onlineSlowLogProvider.addSlowLogPayload(slowLogPayload);
+  } catch (Exception e) {
+LOG.error("Error while adding slowlog response to ringbuffer..", e);
 
 Review comment:
   Why is this logging instead of emitting a metric? When something breaks, is 
this going to log 1000's of times per second? A recording a message in the ring 
buffer is a nice-to-have, not a critical operation; when it fails, it is not a 
server ERROR scenario. Please log at debug instead of error.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371443217
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
 ##
 @@ -426,4 +437,55 @@ public void testAsyncTimeout() throws IOException {
   rpcServer.stop();
 }
   }
+
+  @Test
 
 Review comment:
   This feature isn't related to the IPC server, it's an internal detail of the 
Region Server. Thus the tests should be on the region server, not the IPC tests.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371435945
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +522,44 @@ 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_SERVICECALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (isOnlineSlowLogProviderEnabled && !tooLarge && 
this.onlineSlowLogProvider != null
+&& !call.startsWith("GetSlowLogResponses")
+&& !call.startsWith("ClearSlowLogsResponses")) {
 
 Review comment:
   Are there not static constants elsewhere that can be used in place of String 
literals?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371431530
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -215,6 +224,17 @@
*/
   private RSRpcServices rsRpcServices;
 
+
+  /**
+   * Use to add online slowlog responses
+   */
+  private OnlineSlowLogProvider onlineSlowLogProvider;
+
+  private final boolean isOnlineSlowLogProviderEnabled;
+
+  static final String ENABLED_ONLINE_LOG_PROVIDER = 
"hbase.regionserver.slowlog.enabled";
 
 Review comment:
   In the current implementation, we emit a log message regardless of whether 
this configuration is true or false. If that is to remain the case, a better 
name is `SLOW_LOG_BUFFER_ENABLED_KEY`, 
`hbase.regionserver.slowlog.buffer.enabled `


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371453349
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  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 SlowLogPayload(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 {
+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;
+private String regionName;
+

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

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371436317
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +522,44 @@ 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_SERVICECALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (isOnlineSlowLogProviderEnabled && !tooLarge && 
this.onlineSlowLogProvider != null
+&& !call.startsWith("GetSlowLogResponses")
+&& !call.startsWith("ClearSlowLogsResponses")) {
+  // add too slow log to ringbuffer for retrieval of latest n slow logs
+
+  try {
+final SlowLogParams slowLogParams = 
ProtobufUtil.getSlowLogParams(param);
+
+final SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setStartTime(startTime)
+  .setProcessingTime(processingTime)
+  .setQueueTime(qTime)
+  .setResponseSize(responseSize)
+  .setClientAddress(clientAddress)
+  .setServerClass(className)
+  .setMethodName(methodName)
+  .setCallDetails(call)
+  .setUserName(userName)
+  .setRegionName(slowLogParams != null ? slowLogParams.getRegionName() 
: StringUtils.EMPTY)
+  .setParam(slowLogParams != null ? slowLogParams.getParams() : 
StringUtils.EMPTY)
+  .setMultiGets(responseInfo.containsKey(MULTI_GETS)
+? (int) responseInfo.get(MULTI_GETS) : 0)
+  .setMultiMutations(responseInfo.containsKey(MULTI_MUTATIONS)
+? (int) responseInfo.get(MULTI_MUTATIONS) : 0)
+  .setMultiServiceCalls(responseInfo.containsKey(MULTI_SERVICECALLS)
+? (int) responseInfo.get(MULTI_SERVICECALLS) : 0)
+  .build();
+this.onlineSlowLogProvider.addSlowLogPayload(slowLogPayload);
+  } catch (Exception e) {
+LOG.error("Error while adding slowlog response to ringbuffer..", e);
 
 Review comment:
   nit: extra period in log message.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r37144
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -2205,6 +2216,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;
+  String regionName = 
getStringForByteString(scanRequest.getRegion().getValue());
+  String params = TextFormat.shortDebugString(message);
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof MutationProto) {
+  MutationProto mutationProto = (MutationProto) message;
+  String params = "type= " + mutationProto.getMutateType().toString();
+  return new SlowLogParams(StringUtils.EMPTY, params);
+} else if (message instanceof GetRequest) {
+  GetRequest getRequest = (GetRequest) message;
+  String regionName = 
getStringForByteString(getRequest.getRegion().getValue());
+  String params = "region= " + regionName + ", row= "
++ getStringForByteString(getRequest.getGet().getRow());
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof MultiRequest) {
+  MultiRequest multiRequest = (MultiRequest) message;
+  int actionsCount = multiRequest.getRegionActionList()
+.stream()
+.mapToInt(ClientProtos.RegionAction::getActionCount)
+.sum();
+  RegionAction actions = multiRequest.getRegionActionList().get(0);
+  String regionName = 
getStringForByteString(actions.getRegion().getValue());
+  String params = "region= " + regionName + ", for " + actionsCount + " 
action(s)";
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof MutateRequest) {
+  MutateRequest mutateRequest = (MutateRequest) message;
+  String regionName = 
getStringForByteString(mutateRequest.getRegion().getValue());
+  String params = "region= " + regionName;
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof CoprocessorServiceRequest) {
+  CoprocessorServiceRequest coprocessorServiceRequest = 
(CoprocessorServiceRequest) message;
+  String params = "coprocessorService= "
++ coprocessorServiceRequest.getCall().getServiceName()
++ ":" + coprocessorServiceRequest.getCall().getMethodName();
+  return new SlowLogParams(StringUtils.EMPTY, params);
+}
+String params = "TODO: " + message.getClass().toString();
 
 Review comment:
   I don't follow. Isn't this TODO message going to show up for a user? Please 
don't do 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371448759
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3890,72 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> 
getSlowLogResponses(List serverNames)
+  throws IOException {
+CompletableFuture> slowLoadPayloads = null;
+if (CollectionUtils.isNotEmpty(serverNames)) {
+  for (ServerName serverName : serverNames) {
+CompletableFuture> slowLogPayloadFromServer =
+  getSlowLogResponseFromServer(serverName);
+if (slowLoadPayloads == null) {
+  slowLoadPayloads = slowLogPayloadFromServer;
+  continue;
+}
+slowLoadPayloads = 
slowLoadPayloads.thenCombine(slowLogPayloadFromServer, (l1, l2) -> {
+  List finalSlowLogPayloads = new ArrayList<>();
+  finalSlowLogPayloads.addAll(l1);
+  finalSlowLogPayloads.addAll(l2);
+  return finalSlowLogPayloads;
+});
+  }
+}
+return slowLoadPayloads;
+  }
+
+  private CompletableFuture> getSlowLogResponseFromServer(
+  final ServerName serverName) {
+return this.>newAdminCaller()
+  .action((controller, stub) -> this
+.adminCall(
+  controller, stub, RequestConverter.buildSlowLogResponseRequest(),
+  AdminService.Interface::getSlowLogResponses,
+  ProtobufUtil::toSlowLogPayloads))
+  .serverName(serverName).call();
+  }
+
+  @Override
+  public CompletableFuture> 
clearSlowLogResponses(List serverNames)
+  throws IOException {
 
 Review comment:
   Same observation re: `IOException` in the interface.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371451406
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
 ##
 @@ -0,0 +1,55 @@
+/**
 
 Review comment:
   nit: extra `*` is unnecessary -- this is not a Javadoc 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371456316
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -2213,6 +2224,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;
+  String regionName = 
getStringForByteString(scanRequest.getRegion().getValue());
+  String params = TextFormat.shortDebugString(message);
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof MutationProto) {
+  MutationProto mutationProto = (MutationProto) message;
+  String params = "type= " + mutationProto.getMutateType().toString();
+  return new SlowLogParams(StringUtils.EMPTY, params);
+} else if (message instanceof GetRequest) {
+  GetRequest getRequest = (GetRequest) message;
+  String regionName = 
getStringForByteString(getRequest.getRegion().getValue());
+  String params = "region= " + regionName + ", row= "
++ getStringForByteString(getRequest.getGet().getRow());
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof MultiRequest) {
+  MultiRequest multiRequest = (MultiRequest) message;
+  int actionsCount = multiRequest.getRegionActionList()
+.stream()
+.mapToInt(ClientProtos.RegionAction::getActionCount)
+.sum();
+  RegionAction actions = multiRequest.getRegionActionList().get(0);
+  String regionName = 
getStringForByteString(actions.getRegion().getValue());
+  String params = "region= " + regionName + ", for " + actionsCount + " 
action(s)";
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof MutateRequest) {
+  MutateRequest mutateRequest = (MutateRequest) message;
+  String regionName = 
getStringForByteString(mutateRequest.getRegion().getValue());
+  String params = "region= " + regionName;
+  return new SlowLogParams(regionName, params);
+} else if (message instanceof CoprocessorServiceRequest) {
+  CoprocessorServiceRequest coprocessorServiceRequest = 
(CoprocessorServiceRequest) message;
+  String params = "coprocessorService= "
++ coprocessorServiceRequest.getCall().getServiceName()
++ ":" + coprocessorServiceRequest.getCall().getMethodName();
+  return new SlowLogParams(StringUtils.EMPTY, params);
 
 Review comment:
   Looks like you want a `SlowLogParams` constructor that takes a single 
argument of just `params`.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371458194
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 ##
 @@ -3818,6 +3828,40 @@ public ExecuteProceduresResponse 
executeProcedures(RpcController controller,
 }
   }
 
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public SlowLogResponses getSlowLogResponses(final RpcController controller,
+  final SlowLogResponseRequest request) throws ServiceException {
+final OnlineSlowLogProvider onlineSlowLogProvider =
+  this.regionServer.getOnlineSlowLogProvider();
+List slowLogPayloads;
+if (onlineSlowLogProvider != null) {
+  slowLogPayloads = onlineSlowLogProvider.getSlowLogPayloads();
+} else {
+  slowLogPayloads = new ArrayList<>();
 
 Review comment:
   Use `Collections.emptyList()` and avoid the allocation.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371454324
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  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 SlowLogPayload(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 {
+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;
+private String regionName;
+

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

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371440992
 
 

 ##
 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:
   This inheritance checking is gross. Can you not use constructors or some 
other feature of inheritance to manage this decision? For example, a `protected 
void postConstruction` 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371435074
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
 ##
 @@ -496,11 +522,44 @@ 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_SERVICECALLS, numServiceCalls);
 }
+final String tag = tooLarge ? "TooLarge" : "TooSlow";
 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));
+if (isOnlineSlowLogProviderEnabled && !tooLarge && 
this.onlineSlowLogProvider != null
 
 Review comment:
   The body of this if-block should be broken out into it's own 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371449311
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
 ##
 @@ -0,0 +1,55 @@
+/**
+ *
+ * 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.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 {
+
+  private String regionName;
 
 Review comment:
   Both of these should be `final`.


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371452930
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.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;
+  private transient String regionName;
 
 Review comment:
   A comment to this effect would be helpful for future readers...


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371459518
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 ##
 @@ -1251,6 +1258,9 @@ public RSRpcServices(final HRegionServer rs) throws 
IOException {
 ConnectionUtils.setServerSideHConnectionRetriesConfig(conf, name, LOG);
 rpcServer = createRpcServer(rs, rpcSchedulerFactory, bindAddress, name);
 rpcServer.setRsRpcServices(this);
+if (!(rs instanceof HMaster)) {
+  rpcServer.setOnlineSlowLogProvider(rs.getOnlineSlowLogProvider());
 
 Review comment:
   Same comment as before re: using inheritance instead of instance inspection.
   
   Wait, why must the logic be replicated here as well? Something smells...


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371448413
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3890,72 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> 
getSlowLogResponses(List serverNames)
+  throws IOException {
+CompletableFuture> slowLoadPayloads = null;
+if (CollectionUtils.isNotEmpty(serverNames)) {
+  for (ServerName serverName : serverNames) {
+CompletableFuture> slowLogPayloadFromServer =
+  getSlowLogResponseFromServer(serverName);
+if (slowLoadPayloads == null) {
+  slowLoadPayloads = slowLogPayloadFromServer;
+  continue;
+}
+slowLoadPayloads = 
slowLoadPayloads.thenCombine(slowLogPayloadFromServer, (l1, l2) -> {
+  List finalSlowLogPayloads = new ArrayList<>();
+  finalSlowLogPayloads.addAll(l1);
+  finalSlowLogPayloads.addAll(l2);
+  return finalSlowLogPayloads;
+});
+  }
+}
+return slowLoadPayloads;
+  }
+
+  private CompletableFuture> getSlowLogResponseFromServer(
+  final ServerName serverName) {
+return this.>newAdminCaller()
+  .action((controller, stub) -> this
+.adminCall(
+  controller, stub, RequestConverter.buildSlowLogResponseRequest(),
+  AdminService.Interface::getSlowLogResponses,
+  ProtobufUtil::toSlowLogPayloads))
+  .serverName(serverName).call();
+  }
+
+  @Override
+  public CompletableFuture> 
clearSlowLogResponses(List serverNames)
+  throws IOException {
+List> clearSlowLogResponseList = new 
ArrayList<>();
+if (CollectionUtils.isNotEmpty(serverNames)) {
+  for (ServerName serverName : serverNames) {
+CompletableFuture clearSlowLogResponse = 
clearSlowLogsResponses(serverName);
+clearSlowLogResponseList.add(clearSlowLogResponse);
+  }
+}
+return convertToFutureOfList(clearSlowLogResponseList);
+  }
+
+  private CompletableFuture clearSlowLogsResponses(final ServerName 
serverName) {
+return this.newAdminCaller()
+  .action(((controller, stub) -> this
+.adminCall(
+  controller, stub, 
RequestConverter.buildClearSlowLogResponseRequest(),
+  AdminService.Interface::clearSlowLogsResponses,
+  ProtobufUtil::toClearSlowLogPayload))
+  ).serverName(serverName).call();
+  }
+
+  private static  CompletableFuture> convertToFutureOfList(
+  List> futures) {
+CompletableFuture allDoneFuture =
+  CompletableFuture.allOf(futures.toArray(new 
CompletableFuture[futures.size()]));
 
 Review comment:
   `futures.toArray(new CompletableFuture[0])`


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371444991
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3890,72 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> 
getSlowLogResponses(List serverNames)
+  throws IOException {
+CompletableFuture> slowLoadPayloads = null;
+if (CollectionUtils.isNotEmpty(serverNames)) {
 
 Review comment:
   Can you invert this logic to return early? Skip the indention level...


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371458016
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 ##
 @@ -3818,6 +3828,40 @@ public ExecuteProceduresResponse 
executeProcedures(RpcController controller,
 }
   }
 
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public SlowLogResponses getSlowLogResponses(final RpcController controller,
+  final SlowLogResponseRequest request) throws ServiceException {
+final OnlineSlowLogProvider onlineSlowLogProvider =
+  this.regionServer.getOnlineSlowLogProvider();
+List slowLogPayloads;
 
 Review comment:
   This can be written instead as a simple "one"-liner.
   
   ```
   final List slowLogPayloads = onlineSlowLogProvider != 
null
 ? onlineSlowLogProvider.getSlowLogPayloads()
 : Collections.emptyList();
   ```


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371452696
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogPayload.java
 ##
 @@ -0,0 +1,318 @@
+/**
+ *
+ * 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.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 SlowLogPayload {
+
+  private static final Gson GSON = GsonUtil.createGson()
+.setPrettyPrinting()
+.registerTypeAdapter(SlowLogPayload.class, (JsonSerializer)
+  (slowLogPayload, type, jsonSerializationContext) -> {
 
 Review comment:
   Is there not a provided serializer filter that can exclude "empty" values 
such as 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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371444709
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 ##
 @@ -3889,4 +3890,72 @@ private void getProcedureResult(long procId, 
CompletableFuture future, int
 .call();
   }
 
+  @Override
+  public CompletableFuture> 
getSlowLogResponses(List serverNames)
+  throws IOException {
 
 Review comment:
   This method doesn't throw an `IOException`, yet that checked exception is 
propagated all the way up the call chain and interfaces. Can this aspect of the 
API be removed?


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] ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-01-27 Thread GitBox
ndimiduk commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r371456657
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -3368,4 +3430,58 @@ public static RegionStatesCount 
toTableRegionStatesCount(
   .build();
   }
 
+  /**
+   * Convert Protobuf class
+   * {@link 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload}
+   * To client SlowLog Payload class {@link SlowLogPayload}
+   *
+   * @param slowLogPayload SlowLog Payload protobuf instance
+   * @return SlowLog Payload for client usecase
+   */
+  private static SlowLogPayload getSlowLogPayload(
+  final TooSlowLog.SlowLogPayload slowLogPayload) {
+SlowLogPayload clientSlowLogPayload = new 
SlowLogPayload.SlowLogPayloadBuilder()
+  .setStartTime(slowLogPayload.getStartTime())
+  .setProcessingTime(slowLogPayload.getProcessingTime())
+  .setQueueTime(slowLogPayload.getQueueTime())
+  .setResponseSize(slowLogPayload.getResponseSize())
+  .setClientAddress(slowLogPayload.getClientAddress())
+  .setServerClass(slowLogPayload.getServerClass())
+  .setMethodName(slowLogPayload.getMethodName())
+  .setCallDetails(slowLogPayload.getCallDetails())
+  .setParam(slowLogPayload.getParam())
+  .setRegionName(slowLogPayload.getRegionName())
+  .setUserName(slowLogPayload.getUserName())
+  .setMultiGetsCount(slowLogPayload.getMultiGets())
+  .setMultiMutationsCount(slowLogPayload.getMultiMutations())
+  .setMultiServiceCalls(slowLogPayload.getMultiServiceCalls())
+  .build();
+return clientSlowLogPayload;
+  }
+
+  /**
+   * Convert  AdminProtos#SlowLogResponses to list of {@link SlowLogPayload}
+   *
+   * @param slowLogResponses slowlog response protobuf instance
+   * @return list of SlowLog payloads for client usecase
+   */
+  public static List toSlowLogPayloads(
+  final AdminProtos.SlowLogResponses slowLogResponses) {
+List slowLogPayloads = new ArrayList<>();
+for (TooSlowLog.SlowLogPayload slowLogPayload : 
slowLogResponses.getSlowLogPayloadsList()) {
 
 Review comment:
   This is an ideal use case for a `Stream.collect`.


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