You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2019/01/10 12:12:15 UTC
[flink] 01/03: [FLINK-11134][rest] Do not log stacktrace for
handled exceptions
This is an automated email from the ASF dual-hosted git repository.
chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit c06e7a5427675efb63aff86debd03581343cfdb7
Author: zentol <ch...@apache.org>
AuthorDate: Thu Dec 20 14:18:46 2018 +0100
[FLINK-11134][rest] Do not log stacktrace for handled exceptions
---
.../org/apache/flink/runtime/rest/handler/AbstractHandler.java | 9 ++++++---
.../apache/flink/runtime/rest/handler/AbstractRestHandler.java | 6 +++++-
2 files changed, 11 insertions(+), 4 deletions(-)
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java
index 865bdb1..a87d3ad 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java
@@ -116,15 +116,13 @@ public abstract class AbstractHandler<T extends RestfulGateway, R extends Reques
try {
request = MAPPER.readValue("{}", untypedResponseMessageHeaders.getRequestClass());
} catch (JsonParseException | JsonMappingException je) {
- log.error("Request did not conform to expected format.", je);
- throw new RestHandlerException("Bad request received.", HttpResponseStatus.BAD_REQUEST, je);
+ throw new RestHandlerException("Bad request received. Request did not conform to expected format.", HttpResponseStatus.BAD_REQUEST, je);
}
} else {
try {
ByteBufInputStream in = new ByteBufInputStream(msgContent);
request = MAPPER.readValue(in, untypedResponseMessageHeaders.getRequestClass());
} catch (JsonParseException | JsonMappingException je) {
- log.error("Failed to read request.", je);
throw new RestHandlerException(
String.format("Request did not match expected format %s.", untypedResponseMessageHeaders.getRequestClass().getSimpleName()),
HttpResponseStatus.BAD_REQUEST,
@@ -164,6 +162,11 @@ public abstract class AbstractHandler<T extends RestfulGateway, R extends Reques
});
} catch (RestHandlerException rhe) {
inFlightRequestTracker.deregisterRequest();
+ if (log.isDebugEnabled()) {
+ log.error("Exception occurred in REST handler.", rhe);
+ } else {
+ log.error("Exception occurred in REST handler: {}", rhe.getMessage());
+ }
HandlerUtils.sendErrorResponse(
ctx,
httpRequest,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
index a0cff90..d88abaa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java
@@ -88,7 +88,11 @@ public abstract class AbstractRestHandler<T extends RestfulGateway, R extends Re
Throwable error = ExceptionUtils.stripCompletionException(throwable);
if (error instanceof RestHandlerException) {
final RestHandlerException rhe = (RestHandlerException) error;
- log.error("Exception occurred in REST handler.", rhe);
+ if (log.isDebugEnabled()) {
+ log.error("Exception occurred in REST handler.", rhe);
+ } else {
+ log.error("Exception occurred in REST handler: {}", rhe.getMessage());
+ }
return Tuple2.of(new ErrorResponseBody(rhe.getMessage()), rhe.getHttpResponseStatus());
} else {
log.error("Implementation error: Unhandled exception.", error);