HBASE-17627 Active workers metric for thrift (Ashu Pachauri)

Signed-off-by: Gary Helmling <garyh@apache.org>
This commit is contained in:
Ashu Pachauri 2017-02-13 11:27:51 -08:00 committed by Gary Helmling
parent de23d306eb
commit d2c083d21c
7 changed files with 110 additions and 8 deletions

View File

@ -32,6 +32,7 @@ public interface MetricsThriftServerSource extends ExceptionTrackingSource, JvmP
String THRIFT_CALL_KEY = "thriftCall";
String SLOW_THRIFT_CALL_KEY = "slowThriftCall";
String CALL_QUEUE_LEN_KEY = "callQueueLen";
String ACTIVE_WORKER_COUNT_KEY = "numActiveWorkers";
/**
* Add how long an operation was in the queue.
@ -75,4 +76,14 @@ public interface MetricsThriftServerSource extends ExceptionTrackingSource, JvmP
* @param time Time
*/
void incSlowCall(long time);
/**
* Increment number of active thrift workers.
*/
void incActiveWorkerCount();
/**
* Decrement number of active thrift workers.
*/
void decActiveWorkerCount();
}

View File

@ -44,6 +44,8 @@ public class MetricsThriftServerSourceImpl extends ExceptionTrackingSourceImpl i
private MutableGaugeLong callQueueLenGauge;
private MutableGaugeLong activeWorkerCountGauge;
// pause monitor metrics
private final MutableFastCounter infoPauseThresholdExceeded;
private final MutableFastCounter warnPauseThresholdExceeded;
@ -74,6 +76,7 @@ public class MetricsThriftServerSourceImpl extends ExceptionTrackingSourceImpl i
thriftCallStat = getMetricsRegistry().newTimeHistogram(THRIFT_CALL_KEY);
thriftSlowCallStat = getMetricsRegistry().newTimeHistogram(SLOW_THRIFT_CALL_KEY);
callQueueLenGauge = getMetricsRegistry().getGauge(CALL_QUEUE_LEN_KEY, 0);
activeWorkerCountGauge = getMetricsRegistry().getGauge(ACTIVE_WORKER_COUNT_KEY, 0);
}
@Override
@ -112,6 +115,16 @@ public class MetricsThriftServerSourceImpl extends ExceptionTrackingSourceImpl i
thriftSlowCallStat.add(time);
}
@Override
public void incActiveWorkerCount() {
activeWorkerCountGauge.incr();
}
@Override
public void decActiveWorkerCount() {
activeWorkerCountGauge.decr();
}
@Override
public void incInfoThresholdExceeded(int count) {
infoPauseThresholdExceeded.incr(count);

View File

@ -156,9 +156,9 @@ public class TBoundedThreadPoolServer extends TServer {
tfb.setDaemon(true);
tfb.setNameFormat("thrift-worker-%d");
executorService =
new ThreadPoolExecutor(minWorkerThreads,
new THBaseThreadPoolExecutor(minWorkerThreads,
maxWorkerThreads, options.threadKeepAliveTimeSec,
TimeUnit.SECONDS, this.callQueue, tfb.build());
TimeUnit.SECONDS, this.callQueue, tfb.build(), metrics);
executorService.allowCoreThreadTimeOut(true);
serverOptions = options;
}

View File

@ -0,0 +1,61 @@
/**
* 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.thrift;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
/**
* A ThreadPoolExecutor customized for working with HBase thrift to update metrics before and
* after the execution of a task.
*/
public class THBaseThreadPoolExecutor extends ThreadPoolExecutor {
private ThriftMetrics metrics;
public THBaseThreadPoolExecutor(int corePoolSize, int maxPoolSize, long keepAliveTime,
TimeUnit unit, BlockingQueue<Runnable> workQueue, ThriftMetrics metrics) {
this(corePoolSize, maxPoolSize, keepAliveTime, unit, workQueue, null, metrics);
}
public THBaseThreadPoolExecutor(int corePoolSize, int maxPoolSize, long keepAliveTime,
TimeUnit unit, BlockingQueue<Runnable> workQueue,
ThreadFactory threadFactory,ThriftMetrics metrics) {
super(corePoolSize, maxPoolSize, keepAliveTime, unit, workQueue);
if (threadFactory != null) {
setThreadFactory(threadFactory);
}
this.metrics = metrics;
}
@Override
protected void beforeExecute(Thread t, Runnable r) {
super.beforeExecute(t, r);
metrics.incActiveWorkerCount();
}
@Override
protected void afterExecute(Runnable r, Throwable t) {
metrics.decActiveWorkerCount();
super.afterExecute(r, t);
}
}

View File

@ -99,6 +99,14 @@ public class ThriftMetrics {
}
}
public void incActiveWorkerCount() {
source.incActiveWorkerCount();
}
public void decActiveWorkerCount() {
source.decActiveWorkerCount();
}
/**
* Increment the count for a specific exception type. This is called for each exception type
* that is returned to the thrift handler.

View File

@ -659,8 +659,8 @@ public class ThriftServerRunner implements Runnable {
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
tfb.setDaemon(true);
tfb.setNameFormat("thrift-worker-%d");
ThreadPoolExecutor threadPool = new ThreadPoolExecutor(minWorkers, maxWorkers,
Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build());
ThreadPoolExecutor threadPool = new THBaseThreadPoolExecutor(minWorkers, maxWorkers,
Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
threadPool.allowCoreThreadTimeOut(true);
return threadPool;
}

View File

@ -28,6 +28,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@ -59,6 +60,7 @@ import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.thrift.CallQueue;
import org.apache.hadoop.hbase.thrift.CallQueue.Call;
import org.apache.hadoop.hbase.thrift.THBaseThreadPoolExecutor;
import org.apache.hadoop.hbase.thrift.ThriftMetrics;
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
import org.apache.hadoop.hbase.util.DNS;
@ -312,8 +314,8 @@ public class ThriftServer extends Configured implements Tool {
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
tfb.setDaemon(true);
tfb.setNameFormat("thrift2-worker-%d");
ThreadPoolExecutor pool = new ThreadPoolExecutor(workerThreads, workerThreads,
Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build());
ThreadPoolExecutor pool = new THBaseThreadPoolExecutor(workerThreads, workerThreads,
Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
pool.prestartAllCoreThreads();
return pool;
}
@ -324,7 +326,8 @@ public class ThriftServer extends Configured implements Tool {
int workerThreads,
InetSocketAddress inetSocketAddress,
int backlog,
int clientTimeout)
int clientTimeout,
ThriftMetrics metrics)
throws TTransportException {
TServerTransport serverTransport = new TServerSocket(
new TServerSocket.ServerSocketTransportArgs().
@ -338,6 +341,11 @@ public class ThriftServer extends Configured implements Tool {
if (workerThreads > 0) {
serverArgs.maxWorkerThreads(workerThreads);
}
ThreadPoolExecutor executor = new THBaseThreadPoolExecutor(serverArgs.minWorkerThreads,
serverArgs.maxWorkerThreads, serverArgs.stopTimeoutVal, TimeUnit.SECONDS,
new SynchronousQueue<>(), metrics);
serverArgs.executorService(executor);
return new TThreadPoolServer(serverArgs);
}
@ -574,7 +582,8 @@ public class ThriftServer extends Configured implements Tool {
workerThreads,
inetSocketAddress,
backlog,
readTimeout);
readTimeout,
metrics);
}
final TServer tserver = server;