HBASE-26154: Adds exception metrics for QuotaExceededException and RpcThrottlingException (#3545)
Signed-off-by: Xiaolin Ha <haxiaolin@apache.org> Signed-off-by: Pankaj Kumar<pankajkumar@apache.org>
This commit is contained in:
parent
b248730126
commit
2e9ab3ce94
|
@ -41,6 +41,8 @@ public interface ExceptionTrackingSource extends BaseSource {
|
|||
"rest of the requests will have to be retried.";
|
||||
String EXCEPTIONS_CALL_QUEUE_TOO_BIG = "exceptions.callQueueTooBig";
|
||||
String EXCEPTIONS_CALL_QUEUE_TOO_BIG_DESC = "Call queue is full";
|
||||
String EXCEPTIONS_QUOTA_EXCEEDED = "exceptions.quotaExceeded";
|
||||
String EXCEPTIONS_RPC_THROTTLING = "exceptions.rpcThrottling";
|
||||
|
||||
void exception();
|
||||
|
||||
|
@ -56,4 +58,6 @@ public interface ExceptionTrackingSource extends BaseSource {
|
|||
void tooBusyException();
|
||||
void multiActionTooLargeException();
|
||||
void callQueueTooBigException();
|
||||
void quotaExceededException();
|
||||
void rpcThrottlingException();
|
||||
}
|
||||
|
|
|
@ -38,6 +38,8 @@ public class ExceptionTrackingSourceImpl extends BaseSourceImpl
|
|||
protected MutableFastCounter exceptionsMoved;
|
||||
protected MutableFastCounter exceptionsMultiTooLarge;
|
||||
protected MutableFastCounter exceptionsCallQueueTooBig;
|
||||
protected MutableFastCounter exceptionsQuotaExceeded;
|
||||
protected MutableFastCounter exceptionsRpcThrottling;
|
||||
|
||||
public ExceptionTrackingSourceImpl(String metricsName, String metricsDescription,
|
||||
String metricsContext, String metricsJmxContext) {
|
||||
|
@ -66,6 +68,10 @@ public class ExceptionTrackingSourceImpl extends BaseSourceImpl
|
|||
.newCounter(EXCEPTIONS_MULTI_TOO_LARGE_NAME, EXCEPTIONS_MULTI_TOO_LARGE_DESC, 0L);
|
||||
this.exceptionsCallQueueTooBig = this.getMetricsRegistry()
|
||||
.newCounter(EXCEPTIONS_CALL_QUEUE_TOO_BIG, EXCEPTIONS_CALL_QUEUE_TOO_BIG_DESC, 0L);
|
||||
this.exceptionsQuotaExceeded = this.getMetricsRegistry()
|
||||
.newCounter(EXCEPTIONS_QUOTA_EXCEEDED, EXCEPTIONS_TYPE_DESC, 0L);
|
||||
this.exceptionsRpcThrottling = this.getMetricsRegistry()
|
||||
.newCounter(EXCEPTIONS_RPC_THROTTLING, EXCEPTIONS_TYPE_DESC, 0L);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -117,4 +123,14 @@ public class ExceptionTrackingSourceImpl extends BaseSourceImpl
|
|||
public void callQueueTooBigException() {
|
||||
exceptionsCallQueueTooBig.incr();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void quotaExceededException() {
|
||||
exceptionsQuotaExceeded.incr();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void rpcThrottlingException() {
|
||||
exceptionsRpcThrottling.incr();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -184,7 +184,7 @@ public class MetricsAssertHelperImpl implements MetricsAssertHelper {
|
|||
@Override
|
||||
public void assertCounter(String name, long expected, BaseSource source) {
|
||||
long found = getCounter(name, source);
|
||||
assertEquals("Metrics Counters should be equal", (long) Long.valueOf(expected), found);
|
||||
assertEquals(name + "(" + found + ") should be equal", (long) Long.valueOf(expected), found);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,15 +24,21 @@ import org.apache.hadoop.hbase.MultiActionResultTooLarge;
|
|||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionTooBusyException;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
|
||||
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsHBaseServer {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(MetricsHBaseServer.class);
|
||||
|
||||
private MetricsHBaseServerSource source;
|
||||
private MetricsHBaseServerWrapper serverWrapper;
|
||||
|
||||
|
@ -116,6 +122,12 @@ public class MetricsHBaseServer {
|
|||
source.multiActionTooLargeException();
|
||||
} else if (throwable instanceof CallQueueTooBigException) {
|
||||
source.callQueueTooBigException();
|
||||
} else if (throwable instanceof QuotaExceededException) {
|
||||
source.quotaExceededException();
|
||||
} else if (throwable instanceof RpcThrottlingException) {
|
||||
source.rpcThrottlingException();
|
||||
} else if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Unknown exception type", throwable);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,9 +31,13 @@ import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
|||
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
|
||||
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
|
||||
import org.apache.hadoop.hbase.thrift.generated.IOError;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIOError;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* This class is for maintaining the various statistics of thrift server
|
||||
|
@ -42,6 +46,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public class ThriftMetrics {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ThriftMetrics.class);
|
||||
|
||||
public enum ThriftServerType {
|
||||
ONE,
|
||||
|
@ -145,6 +150,12 @@ public class ThriftMetrics {
|
|||
source.multiActionTooLargeException();
|
||||
} else if (throwable instanceof CallQueueTooBigException) {
|
||||
source.callQueueTooBigException();
|
||||
} else if (throwable instanceof QuotaExceededException) {
|
||||
source.quotaExceededException();
|
||||
} else if (throwable instanceof RpcThrottlingException) {
|
||||
source.rpcThrottlingException();
|
||||
} else if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Unknown exception type", throwable);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,8 @@ import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
|||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
|
||||
import org.apache.hadoop.hbase.metrics.ExceptionTrackingSource;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
|
||||
import org.apache.hadoop.hbase.quotas.RpcThrottlingException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -79,6 +81,10 @@ public class ErrorThrowingGetObserver implements RegionCoprocessor, RegionObserv
|
|||
throw new RegionTooBusyException("Failing for test");
|
||||
case OUT_OF_ORDER_SCANNER_NEXT:
|
||||
throw new OutOfOrderScannerNextException("Failing for test");
|
||||
case QUOTA_EXCEEDED:
|
||||
throw new QuotaExceededException("Failing for test");
|
||||
case RPC_THROTTLING:
|
||||
throw new RpcThrottlingException("Failing for test");
|
||||
default:
|
||||
throw new DoNotRetryIOException("Failing for test");
|
||||
}
|
||||
|
@ -94,7 +100,9 @@ public class ErrorThrowingGetObserver implements RegionCoprocessor, RegionObserv
|
|||
SCANNER_RESET(ExceptionTrackingSource.EXCEPTIONS_SCANNER_RESET_NAME),
|
||||
UNKNOWN_SCANNER(ExceptionTrackingSource.EXCEPTIONS_UNKNOWN_NAME),
|
||||
REGION_TOO_BUSY(ExceptionTrackingSource.EXCEPTIONS_BUSY_NAME),
|
||||
OUT_OF_ORDER_SCANNER_NEXT(ExceptionTrackingSource.EXCEPTIONS_OOO_NAME);
|
||||
OUT_OF_ORDER_SCANNER_NEXT(ExceptionTrackingSource.EXCEPTIONS_OOO_NAME),
|
||||
QUOTA_EXCEEDED(ExceptionTrackingSource.EXCEPTIONS_QUOTA_EXCEEDED),
|
||||
RPC_THROTTLING(ExceptionTrackingSource.EXCEPTIONS_RPC_THROTTLING);
|
||||
|
||||
private final String metricName;
|
||||
|
||||
|
|
Loading…
Reference in New Issue