HBASE-26623 Report CallDroppedException in exception metrics (#3980)

`CallDroppedException` can be thrown with `CallRunner.drop()` by queue implementations that decide to drop calls to groom the RPC call backlog. The LifoCoDel queue does this I believe and with Pluggable queue it's possible for 3rd party queue implementations to be using `drop()` for similar reasons. It would be nice for the server to be tracking these exceptions in metrics since otherwise you might have to do some extra lifting on the client side.

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Reviewed-by: Bryan Beaudreault <bbeaudreault@hubspot.com>
This commit is contained in:
Richard Marscher 2021-12-29 11:02:50 -05:00 committed by GitHub
parent a599d52820
commit c8b014866a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 25 additions and 2 deletions

View File

@ -43,6 +43,7 @@ public interface ExceptionTrackingSource extends BaseSource {
String EXCEPTIONS_CALL_QUEUE_TOO_BIG_DESC = "Call queue is full";
String EXCEPTIONS_QUOTA_EXCEEDED = "exceptions.quotaExceeded";
String EXCEPTIONS_RPC_THROTTLING = "exceptions.rpcThrottling";
String EXCEPTIONS_CALL_DROPPED = "exceptions.callDropped";
void exception();
@ -60,4 +61,5 @@ public interface ExceptionTrackingSource extends BaseSource {
void callQueueTooBigException();
void quotaExceededException();
void rpcThrottlingException();
void callDroppedException();
}

View File

@ -40,6 +40,7 @@ public class ExceptionTrackingSourceImpl extends BaseSourceImpl
protected MutableFastCounter exceptionsCallQueueTooBig;
protected MutableFastCounter exceptionsQuotaExceeded;
protected MutableFastCounter exceptionsRpcThrottling;
protected MutableFastCounter exceptionsCallDropped;
public ExceptionTrackingSourceImpl(String metricsName, String metricsDescription,
String metricsContext, String metricsJmxContext) {
@ -72,6 +73,8 @@ public class ExceptionTrackingSourceImpl extends BaseSourceImpl
.newCounter(EXCEPTIONS_QUOTA_EXCEEDED, EXCEPTIONS_TYPE_DESC, 0L);
this.exceptionsRpcThrottling = this.getMetricsRegistry()
.newCounter(EXCEPTIONS_RPC_THROTTLING, EXCEPTIONS_TYPE_DESC, 0L);
this.exceptionsCallDropped = this.getMetricsRegistry()
.newCounter(EXCEPTIONS_CALL_DROPPED, EXCEPTIONS_TYPE_DESC, 0L);
}
@Override
@ -133,4 +136,9 @@ public class ExceptionTrackingSourceImpl extends BaseSourceImpl
public void rpcThrottlingException() {
exceptionsRpcThrottling.incr();
}
@Override
public void callDroppedException() {
exceptionsCallDropped.incr();
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.CallQueueTooBigException;
import org.apache.hadoop.hbase.MultiActionResultTooLarge;
import org.apache.hadoop.hbase.NotServingRegionException;
@ -126,6 +127,8 @@ public class MetricsHBaseServer {
source.quotaExceededException();
} else if (throwable instanceof RpcThrottlingException) {
source.rpcThrottlingException();
} else if (throwable instanceof CallDroppedException) {
source.callDroppedException();
} else if (LOG.isDebugEnabled()) {
LOG.debug("Unknown exception type", throwable);
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.ipc;
import static org.junit.Assert.*;
import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.CompatibilityFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.NotServingRegionException;
@ -143,11 +144,13 @@ public class TestRpcMetrics {
mrpc.exception(new RegionTooBusyException("Some region"));
mrpc.exception(new OutOfOrderScannerNextException());
mrpc.exception(new NotServingRegionException());
mrpc.exception(new CallDroppedException());
HELPER.assertCounter("exceptions.RegionMovedException", 1, serverSource);
HELPER.assertCounter("exceptions.RegionTooBusyException", 1, serverSource);
HELPER.assertCounter("exceptions.OutOfOrderScannerNextException", 1, serverSource);
HELPER.assertCounter("exceptions.NotServingRegionException", 1, serverSource);
HELPER.assertCounter("exceptions", 5, serverSource);
HELPER.assertCounter("exceptions.callDropped", 1, serverSource);
HELPER.assertCounter("exceptions", 6, serverSource);
}
@Test

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.thrift;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.CallQueueTooBigException;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.MultiActionResultTooLarge;
@ -154,6 +155,8 @@ public class ThriftMetrics {
source.quotaExceededException();
} else if (throwable instanceof RpcThrottlingException) {
source.rpcThrottlingException();
} else if (throwable instanceof CallDroppedException) {
source.callDroppedException();
} else if (LOG.isDebugEnabled()) {
LOG.debug("Unknown exception type", throwable);
}

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.List;
import java.util.Optional;
import org.apache.hadoop.hbase.CallDroppedException;
import org.apache.hadoop.hbase.CallQueueTooBigException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -85,6 +86,8 @@ public class ErrorThrowingGetObserver implements RegionCoprocessor, RegionObserv
throw new QuotaExceededException("Failing for test");
case RPC_THROTTLING:
throw new RpcThrottlingException("Failing for test");
case CALL_DROPPED:
throw new CallDroppedException("Failing for test");
default:
throw new DoNotRetryIOException("Failing for test");
}
@ -102,7 +105,8 @@ public class ErrorThrowingGetObserver implements RegionCoprocessor, RegionObserv
REGION_TOO_BUSY(ExceptionTrackingSource.EXCEPTIONS_BUSY_NAME),
OUT_OF_ORDER_SCANNER_NEXT(ExceptionTrackingSource.EXCEPTIONS_OOO_NAME),
QUOTA_EXCEEDED(ExceptionTrackingSource.EXCEPTIONS_QUOTA_EXCEEDED),
RPC_THROTTLING(ExceptionTrackingSource.EXCEPTIONS_RPC_THROTTLING);
RPC_THROTTLING(ExceptionTrackingSource.EXCEPTIONS_RPC_THROTTLING),
CALL_DROPPED(ExceptionTrackingSource.EXCEPTIONS_CALL_DROPPED);
private final String metricName;