HBASE-2782 QOS for META table access
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@997541 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
191a334487
commit
5b600805b5
@ -910,6 +910,7 @@ Release 0.21.0 - Unreleased
|
||||
HBASE-2980 Refactor region server command line to a new class
|
||||
HBASE-2988 Support alternate compression for major compactions
|
||||
HBASE-2941 port HADOOP-6713 - threading scalability for RPC reads - to HBase
|
||||
HBASE-2782 QOS for META table access
|
||||
|
||||
NEW FEATURES
|
||||
HBASE-1961 HBase EC2 scripts
|
||||
|
@ -30,6 +30,7 @@ import java.io.DataInput;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
@ -75,6 +76,10 @@ public final class MultiAction implements Writable {
|
||||
rsActions.add(a);
|
||||
}
|
||||
|
||||
public Set<byte[]> getRegions() {
|
||||
return actions.keySet();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return All actions from all regions in this container
|
||||
*/
|
||||
|
@ -20,12 +20,14 @@
|
||||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.ipc.VersionedProtocol;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
@ -82,8 +84,9 @@ public class HBaseRPC {
|
||||
super();
|
||||
} // no public ctor
|
||||
|
||||
|
||||
/** A method invocation, including the method name and its parameters.*/
|
||||
private static class Invocation implements Writable, Configurable {
|
||||
public static class Invocation implements Writable, Configurable {
|
||||
private String methodName;
|
||||
@SuppressWarnings("unchecked")
|
||||
private Class[] parameterClasses;
|
||||
@ -497,9 +500,9 @@ public class HBaseRPC {
|
||||
final Class<?>[] ifaces,
|
||||
final String bindAddress, final int port,
|
||||
final int numHandlers,
|
||||
final boolean verbose, Configuration conf)
|
||||
int metaHandlerCount, final boolean verbose, Configuration conf, int highPriorityLevel)
|
||||
throws IOException {
|
||||
return new Server(instance, ifaces, conf, bindAddress, port, numHandlers, verbose);
|
||||
return new Server(instance, ifaces, conf, bindAddress, port, numHandlers, metaHandlerCount, verbose, highPriorityLevel);
|
||||
}
|
||||
|
||||
/** An RPC Server. */
|
||||
@ -527,9 +530,9 @@ public class HBaseRPC {
|
||||
* @throws IOException e
|
||||
*/
|
||||
public Server(Object instance, final Class<?>[] ifaces,
|
||||
Configuration conf, String bindAddress, int port,
|
||||
int numHandlers, boolean verbose) throws IOException {
|
||||
super(bindAddress, port, Invocation.class, numHandlers, conf, classNameBase(instance.getClass().getName()));
|
||||
Configuration conf, String bindAddress, int port,
|
||||
int numHandlers, int metaHandlerCount, boolean verbose, int highPriorityLevel) throws IOException {
|
||||
super(bindAddress, port, Invocation.class, numHandlers, metaHandlerCount, conf, classNameBase(instance.getClass().getName()), highPriorityLevel);
|
||||
this.instance = instance;
|
||||
this.implementation = instance.getClass();
|
||||
|
||||
|
@ -20,9 +20,11 @@
|
||||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.io.ObjectWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
@ -131,6 +133,7 @@ public abstract class HBaseServer {
|
||||
protected String bindAddress;
|
||||
protected int port; // port we listen on
|
||||
private int handlerCount; // number of handler threads
|
||||
private int priorityHandlerCount;
|
||||
private int readThreads; // number of read threads
|
||||
protected Class<? extends Writable> paramClass; // class of call parameters
|
||||
protected int maxIdleTime; // the maximum idle time after
|
||||
@ -156,6 +159,9 @@ public abstract class HBaseServer {
|
||||
|
||||
volatile protected boolean running = true; // true while server runs
|
||||
protected BlockingQueue<Call> callQueue; // queued calls
|
||||
protected BlockingQueue<Call> priorityCallQueue;
|
||||
|
||||
private int highPriorityLevel; // what level a high priority call is at
|
||||
|
||||
protected final List<Connection> connectionList =
|
||||
Collections.synchronizedList(new LinkedList<Connection>());
|
||||
@ -165,6 +171,7 @@ public abstract class HBaseServer {
|
||||
protected Responder responder = null;
|
||||
protected int numConnections = 0;
|
||||
private Handler[] handlers = null;
|
||||
private Handler[] priorityHandlers = null;
|
||||
protected HBaseRPCErrorHandler errorHandler = null;
|
||||
|
||||
/**
|
||||
@ -959,7 +966,12 @@ public abstract class HBaseServer {
|
||||
param.readFields(dis);
|
||||
|
||||
Call call = new Call(id, param, this);
|
||||
callQueue.put(call); // queue the call; maybe blocked here
|
||||
|
||||
if (priorityCallQueue != null && getQosLevel(param) > highPriorityLevel) {
|
||||
priorityCallQueue.put(call);
|
||||
} else {
|
||||
callQueue.put(call); // queue the call; maybe blocked here
|
||||
}
|
||||
}
|
||||
|
||||
protected synchronized void close() {
|
||||
@ -977,9 +989,17 @@ public abstract class HBaseServer {
|
||||
|
||||
/** Handles queued calls . */
|
||||
private class Handler extends Thread {
|
||||
public Handler(int instanceNumber) {
|
||||
private final BlockingQueue<Call> myCallQueue;
|
||||
public Handler(final BlockingQueue<Call> cq, int instanceNumber) {
|
||||
this.myCallQueue = cq;
|
||||
this.setDaemon(true);
|
||||
this.setName("IPC Server handler "+ instanceNumber + " on " + port);
|
||||
|
||||
String threadName = "IPC Server handler " + instanceNumber + " on " + port;
|
||||
if (cq == priorityCallQueue) {
|
||||
// this is just an amazing hack, but it works.
|
||||
threadName = "PRI " + threadName;
|
||||
}
|
||||
this.setName(threadName);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -990,7 +1010,7 @@ public abstract class HBaseServer {
|
||||
ByteArrayOutputStream buf = new ByteArrayOutputStream(buffersize);
|
||||
while (running) {
|
||||
try {
|
||||
Call call = callQueue.take(); // pop the queue; maybe blocked here
|
||||
Call call = myCallQueue.take(); // pop the queue; maybe blocked here
|
||||
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(getName() + ": has #" + call.id + " from " +
|
||||
@ -1058,33 +1078,58 @@ public abstract class HBaseServer {
|
||||
|
||||
}
|
||||
|
||||
protected HBaseServer(String bindAddress, int port,
|
||||
Class<? extends Writable> paramClass, int handlerCount,
|
||||
Configuration conf)
|
||||
throws IOException
|
||||
{
|
||||
this(bindAddress, port, paramClass, handlerCount, conf, Integer.toString(port));
|
||||
/**
|
||||
* Gets the QOS level for this call. If it is higher than the highPriorityLevel and there
|
||||
* are priorityHandlers available it will be processed in it's own thread set.
|
||||
*
|
||||
* @param param
|
||||
* @return priority, higher is better
|
||||
*/
|
||||
private Function<Writable,Integer> qosFunction = null;
|
||||
public void setQosFunction(Function<Writable, Integer> newFunc) {
|
||||
qosFunction = newFunc;
|
||||
}
|
||||
|
||||
protected int getQosLevel(Writable param) {
|
||||
if (qosFunction == null) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
Integer res = qosFunction.apply(param);
|
||||
if (res == null) {
|
||||
return 0;
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/* Constructs a server listening on the named port and address. Parameters passed must
|
||||
* be of the named class. The <code>handlerCount</handlerCount> determines
|
||||
* the number of handler threads that will be used to process calls.
|
||||
*
|
||||
*/
|
||||
protected HBaseServer(String bindAddress, int port,
|
||||
Class<? extends Writable> paramClass, int handlerCount,
|
||||
Configuration conf, String serverName)
|
||||
Class<? extends Writable> paramClass, int handlerCount,
|
||||
int priorityHandlerCount, Configuration conf, String serverName,
|
||||
int highPriorityLevel)
|
||||
throws IOException {
|
||||
this.bindAddress = bindAddress;
|
||||
this.conf = conf;
|
||||
this.port = port;
|
||||
this.paramClass = paramClass;
|
||||
this.handlerCount = handlerCount;
|
||||
this.priorityHandlerCount = priorityHandlerCount;
|
||||
this.socketSendBufferSize = 0;
|
||||
this.maxQueueSize = handlerCount * MAX_QUEUE_SIZE_PER_HANDLER;
|
||||
this.readThreads = conf.getInt(
|
||||
"ipc.server.read.threadpool.size",
|
||||
10);
|
||||
this.callQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
|
||||
if (priorityHandlerCount > 0) {
|
||||
this.priorityCallQueue = new LinkedBlockingQueue<Call>(maxQueueSize); // TODO hack on size
|
||||
} else {
|
||||
this.priorityCallQueue = null;
|
||||
}
|
||||
this.highPriorityLevel = highPriorityLevel;
|
||||
this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
|
||||
this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
|
||||
this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
|
||||
@ -1121,9 +1166,17 @@ public abstract class HBaseServer {
|
||||
handlers = new Handler[handlerCount];
|
||||
|
||||
for (int i = 0; i < handlerCount; i++) {
|
||||
handlers[i] = new Handler(i);
|
||||
handlers[i] = new Handler(callQueue, i);
|
||||
handlers[i].start();
|
||||
}
|
||||
|
||||
if (priorityHandlerCount > 0) {
|
||||
priorityHandlers = new Handler[priorityHandlerCount];
|
||||
for (int i = 0 ; i < priorityHandlerCount; i++) {
|
||||
priorityHandlers[i] = new Handler(priorityCallQueue, i);
|
||||
priorityHandlers[i].start();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Stops the service. No new calls will be handled after this is called. */
|
||||
@ -1131,9 +1184,16 @@ public abstract class HBaseServer {
|
||||
LOG.info("Stopping server on " + port);
|
||||
running = false;
|
||||
if (handlers != null) {
|
||||
for (int i = 0; i < handlerCount; i++) {
|
||||
if (handlers[i] != null) {
|
||||
handlers[i].interrupt();
|
||||
for (Handler handler : handlers) {
|
||||
if (handler != null) {
|
||||
handler.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
if (priorityHandlers != null) {
|
||||
for (Handler handler : priorityHandlers) {
|
||||
if (handler != null) {
|
||||
handler.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -185,7 +185,10 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
||||
this.rpcServer = HBaseRPC.getServer(this,
|
||||
new Class<?>[]{HMasterInterface.class, HMasterRegionInterface.class},
|
||||
a.getBindAddress(), a.getPort(),
|
||||
numHandlers, false, conf);
|
||||
numHandlers,
|
||||
0, // we dont use high priority handlers in master
|
||||
false, conf,
|
||||
0); // this is a DNC w/o high priority handlers
|
||||
this.address = new HServerAddress(rpcServer.getListenerAddress());
|
||||
|
||||
// set the thread name now we have an address
|
||||
|
@ -2135,6 +2135,9 @@ public class HRegion implements HeapSize { // , Writable{
|
||||
private boolean filterClosed = false;
|
||||
private long readPt;
|
||||
|
||||
public HRegionInfo getRegionName() {
|
||||
return regionInfo;
|
||||
}
|
||||
RegionScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
|
||||
//DebugPrint.println("HRegionScanner.<init>");
|
||||
this.filter = scan.getFilter();
|
||||
|
@ -48,6 +48,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@ -304,6 +305,85 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
||||
initialize();
|
||||
}
|
||||
|
||||
private static final int NORMAL_QOS = 0;
|
||||
private static final int QOS_THRESHOLD = 10; // the line between low and high qos
|
||||
private static final int HIGH_QOS = 100;
|
||||
|
||||
class QosFunction implements Function<Writable,Integer> {
|
||||
public boolean isMetaRegion(byte[] regionName) {
|
||||
HRegion region;
|
||||
try {
|
||||
region = getRegion(regionName);
|
||||
} catch (NotServingRegionException ignored) {
|
||||
return false;
|
||||
}
|
||||
return region.getRegionInfo().isMetaRegion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer apply(Writable from) {
|
||||
if (from instanceof HBaseRPC.Invocation) {
|
||||
HBaseRPC.Invocation inv = (HBaseRPC.Invocation) from;
|
||||
|
||||
String methodName = inv.getMethodName();
|
||||
|
||||
// scanner methods...
|
||||
if (methodName.equals("next") || methodName.equals("close")) {
|
||||
// translate!
|
||||
Long scannerId;
|
||||
try {
|
||||
scannerId = (Long) inv.getParameters()[0];
|
||||
} catch (ClassCastException ignored) {
|
||||
//LOG.debug("Low priority: " + from);
|
||||
return NORMAL_QOS; // doh.
|
||||
}
|
||||
String scannerIdString = Long.toString(scannerId);
|
||||
InternalScanner scanner = scanners.get(scannerIdString);
|
||||
if (scanner instanceof HRegion.RegionScanner) {
|
||||
HRegion.RegionScanner rs = (HRegion.RegionScanner) scanner;
|
||||
HRegionInfo regionName = rs.getRegionName();
|
||||
if (regionName.isMetaRegion()) {
|
||||
//LOG.debug("High priority scanner request: " + scannerId);
|
||||
return HIGH_QOS;
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (methodName.equals("getHServerInfo") ||
|
||||
methodName.equals("getRegionsAssignment") ||
|
||||
methodName.equals("unlockRow") ||
|
||||
methodName.equals("getProtocolVersion") ||
|
||||
methodName.equals("getClosestRowBefore")) {
|
||||
//LOG.debug("High priority method: " + methodName);
|
||||
return HIGH_QOS;
|
||||
}
|
||||
else if (inv.getParameterClasses()[0] == byte[].class) {
|
||||
// first arg is byte array, so assume this is a regionname:
|
||||
if (isMetaRegion((byte[]) inv.getParameters()[0])) {
|
||||
//LOG.debug("High priority with method: " + methodName + " and region: "
|
||||
// + Bytes.toString((byte[]) inv.getParameters()[0]));
|
||||
return HIGH_QOS;
|
||||
}
|
||||
}
|
||||
else if (inv.getParameterClasses()[0] == MultiAction.class) {
|
||||
MultiAction ma = (MultiAction) inv.getParameters()[0];
|
||||
Set<byte[]> regions = ma.getRegions();
|
||||
// ok this sucks, but if any single of the actions touches a meta, the whole
|
||||
// thing gets pingged high priority. This is a dangerous hack because people
|
||||
// can get their multi action tagged high QOS by tossing a Get(.META.) AND this
|
||||
// regionserver hosts META/-ROOT-
|
||||
for (byte[] region: regions) {
|
||||
if (isMetaRegion(region)) {
|
||||
//LOG.debug("High priority multi with region: " + Bytes.toString(region));
|
||||
return HIGH_QOS; // short circuit for the win.
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
//LOG.debug("Low priority: " + from.toString());
|
||||
return NORMAL_QOS;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates all of the state that needs to be reconstructed in case we are
|
||||
* doing a restart. This is shared between the constructor and restart(). Both
|
||||
@ -326,8 +406,11 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
||||
OnlineRegions.class},
|
||||
address.getBindAddress(),
|
||||
address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
|
||||
false, conf);
|
||||
conf.getInt("hbase.regionserver.metahandler.count", 10),
|
||||
false, conf, QOS_THRESHOLD);
|
||||
this.server.setErrorHandler(this);
|
||||
this.server.setQosFunction(new QosFunction());
|
||||
|
||||
// Address is giving a default IP for the moment. Will be changed after
|
||||
// calling the master.
|
||||
this.serverInfo = new HServerInfo(new HServerAddress(new InetSocketAddress(
|
||||
|
Loading…
x
Reference in New Issue
Block a user