From c06e7a5427675efb63aff86debd03581343cfdb7 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 20 Dec 2018 14:18:46 +0100 Subject: [PATCH] [FLINK-11134][rest] Do not log stacktrace for handled exceptions --- .../flink/runtime/rest/handler/AbstractHandler.java | 9 ++++++--- .../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 865bdb1c9248a..a87d3ada5eb8b 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 @@ protected void respondAsLeader(ChannelHandlerContext ctx, RoutedRequest routedRe 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 @@ protected void respondAsLeader(ChannelHandlerContext ctx, RoutedRequest routedRe }); } 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 a0cff90e9d5cd..d88abaabc66aa 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 @@ private Tuple2 errorResponse(Throwable throwab 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);