Details
-
Bug
-
Status: Closed
-
Critical
-
Resolution: Fixed
-
1.10.0, 1.10.1, 1.11.0
Description
AbstractHandler throw NPE cause by FlinkHttpObjectAggregator is null
when rest throw exception, it will do this code
private CompletableFuture<Void> handleException(Throwable throwable, ChannelHandlerContext ctx, HttpRequest httpRequest) { FlinkHttpObjectAggregator flinkHttpObjectAggregator = ctx.pipeline().get(FlinkHttpObjectAggregator.class); int maxLength = flinkHttpObjectAggregator.maxContentLength() - OTHER_RESP_PAYLOAD_OVERHEAD; if (throwable instanceof RestHandlerException) { RestHandlerException rhe = (RestHandlerException) throwable; String stackTrace = ExceptionUtils.stringifyException(rhe); String truncatedStackTrace = Ascii.truncate(stackTrace, maxLength, "..."); if (log.isDebugEnabled()) { log.error("Exception occurred in REST handler.", rhe); } else { log.error("Exception occurred in REST handler: {}", rhe.getMessage()); } return HandlerUtils.sendErrorResponse( ctx, httpRequest, new ErrorResponseBody(truncatedStackTrace), rhe.getHttpResponseStatus(), responseHeaders); } else { log.error("Unhandled exception.", throwable); String stackTrace = String.format("<Exception on server side:%n%s%nEnd of exception on server side>", ExceptionUtils.stringifyException(throwable)); String truncatedStackTrace = Ascii.truncate(stackTrace, maxLength, "..."); return HandlerUtils.sendErrorResponse( ctx, httpRequest, new ErrorResponseBody(Arrays.asList("Internal server error.", truncatedStackTrace)), HttpResponseStatus.INTERNAL_SERVER_ERROR, responseHeaders); } }
but flinkHttpObjectAggregator some case is null,so this will throw NPE,but this method called by AbstractHandler#respondAsLeader
requestProcessingFuture .whenComplete((Void ignored, Throwable throwable) -> { if (throwable != null) { handleException(ExceptionUtils.stripCompletionException(throwable), ctx, httpRequest) .whenComplete((Void ignored2, Throwable throwable2) -> finalizeRequestProcessing(finalUploadedFiles)); } else { finalizeRequestProcessing(finalUploadedFiles); } });
the result is InFlightRequestTracker Cannot be cleared.
so the CompletableFuture does‘t complete that handler's closeAsync returned
Attachments
Attachments
Issue Links
- causes
-
FLINK-18902 Cannot serve results of asynchronous REST operations in per-job mode
- Closed
- links to