Skip to content

Commit

Permalink
HBASE-23938 : System table hbase:slowlog to store complete slow/large… (
Browse files Browse the repository at this point in the history
#1681)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Anoop Sam John <anoopsamjohn@apache.org>
Signed-off-by: ramkrish86 <ramkrishna@apache.org>
  • Loading branch information
virajjasani committed May 20, 2020
1 parent 5fb9e51 commit 86a2692
Show file tree
Hide file tree
Showing 12 changed files with 619 additions and 21 deletions.
Original file line number Diff line number Diff line change
@@ -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.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<Put> 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<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)
.setPriority(HConstants.NORMAL_QOS)
.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) {
createConnection(configuration);
}
doPut(connection, puts);
} catch (Exception e) {
LOG.warn("Failed to add slow/large log records to hbase:slowlog table.", e);
}
}

private static synchronized void createConnection(Configuration configuration)
throws IOException {
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);
}

/**
* 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) {
String hashcode = String.valueOf(slowLogPayload.hashCode());
String lastFiveDig =
hashcode.substring((hashcode.length() > 5) ? (hashcode.length() - 5) : 0);
if (lastFiveDig.startsWith("-")) {
lastFiveDig = String.valueOf(RANDOM.nextInt(99999));
}
final long currentTimeMillis = EnvironmentEdgeManager.currentTime();
final String timeAndHashcode = currentTimeMillis + lastFiveDig;
final long rowKeyLong = Long.parseLong(timeAndHashcode);
return Bytes.toBytes(rowKeyLong);
}

}
10 changes: 10 additions & 0 deletions hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
Original file line number Diff line number Diff line change
Expand Up @@ -1536,6 +1536,16 @@ public enum OperationStatusCode {
"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;

private HConstants() {
// Can't be instantiated with this ctor.
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -437,7 +437,7 @@ public Pair<Message, CellScanner> call(RpcCall call,
final String className = server == null ? StringUtils.EMPTY :
server.getClass().getSimpleName();
this.slowLogRecorder.addSlowLogPayload(
new RpcLogDetails(call, status.getClient(), responseSize, className, tooSlow,
new RpcLogDetails(call, param, status.getClient(), responseSize, className, tooSlow,
tooLarge));
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,7 @@
import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager;
import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
import org.apache.hadoop.hbase.master.slowlog.SlowLogMasterService;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
Expand Down Expand Up @@ -1170,6 +1171,8 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// Start the chore to read snapshots and add their usage to table/NS quotas
getChoreService().scheduleChore(snapshotQuotaChore);
}
final SlowLogMasterService slowLogMasterService = new SlowLogMasterService(conf, this);
slowLogMasterService.init();

// clear the dead servers with same host name and port of online server because we are not
// removing dead server with same hostname and port of rs which is trying to check in before
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
*
* 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.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
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(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)
.setRegionReplication(1)
.setColumnFamily(
ColumnFamilyDescriptorBuilder.newBuilder(HConstants.SLOWLOG_INFO_FAMILY)
.setScope(HConstants.REPLICATION_SCOPE_LOCAL)
.setBlockCacheEnabled(false)
.setMaxVersions(1).build());

public SlowLogMasterService(final Configuration configuration,
final MasterServices masterServices) {
slowlogTableEnabled = configuration.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
this.masterServices = masterServices;
}

public void init() throws IOException {
if (!slowlogTableEnabled) {
LOG.info("Slow/Large requests logging to system table hbase:slowlog is disabled. Quitting.");
return;
}
if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {
LOG.info("slowlog table not found. Creating.");
this.masterServices.createSystemTable(TABLE_DESCRIPTOR_BUILDER.build());
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,7 @@
import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
import org.apache.hadoop.hbase.regionserver.slowlog.SlowLogRecorder;
import org.apache.hadoop.hbase.regionserver.slowlog.SlowLogTableOpsChore;
import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
Expand Down Expand Up @@ -427,6 +428,8 @@ public class HRegionServer extends HasThread implements

private final RegionServerAccounting regionServerAccounting;

private SlowLogTableOpsChore slowLogTableOpsChore = null;

// Block cache
private BlockCache blockCache;
// The cache for mob files
Expand Down Expand Up @@ -2033,6 +2036,9 @@ private void startServices() throws IOException {
if (this.fsUtilizationChore != null) {
choreService.scheduleChore(fsUtilizationChore);
}
if (this.slowLogTableOpsChore != null) {
choreService.scheduleChore(slowLogTableOpsChore);
}

// Leases is not a Thread. Internally it runs a daemon thread. If it gets
// an unhandled exception, it will just exit.
Expand Down Expand Up @@ -2077,6 +2083,14 @@ private void initializeThreads() {
this.periodicFlusher = new PeriodicMemStoreFlusher(this.flushCheckFrequency, this);
this.leaseManager = new LeaseManager(this.threadWakeFrequency);

final boolean isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
if (isSlowLogTableEnabled) {
// default chore duration: 10 min
final int duration = conf.getInt("hbase.slowlog.systable.chore.duration", 10 * 60 * 1000);
slowLogTableOpsChore = new SlowLogTableOpsChore(this, duration, this.slowLogRecorder);
}

// Create the thread to clean the moved regions list
movedRegionsCleaner = MovedRegionsCleaner.create(this);

Expand Down Expand Up @@ -2569,6 +2583,7 @@ protected void stopServiceThreads() {
choreService.cancelChore(storefileRefresher);
choreService.cancelChore(movedRegionsCleaner);
choreService.cancelChore(fsUtilizationChore);
choreService.cancelChore(slowLogTableOpsChore);
// clean up the remaining scheduled chores (in case we missed out any)
choreService.shutdown();
}
Expand Down
Loading

0 comments on commit 86a2692

Please sign in to comment.