[GitHub] [hbase] Reidddddd commented on a change in pull request #2312: HBASE-23935 : Backport HBASE-22978(with sub-tasks) to branch-1

2020-10-15 Thread GitBox


Reidd commented on a change in pull request #2312:
URL: https://github.com/apache/hbase/pull/2312#discussion_r506056833



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,154 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.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.classification.InterfaceAudience;
+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.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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 final class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  private SlowLogTableAccessor() {
+  }
+
+  /**
+   * 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)) {

Review comment:
   This can be done afterwards.
   So here I suggest, we should create something like TTL, if the connections 
hasn't been touched for a time, there then we close it, otherwise we should 
keep 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




[GitHub] [hbase] Reidddddd commented on a change in pull request #2312: HBASE-23935 : Backport HBASE-22978(with sub-tasks) to branch-1

2020-10-15 Thread GitBox


Reidd commented on a change in pull request #2312:
URL: https://github.com/apache/hbase/pull/2312#discussion_r506057237



##
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
##
@@ -1401,6 +1401,22 @@
*/
   public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
+  public static final int DEFAULT_SLOW_LOG_RING_BUFFER_SIZE = 256;

Review comment:
   why do we still introduce these parameters in HConstants. I thought we 
have avoided to add parameters in it for quite 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




[GitHub] [hbase] Reidddddd commented on a change in pull request #2312: HBASE-23935 : Backport HBASE-22978(with sub-tasks) to branch-1

2020-10-15 Thread GitBox


Reidd commented on a change in pull request #2312:
URL: https://github.com/apache/hbase/pull/2312#discussion_r506056833



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,154 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.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.classification.InterfaceAudience;
+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.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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 final class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  private SlowLogTableAccessor() {
+  }
+
+  /**
+   * 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)) {

Review comment:
   This can be done afterwards.
   There's one case, when a server gets stuck, every call is a slow response. 
Then your connection will be created-closed, created-closed many times. 
   So here I suggest, we should create something like TTL, if the connections 
hasn't been touched for a time, there then we close it, otherwise we should 
keep 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




[GitHub] [hbase] Reidddddd commented on a change in pull request #2312: HBASE-23935 : Backport HBASE-22978(with sub-tasks) to branch-1

2020-10-15 Thread GitBox


Reidd commented on a change in pull request #2312:
URL: https://github.com/apache/hbase/pull/2312#discussion_r506055382



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java
##
@@ -0,0 +1,154 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.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.classification.InterfaceAudience;
+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.protobuf.generated.TooSlowLog;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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 final class SlowLogTableAccessor {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogTableAccessor.class);
+
+  private static final Random RANDOM = new Random();
+
+  private static Connection connection;
+
+  private SlowLogTableAccessor() {
+  }
+
+  /**
+   * 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);
+  put.setDurability(Durability.SKIP_WAL);
+  put.setPriority(HConstants.NORMAL_QOS);
+  put.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()))
+