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



##########
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:
       Are we not logging the region's encoded name?  This is the region name 
which comes as part of the req param:

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
##########
@@ -83,6 +83,12 @@
   public static final TableName META_TABLE_NAME =
       valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");
 
+  /** hbase:slowlog table name - can be enabled
+   * with config - hbase.regionserver.slowlog.systable.enabled
+   */
+  public static final TableName SLOW_LOG_TABLE_NAME =

Review comment:
       Are we exposing this table name to customer? This is a public class.  If 
we expose the name for user to create queries, we might have to expose the 
column names also?  But those can not be done in this class. Any thinking of 
having a new Public class if needed to expose? If you dont want to expose this 
table name, we should not keep this in TableName public class

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##########
@@ -53,12 +55,28 @@
 class LogEventHandler implements EventHandler<RingBufferEnvelope> {
 
   private static final Logger LOG = 
LoggerFactory.getLogger(LogEventHandler.class);
+  private static final int SYS_TABLE_QUEUE_SIZE = 1000;
 
-  private final Queue<SlowLogPayload> queue;
+  private final Queue<SlowLogPayload> queueForRingBuffer;
+  private final Queue<SlowLogPayload> queueForSysTable;
+  private final boolean isSlowLogTableEnabled;
 
-  LogEventHandler(int eventCount) {
+  private Configuration configuration;
+
+  private static final ReentrantLock LOCK = new ReentrantLock();
+
+  LogEventHandler(int eventCount, boolean isSlowLogTableEnabled, Configuration 
conf) {
+    this.configuration = conf;
     EvictingQueue<SlowLogPayload> evictingQueue = 
EvictingQueue.create(eventCount);
-    queue = Queues.synchronizedQueue(evictingQueue);
+    queueForRingBuffer = Queues.synchronizedQueue(evictingQueue);
+    this.isSlowLogTableEnabled = isSlowLogTableEnabled;
+    if (isSlowLogTableEnabled) {
+      EvictingQueue<SlowLogPayload> evictingQueueForTable = 
EvictingQueue.create(

Review comment:
       When system table logging is enabled, we have 2 queue.  This new one 
will always have 1000 max size where as the 'queueForRingBuffer' will have a 
default of 256 only.  Anyways we will occupy more heap memory upto 1000 
messages. Why we should reduce the max numbers in queueForRingBuffer?
   

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##########
@@ -53,12 +55,28 @@
 class LogEventHandler implements EventHandler<RingBufferEnvelope> {
 
   private static final Logger LOG = 
LoggerFactory.getLogger(LogEventHandler.class);
+  private static final int SYS_TABLE_QUEUE_SIZE = 1000;
 
-  private final Queue<SlowLogPayload> queue;
+  private final Queue<SlowLogPayload> queueForRingBuffer;
+  private final Queue<SlowLogPayload> queueForSysTable;
+  private final boolean isSlowLogTableEnabled;
 
-  LogEventHandler(int eventCount) {
+  private Configuration configuration;
+
+  private static final ReentrantLock LOCK = new ReentrantLock();
+
+  LogEventHandler(int eventCount, boolean isSlowLogTableEnabled, Configuration 
conf) {
+    this.configuration = conf;
     EvictingQueue<SlowLogPayload> evictingQueue = 
EvictingQueue.create(eventCount);
-    queue = Queues.synchronizedQueue(evictingQueue);
+    queueForRingBuffer = Queues.synchronizedQueue(evictingQueue);
+    this.isSlowLogTableEnabled = isSlowLogTableEnabled;
+    if (isSlowLogTableEnabled) {
+      EvictingQueue<SlowLogPayload> evictingQueueForTable = 
EvictingQueue.create(

Review comment:
       Can we have a way to avoid this 2 queues but keep it single?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##########
@@ -53,12 +55,28 @@
 class LogEventHandler implements EventHandler<RingBufferEnvelope> {
 
   private static final Logger LOG = 
LoggerFactory.getLogger(LogEventHandler.class);
+  private static final int SYS_TABLE_QUEUE_SIZE = 1000;

Review comment:
       You want a config for this like 
'hbase.regionserver.slowlog.ringbuffer.size'?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogTableAccessor.java
##########
@@ -0,0 +1,124 @@
+/*
+ *
+ * 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.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 List<Put> puts)
+      throws IOException {
+    try (Table table = connection.getTable(TableName.SLOW_LOG_TABLE_NAME)) {
+      table.put(puts);

Review comment:
       So I can see that u have reduced the RPC time out and retries. Fine.

##########
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:
       The above is a valid Q and nice observation.  The priority of the req 
been calculated based on the System ns. As this table is in 'hbase' NS it will 
get highest priority.  Its not about WAL write or cache usage.  The RPC will be 
handled by the priority handler threads and Q.    I believe we can set it at 
the individual req level.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##########
@@ -160,7 +183,7 @@ boolean clearSlowLogs() {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Received request to clean up online slowlog buffer..");
     }
-    queue.clear();
+    queueForRingBuffer.clear();

Review comment:
       This will be done when Admin issues clearSlowLogsResponses req.  Still 
it wont reduce the RS heap usage as we have 2 queues. Is that what we intent?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/LogEventHandler.java
##########
@@ -129,7 +147,12 @@ public void onEvent(RingBufferEnvelope event, long 
sequence, boolean endOfBatch)
       .setType(type)
       .setUserName(userName)
       .build();
-    queue.add(slowLogPayload);
+    queueForRingBuffer.add(slowLogPayload);

Review comment:
       Even while adding here also u can have below check right?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogTableAccessor.java
##########
@@ -0,0 +1,124 @@
+/*
+ *
+ * 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.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 List<Put> puts)
+      throws IOException {
+    try (Table table = connection.getTable(TableName.SLOW_LOG_TABLE_NAME)) {
+      table.put(puts);

Review comment:
       Ya seems for Server -> Server RPCs  we will have default of 15 * 3 = 45 
retries.
   I can see the client side retries is 15 now not 35. I believe this is 
changed in master branch. Not sure which 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


Reply via email to