|
@@ -142,8 +142,22 @@ public abstract class Server {
|
|
|
private ExceptionsHandler exceptionsHandler = new ExceptionsHandler();
|
|
|
private Tracer tracer;
|
|
|
|
|
|
+ /**
|
|
|
+ * Add exception classes for which server won't log stack traces.
|
|
|
+ *
|
|
|
+ * @param exceptionClass exception classes
|
|
|
+ */
|
|
|
public void addTerseExceptions(Class<?>... exceptionClass) {
|
|
|
- exceptionsHandler.addTerseExceptions(exceptionClass);
|
|
|
+ exceptionsHandler.addTerseLoggingExceptions(exceptionClass);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add exception classes which server won't log at all.
|
|
|
+ *
|
|
|
+ * @param exceptionClass exception classes
|
|
|
+ */
|
|
|
+ public void addSuppressedLoggingExceptions(Class<?>... exceptionClass) {
|
|
|
+ exceptionsHandler.addSuppressedLoggingExceptions(exceptionClass);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -151,29 +165,54 @@ public abstract class Server {
|
|
|
* e.g., terse exception group for concise logging messages
|
|
|
*/
|
|
|
static class ExceptionsHandler {
|
|
|
- private volatile Set<String> terseExceptions = new HashSet<String>();
|
|
|
+ private volatile Set<String> terseExceptions = new HashSet<>();
|
|
|
+ private volatile Set<String> suppressedExceptions = new HashSet<>();
|
|
|
|
|
|
/**
|
|
|
- * Add exception class so server won't log its stack trace.
|
|
|
- * Modifying the terseException through this method is thread safe.
|
|
|
- *
|
|
|
+ * Add exception classes for which server won't log stack traces.
|
|
|
+ * Optimized for infrequent invocation.
|
|
|
* @param exceptionClass exception classes
|
|
|
*/
|
|
|
- void addTerseExceptions(Class<?>... exceptionClass) {
|
|
|
+ void addTerseLoggingExceptions(Class<?>... exceptionClass) {
|
|
|
+ // Thread-safe replacement of terseExceptions.
|
|
|
+ terseExceptions = addExceptions(terseExceptions, exceptionClass);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add exception classes which server won't log at all.
|
|
|
+ * Optimized for infrequent invocation.
|
|
|
+ * @param exceptionClass exception classes
|
|
|
+ */
|
|
|
+ void addSuppressedLoggingExceptions(Class<?>... exceptionClass) {
|
|
|
+ // Thread-safe replacement of suppressedExceptions.
|
|
|
+ suppressedExceptions = addExceptions(
|
|
|
+ suppressedExceptions, exceptionClass);
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isTerseLog(Class<?> t) {
|
|
|
+ return terseExceptions.contains(t.toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isSuppressedLog(Class<?> t) {
|
|
|
+ return suppressedExceptions.contains(t.toString());
|
|
|
+ }
|
|
|
|
|
|
- // Make a copy of terseException for performing modification
|
|
|
- final HashSet<String> newSet = new HashSet<String>(terseExceptions);
|
|
|
+ /**
|
|
|
+ * Return a new set containing all the exceptions in exceptionsSet
|
|
|
+ * and exceptionClass.
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ private static Set<String> addExceptions(
|
|
|
+ final Set<String> exceptionsSet, Class<?>[] exceptionClass) {
|
|
|
+ // Make a copy of the exceptionSet for performing modification
|
|
|
+ final HashSet<String> newSet = new HashSet<>(exceptionsSet);
|
|
|
|
|
|
// Add all class names into the HashSet
|
|
|
for (Class<?> name : exceptionClass) {
|
|
|
newSet.add(name.toString());
|
|
|
}
|
|
|
- // Replace terseException set
|
|
|
- terseExceptions = Collections.unmodifiableSet(newSet);
|
|
|
- }
|
|
|
|
|
|
- boolean isTerse(Class<?> t) {
|
|
|
- return terseExceptions.contains(t.toString());
|
|
|
+ return Collections.unmodifiableSet(newSet);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -881,7 +920,7 @@ public abstract class Server {
|
|
|
}
|
|
|
|
|
|
void doRead(SelectionKey key) throws InterruptedException {
|
|
|
- int count = 0;
|
|
|
+ int count;
|
|
|
Connection c = (Connection)key.attachment();
|
|
|
if (c == null) {
|
|
|
return;
|
|
@@ -894,13 +933,17 @@ public abstract class Server {
|
|
|
LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException", ieo);
|
|
|
throw ieo;
|
|
|
} catch (Exception e) {
|
|
|
- // a WrappedRpcServerException is an exception that has been sent
|
|
|
- // to the client, so the stacktrace is unnecessary; any other
|
|
|
- // exceptions are unexpected internal server errors and thus the
|
|
|
- // stacktrace should be logged
|
|
|
- LOG.info(Thread.currentThread().getName() + ": readAndProcess from client " +
|
|
|
- c.getHostAddress() + " threw exception [" + e + "]",
|
|
|
- (e instanceof WrappedRpcServerException) ? null : e);
|
|
|
+ // Do not log WrappedRpcServerExceptionSuppressed.
|
|
|
+ if (!(e instanceof WrappedRpcServerExceptionSuppressed)) {
|
|
|
+ // A WrappedRpcServerException is an exception that has been sent
|
|
|
+ // to the client, so the stacktrace is unnecessary; any other
|
|
|
+ // exceptions are unexpected internal server errors and thus the
|
|
|
+ // stacktrace should be logged.
|
|
|
+ LOG.info(Thread.currentThread().getName() +
|
|
|
+ ": readAndProcess from client " + c.getHostAddress() +
|
|
|
+ " threw exception [" + e + "]",
|
|
|
+ (e instanceof WrappedRpcServerException) ? null : e);
|
|
|
+ }
|
|
|
count = -1; //so that the (count < 0) block is executed
|
|
|
}
|
|
|
if (count < 0) {
|
|
@@ -1243,6 +1286,18 @@ public abstract class Server {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * A WrappedRpcServerException that is suppressed altogether
|
|
|
+ * for the purposes of logging.
|
|
|
+ */
|
|
|
+ private static class WrappedRpcServerExceptionSuppressed
|
|
|
+ extends WrappedRpcServerException {
|
|
|
+ public WrappedRpcServerExceptionSuppressed(
|
|
|
+ RpcErrorCodeProto errCode, IOException ioe) {
|
|
|
+ super(errCode, ioe);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/** Reads calls from a connection and queues them for handling. */
|
|
|
public class Connection {
|
|
|
private boolean connectionHeaderRead = false; // connection header is read?
|
|
@@ -2117,7 +2172,7 @@ public abstract class Server {
|
|
|
rpcMetrics.incrClientBackoff();
|
|
|
RetriableException retriableException =
|
|
|
new RetriableException("Server is too busy.");
|
|
|
- throw new WrappedRpcServerException(
|
|
|
+ throw new WrappedRpcServerExceptionSuppressed(
|
|
|
RpcErrorCodeProto.ERROR_RPC_SERVER, retriableException);
|
|
|
}
|
|
|
}
|
|
@@ -2313,18 +2368,7 @@ public abstract class Server {
|
|
|
if (e instanceof UndeclaredThrowableException) {
|
|
|
e = e.getCause();
|
|
|
}
|
|
|
- String logMsg = Thread.currentThread().getName() + ", call " + call;
|
|
|
- if (exceptionsHandler.isTerse(e.getClass())) {
|
|
|
- // Don't log the whole stack trace. Way too noisy!
|
|
|
- LOG.info(logMsg + ": " + e);
|
|
|
- } else if (e instanceof RuntimeException || e instanceof Error) {
|
|
|
- // These exception types indicate something is probably wrong
|
|
|
- // on the server side, as opposed to just a normal exceptional
|
|
|
- // result.
|
|
|
- LOG.warn(logMsg, e);
|
|
|
- } else {
|
|
|
- LOG.info(logMsg, e);
|
|
|
- }
|
|
|
+ logException(LOG, e, call);
|
|
|
if (e instanceof RpcServerException) {
|
|
|
RpcServerException rse = ((RpcServerException)e);
|
|
|
returnStatus = rse.getRpcStatusProto();
|
|
@@ -2377,6 +2421,26 @@ public abstract class Server {
|
|
|
}
|
|
|
|
|
|
}
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ void logException(Log logger, Throwable e, Call call) {
|
|
|
+ if (exceptionsHandler.isSuppressedLog(e.getClass())) {
|
|
|
+ return; // Log nothing.
|
|
|
+ }
|
|
|
+
|
|
|
+ final String logMsg = Thread.currentThread().getName() + ", call " + call;
|
|
|
+ if (exceptionsHandler.isTerseLog(e.getClass())) {
|
|
|
+ // Don't log the whole stack trace. Way too noisy!
|
|
|
+ logger.info(logMsg + ": " + e);
|
|
|
+ } else if (e instanceof RuntimeException || e instanceof Error) {
|
|
|
+ // These exception types indicate something is probably wrong
|
|
|
+ // on the server side, as opposed to just a normal exceptional
|
|
|
+ // result.
|
|
|
+ logger.warn(logMsg, e);
|
|
|
+ } else {
|
|
|
+ logger.info(logMsg, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
protected Server(String bindAddress, int port,
|
|
|
Class<? extends Writable> paramClass, int handlerCount,
|
|
@@ -2482,7 +2546,7 @@ public abstract class Server {
|
|
|
saslPropsResolver = SaslPropertiesResolver.getInstance(conf);
|
|
|
}
|
|
|
|
|
|
- this.exceptionsHandler.addTerseExceptions(StandbyException.class);
|
|
|
+ this.exceptionsHandler.addTerseLoggingExceptions(StandbyException.class);
|
|
|
}
|
|
|
|
|
|
private RpcSaslProto buildNegotiateResponse(List<AuthMethod> authMethods)
|