[ https://issues.apache.org/jira/browse/FLINK-9841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16545107#comment-16545107 ]
ASF GitHub Bot commented on FLINK-9841: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/6329#discussion_r202655436 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java --- @@ -208,51 +209,77 @@ private void removeBlob(RemovalNotification<ResourceID, CompletableFuture<Transi } private void transferFile(ChannelHandlerContext ctx, File file, HttpRequest httpRequest) throws FlinkException { - try (final RandomAccessFile randomAccessFile = new RandomAccessFile(file, "r")) { - final long fileLength = randomAccessFile.length(); + final RandomAccessFile randomAccessFile; - try (final FileChannel fileChannel = randomAccessFile.getChannel()) { + try { + randomAccessFile = new RandomAccessFile(file, "r"); + } catch (FileNotFoundException e) { + throw new FlinkException("Can not find file " + file + ".", e); + } - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); - response.headers().set(CONTENT_TYPE, "text/plain"); + final long fileLength; - if (HttpHeaders.isKeepAlive(httpRequest)) { - response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE); - } - HttpHeaders.setContentLength(response, fileLength); - - // write the initial line and the header. - ctx.write(response); - - // write the content. - final ChannelFuture lastContentFuture; - final GenericFutureListener<Future<? super Void>> completionListener = future -> { - fileChannel.close(); - randomAccessFile.close(); - }; - - if (ctx.pipeline().get(SslHandler.class) == null) { - ctx.write( - new DefaultFileRegion(fileChannel, 0, fileLength), ctx.newProgressivePromise()) - .addListener(completionListener); - lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); - - } else { - lastContentFuture = ctx - .writeAndFlush( - new HttpChunkedInput(new ChunkedFile(randomAccessFile, 0, fileLength, 8192)), - ctx.newProgressivePromise()) - .addListener(completionListener); - - // HttpChunkedInput will write the end marker (LastHttpContent) for us. - } + try { + fileLength = randomAccessFile.length(); --- End diff -- you could move this back into the following try block, like this: ``` try { final long fileLength = randomAccessFile.length(); final FileChannel fileChannel = randomAccessFile.getChannel(); try { [... actual handler code ...] } catch (Exception e) { fileChannel.close(); throw e; } } catch (IOException ioe) { try { randomAccessFile.close(); } catch (IOException e) { log.warn("Error while closing file.", e); } } ``` > Web UI only show partial taskmanager log > ----------------------------------------- > > Key: FLINK-9841 > URL: https://issues.apache.org/jira/browse/FLINK-9841 > Project: Flink > Issue Type: Bug > Affects Versions: 1.5.0 > Environment: env : Flink on YARN > version : 1.5.0 > Reporter: vinoyang > Assignee: vinoyang > Priority: Major > Labels: pull-request-available > > > In the web UI, we select a task manager and click the "log" tab, but the UI > only show the partial log (first part), can never update even if we click the > "refresh" button. > However, the job manager is always OK. > The reason is the resource be closed twice. -- This message was sent by Atlassian JIRA (v7.6.3#76005)