[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427475754



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##
@@ -256,4 +283,36 @@ private boolean 
isFilterProvided(AdminProtos.SlowLogResponseRequest request) {
 return filteredSlowLogPayloads;
   }
 
+  /**
+   * Poll from queueForSysTable and insert 100 records in hbase:slowlog table 
in single batch
+   */
+  void addAllLogsToSysTable() {
+if (queueForSysTable == null) {
+  // hbase.regionserver.slowlog.systable.enabled is turned off. Exiting.
+  return;
+}
+if (LOCK.isLocked()) {
+  return;
+}
+LOCK.lock();
+try {
+  List slowLogPayloads = new ArrayList<>();
+  int i = 0;
+  while (!queueForSysTable.isEmpty()) {
+slowLogPayloads.add(queueForSysTable.poll());
+i++;
+if (i == 100) {

Review comment:
   Better to move this to constant.





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




[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427475754



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##
@@ -256,4 +283,36 @@ private boolean 
isFilterProvided(AdminProtos.SlowLogResponseRequest request) {
 return filteredSlowLogPayloads;
   }
 
+  /**
+   * Poll from queueForSysTable and insert 100 records in hbase:slowlog table 
in single batch
+   */
+  void addAllLogsToSysTable() {
+if (queueForSysTable == null) {
+  // hbase.regionserver.slowlog.systable.enabled is turned off. Exiting.
+  return;
+}
+if (LOCK.isLocked()) {
+  return;
+}
+LOCK.lock();
+try {
+  List slowLogPayloads = new ArrayList<>();
+  int i = 0;
+  while (!queueForSysTable.isEmpty()) {
+slowLogPayloads.add(queueForSysTable.poll());
+i++;
+if (i == 100) {

Review comment:
   Better to move this to constant. nit





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




[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427474882



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,149 @@
+/*
+ *
+ * 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.slowlog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Slowlog Accessor to record slow/large RPC log identified at each 
RegionServer RpcServer level.
+ * This can be done only optionally to record the entire history of slow/large 
rpc calls
+ * since RingBuffer can handle only limited latest records.
+ */
+@InterfaceAudience.Private
+public class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  /**
+   * hbase:slowlog table name - can be enabled
+   * with config - hbase.regionserver.slowlog.systable.enabled
+   */
+  public static final TableName SLOW_LOG_TABLE_NAME =
+TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, 
"slowlog");
+
+  private static void doPut(final Connection connection, final List puts)
+  throws IOException {
+try (Table table = connection.getTable(SLOW_LOG_TABLE_NAME)) {
+  table.put(puts);
+}
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   * @param slowLogPayloads List of SlowLogPayload to process
+   * @param configuration Configuration to use for connection
+   */
+  public static void addSlowLogRecords(final List 
slowLogPayloads,
+  final Configuration configuration) {
+List puts = new ArrayList<>(slowLogPayloads.size());
+for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
+  final byte[] rowKey = getRowKey(slowLogPayload);
+  final Put put = new Put(rowKey).setDurability(Durability.SKIP_WAL)
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("call_details"),
+  Bytes.toBytes(slowLogPayload.getCallDetails()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("client_address"),
+  Bytes.toBytes(slowLogPayload.getClientAddress()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("method_name"),
+  Bytes.toBytes(slowLogPayload.getMethodName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
+  Bytes.toBytes(slowLogPayload.getParam()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("processing_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("region_name"),
+  Bytes.toBytes(slowLogPayload.getRegionName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("response_size"),
+  Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("server_class"),
+  Bytes.toBytes(slowLogPayload.getServerClass()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
+  Bytes.toBytes(Long.toString(slowLogPayload

[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427474466



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,149 @@
+/*
+ *
+ * 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.slowlog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Slowlog Accessor to record slow/large RPC log identified at each 
RegionServer RpcServer level.
+ * This can be done only optionally to record the entire history of slow/large 
rpc calls
+ * since RingBuffer can handle only limited latest records.
+ */
+@InterfaceAudience.Private
+public class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  /**
+   * hbase:slowlog table name - can be enabled
+   * with config - hbase.regionserver.slowlog.systable.enabled
+   */
+  public static final TableName SLOW_LOG_TABLE_NAME =
+TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, 
"slowlog");
+
+  private static void doPut(final Connection connection, final List puts)
+  throws IOException {
+try (Table table = connection.getTable(SLOW_LOG_TABLE_NAME)) {
+  table.put(puts);
+}
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   * @param slowLogPayloads List of SlowLogPayload to process
+   * @param configuration Configuration to use for connection
+   */
+  public static void addSlowLogRecords(final List 
slowLogPayloads,
+  final Configuration configuration) {
+List puts = new ArrayList<>(slowLogPayloads.size());
+for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
+  final byte[] rowKey = getRowKey(slowLogPayload);
+  final Put put = new Put(rowKey).setDurability(Durability.SKIP_WAL)
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("call_details"),
+  Bytes.toBytes(slowLogPayload.getCallDetails()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("client_address"),
+  Bytes.toBytes(slowLogPayload.getClientAddress()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("method_name"),
+  Bytes.toBytes(slowLogPayload.getMethodName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
+  Bytes.toBytes(slowLogPayload.getParam()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("processing_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("region_name"),
+  Bytes.toBytes(slowLogPayload.getRegionName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("response_size"),
+  Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("server_class"),
+  Bytes.toBytes(slowLogPayload.getServerClass()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
+  Bytes.toBytes(Long.toString(slowLogPayload

[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427474544



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,149 @@
+/*
+ *
+ * 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.slowlog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Slowlog Accessor to record slow/large RPC log identified at each 
RegionServer RpcServer level.
+ * This can be done only optionally to record the entire history of slow/large 
rpc calls
+ * since RingBuffer can handle only limited latest records.
+ */
+@InterfaceAudience.Private
+public class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  /**
+   * hbase:slowlog table name - can be enabled
+   * with config - hbase.regionserver.slowlog.systable.enabled
+   */
+  public static final TableName SLOW_LOG_TABLE_NAME =
+TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, 
"slowlog");
+
+  private static void doPut(final Connection connection, final List puts)
+  throws IOException {
+try (Table table = connection.getTable(SLOW_LOG_TABLE_NAME)) {
+  table.put(puts);
+}
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   * @param slowLogPayloads List of SlowLogPayload to process
+   * @param configuration Configuration to use for connection
+   */
+  public static void addSlowLogRecords(final List 
slowLogPayloads,
+  final Configuration configuration) {
+List puts = new ArrayList<>(slowLogPayloads.size());
+for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
+  final byte[] rowKey = getRowKey(slowLogPayload);
+  final Put put = new Put(rowKey).setDurability(Durability.SKIP_WAL)
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("call_details"),
+  Bytes.toBytes(slowLogPayload.getCallDetails()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("client_address"),
+  Bytes.toBytes(slowLogPayload.getClientAddress()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("method_name"),
+  Bytes.toBytes(slowLogPayload.getMethodName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
+  Bytes.toBytes(slowLogPayload.getParam()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("processing_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("region_name"),
+  Bytes.toBytes(slowLogPayload.getRegionName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("response_size"),
+  Bytes.toBytes(Long.toString(slowLogPayload.getResponseSize(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("server_class"),
+  Bytes.toBytes(slowLogPayload.getServerClass()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("start_time"),
+  Bytes.toBytes(Long.toString(slowLogPayload

[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427473270



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java
##
@@ -101,8 +102,12 @@ public void setPriority(int priority) {
 
   @Override
   public void setPriority(final TableName tn) {
-setPriority(
-  tn != null && tn.isSystemTable() ? HConstants.SYSTEMTABLE_QOS : 
HConstants.NORMAL_QOS);
+int priority = HConstants.NORMAL_QOS;
+if (tn != null && tn.isSystemTable()
+&& !tn.equals(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {

Review comment:
   May be move this to a util method some where. Like isMeta that we have. 
We can say as isLogTable()? 





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




[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-19 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r427472369



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
##
@@ -532,7 +533,7 @@ static int calcPriority(int priority, TableName tableName) {
   }
 
   static int getPriority(TableName tableName) {
-if (tableName.isSystemTable()) {
+if (tableName.isSystemTable() && 
!tableName.equals(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {

Review comment:
   Good. This is where we set the priority. 





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




[GitHub] [hbase] ramkrish86 commented on a change in pull request #1681: HBASE-23938 : System table hbase:slowlog to store complete slow/large…

2020-05-16 Thread GitBox


ramkrish86 commented on a change in pull request #1681:
URL: https://github.com/apache/hbase/pull/1681#discussion_r426174968



##
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
##
@@ -1532,6 +1532,16 @@
 "hbase.regionserver.slowlog.buffer.enabled";
   public static final boolean DEFAULT_ONLINE_LOG_PROVIDER_ENABLED = false;
 
+  /** The slowlog info family as a string*/
+  private static final String SLOWLOG_INFO_FAMILY_STR = "info";
+
+  /** The slowlog info family */
+  public static final byte [] SLOWLOG_INFO_FAMILY = 
Bytes.toBytes(SLOWLOG_INFO_FAMILY_STR);
+
+  public static final String SLOW_LOG_SYS_TABLE_ENABLED_KEY =
+"hbase.regionserver.slowlog.systable.enabled";
+  public static final boolean DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY = false;

Review comment:
   Good. Generally if we have a log system which will store the logs then 
this may be redundant in such systems. 

##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,140 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Slowlog Accessor to record slow/large RPC log identified at each 
RegionServer RpcServer level.
+ * This can be done only optionally to record the entire history of slow/large 
rpc calls
+ * since RingBuffer can handle only limited latest records.
+ */
+@InterfaceAudience.Private
+public class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  private static void doPut(final Connection connection, final List puts)
+  throws IOException {
+try (Table table = connection.getTable(TableName.SLOW_LOG_TABLE_NAME)) {
+  table.put(puts);
+}
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   * @param slowLogPayloads List of SlowLogPayload to process
+   * @param configuration Configuration to use for connection
+   */
+  public static void addSlowLogRecords(final List 
slowLogPayloads,
+  final Configuration configuration) {
+List puts = new ArrayList<>(slowLogPayloads.size());
+for (TooSlowLog.SlowLogPayload slowLogPayload : slowLogPayloads) {
+  final byte[] rowKey = getRowKey(slowLogPayload);
+  final Put put = new Put(rowKey).setDurability(Durability.SKIP_WAL)
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("call_details"),
+  Bytes.toBytes(slowLogPayload.getCallDetails()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("client_address"),
+  Bytes.toBytes(slowLogPayload.getClientAddress()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("method_name"),
+  Bytes.toBytes(slowLogPayload.getMethodName()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("param"),
+  Bytes.toBytes(slowLogPayload.getParam()))
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, 
Bytes.toBytes("processing_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getProcessingTime(
+.addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+  Bytes.toBytes(Integer.toString(slowLogPayload.getQueueTime(
+.addColumn(HConstants.SLOWLO