You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/05/16 18:11:31 UTC

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

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<Put> 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<TooSlowLog.SlowLogPayload> slowLogPayloads,
+      final Configuration configuration) {
+    List<Put> 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.getStartTime())))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
+          Bytes.toBytes(slowLogPayload.getType().name()))
+        .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
+          Bytes.toBytes(slowLogPayload.getUserName()));
+      puts.add(put);
+    }
+    try {
+      if (connection == null) {
+        synchronized (SlowLogTableAccessor.class) {
+          if (connection == null) {
+            Configuration conf = new Configuration(configuration);
+            // rpc timeout: 20s
+            conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 20000);
+            // retry count: 5
+            conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
+            conf.setInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 1);
+            connection = ConnectionFactory.createConnection(conf);
+          }
+        }
+      }
+      doPut(connection, puts);

Review comment:
       In a 10 min window we will do the puts . Can it really hotspot the region? Also at the Rpc handler layer should we give the lowest priority? System tables like META had high priority i belive - need to check the code. Should we lower for this table?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogRecorder.java
##########
@@ -86,7 +87,9 @@ public SlowLogRecorder(Configuration conf) {
     this.disruptor.setDefaultExceptionHandler(new DisruptorExceptionHandler());
 
     // initialize ringbuffer event handler
-    this.logEventHandler = new LogEventHandler(this.eventCount);
+    final boolean isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,

Review comment:
       Even if not enabled we will create that handler?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogTableAccessor.java
##########
@@ -0,0 +1,120 @@
+/*
+ *
+ * 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.Random;
+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.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 void doPut(final Connection connection, final Put put)
+      throws IOException {
+    try (Table table = connection.getTable(TableName.SLOW_LOG_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @param connection Connection to put data
+   */
+  public static void addSlowLogRecord(final TooSlowLog.SlowLogPayload slowLogPayload,
+      final Connection connection) {
+    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(slowLogPayload.getProcessingTime()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+        Bytes.toBytes(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(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(slowLogPayload.getStartTime()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
+        Bytes.toBytes(slowLogPayload.getType().name()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
+        Bytes.toBytes(slowLogPayload.getUserName()));
+    try {
+      doPut(connection, put);
+    } catch (IOException e) {
+      LOG.error("Failed to add slow/large log record to hbase:slowlog table for region: {}",
+        slowLogPayload.getRegionName(), e);
+    }
+  }
+
+  /**
+   * Create rowKey: currentTimeMillis APPEND slowLogPayload.hashcode
+   * Scan on slowlog table should keep records with sorted order of time, however records
+   * added at the very same time (currentTimeMillis) could be in random order.
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @return rowKey byte[]
+   */
+  private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) {

Review comment:
       In a 10 min window we run the chore - but if there are slow sync happening during that 10 min window - we accumulate it in memory ?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogTableAccessor.java
##########
@@ -0,0 +1,120 @@
+/*
+ *
+ * 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.Random;
+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.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 void doPut(final Connection connection, final Put put)
+      throws IOException {
+    try (Table table = connection.getTable(TableName.SLOW_LOG_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @param connection Connection to put data
+   */
+  public static void addSlowLogRecord(final TooSlowLog.SlowLogPayload slowLogPayload,
+      final Connection connection) {
+    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(slowLogPayload.getProcessingTime()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+        Bytes.toBytes(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(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(slowLogPayload.getStartTime()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
+        Bytes.toBytes(slowLogPayload.getType().name()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
+        Bytes.toBytes(slowLogPayload.getUserName()));
+    try {
+      doPut(connection, put);
+    } catch (IOException e) {
+      LOG.error("Failed to add slow/large log record to hbase:slowlog table for region: {}",
+        slowLogPayload.getRegionName(), e);
+    }
+  }
+
+  /**
+   * Create rowKey: currentTimeMillis APPEND slowLogPayload.hashcode
+   * Scan on slowlog table should keep records with sorted order of time, however records
+   * added at the very same time (currentTimeMillis) could be in random order.
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @return rowKey byte[]
+   */
+  private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) {

Review comment:
       Just one Q - this system table if not coming online due to some assignment issues -  we don't bother right? Say if the table is not online and while doing the puts we don't get the table - we should not do a WARN for certain time and then stop the connection fetching operatoin itself. Something like file based bucket cache - if there is afailure in writing to cache we keep retrying and if the error count exceeds a threshold we will WARN and no longer cache the data. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/slowlog/SlowLogMasterService.java
##########
@@ -0,0 +1,72 @@
+/*
+ *
+ * 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.master.slowlog;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Slowlog Master services - Table creation to be used by HMaster
+ */
+@InterfaceAudience.Private
+public class SlowLogMasterService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SlowLogMasterService.class);
+
+  private final boolean slowlogTableEnabled;
+  private final MasterServices masterServices;
+
+  private static final TableDescriptorBuilder TABLE_DESCRIPTOR_BUILDER =
+    TableDescriptorBuilder.newBuilder(TableName.SLOW_LOG_TABLE_NAME).setRegionReplication(1)
+      .setColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(HConstants.SLOWLOG_INFO_FAMILY)
+          .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+          .setBlockCacheEnabled(false)

Review comment:
       Better to disable all types of cache for this. Good. 

##########
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<Put> 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<TooSlowLog.SlowLogPayload> slowLogPayloads,
+      final Configuration configuration) {
+    List<Put> 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)

Review comment:
       Good. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogTableAccessor.java
##########
@@ -0,0 +1,120 @@
+/*
+ *
+ * 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.Random;
+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.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 void doPut(final Connection connection, final Put put)
+      throws IOException {
+    try (Table table = connection.getTable(TableName.SLOW_LOG_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
+  /**
+   * Add slow/large log records to hbase:slowlog table
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @param connection Connection to put data
+   */
+  public static void addSlowLogRecord(final TooSlowLog.SlowLogPayload slowLogPayload,
+      final Connection connection) {
+    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(slowLogPayload.getProcessingTime()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("queue_time"),
+        Bytes.toBytes(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(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(slowLogPayload.getStartTime()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("type"),
+        Bytes.toBytes(slowLogPayload.getType().name()))
+      .addColumn(HConstants.SLOWLOG_INFO_FAMILY, Bytes.toBytes("username"),
+        Bytes.toBytes(slowLogPayload.getUserName()));
+    try {
+      doPut(connection, put);
+    } catch (IOException e) {
+      LOG.error("Failed to add slow/large log record to hbase:slowlog table for region: {}",
+        slowLogPayload.getRegionName(), e);
+    }
+  }
+
+  /**
+   * Create rowKey: currentTimeMillis APPEND slowLogPayload.hashcode
+   * Scan on slowlog table should keep records with sorted order of time, however records
+   * added at the very same time (currentTimeMillis) could be in random order.
+   *
+   * @param slowLogPayload SlowLogPayload to process
+   * @return rowKey byte[]
+   */
+  private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) {

Review comment:
       Ok I saw this public static final int DEFAULT_SLOW_LOG_RING_BUFFER_SIZE = 256;
   So at max it is only 256 elements that we store in that queue. Rest will be removed anyway. At max we have 256 put list at any point of time. 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:
users@infra.apache.org