Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-18663

RestServerEndpoint may prevent server shutdown

    XMLWordPrintableJSON

Details

    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

        1. 110.png
          1.59 MB
          tartarus
        2. 111.png
          602 kB
          tartarus
        3. C49A7310-F932-451B-A203-6D17F3140C0D.png
          308 kB
          tartarus
        4. e18e00dd6664485c2ff55284fe969474.png
          1.41 MB
          tartarus
        5. jobmanager.log.noyarn.tar.gz
          2.05 MB
          tartarus

        Issue Links

          Activity

            People

              tartarus tartarus
              tartarus tartarus
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: