[ 
https://issues.apache.org/jira/browse/FLINK-8503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16352385#comment-16352385
 ] 

ASF GitHub Bot commented on FLINK-8503:
---------------------------------------

Github user GJL commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5353#discussion_r165972923
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java
 ---
    @@ -0,0 +1,236 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.handler.taskmanager;
    +
    +import org.apache.flink.api.common.time.Time;
    +import org.apache.flink.runtime.blob.TransientBlobKey;
    +import org.apache.flink.runtime.blob.TransientBlobService;
    +import org.apache.flink.runtime.clusterframework.types.ResourceID;
    +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
    +import org.apache.flink.runtime.rest.AbstractHandler;
    +import org.apache.flink.runtime.rest.handler.HandlerRequest;
    +import org.apache.flink.runtime.rest.handler.RestHandlerException;
    +import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
    +import 
org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
    +import 
org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter;
    +import 
org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
    +import org.apache.flink.runtime.taskexecutor.TaskExecutor;
    +import org.apache.flink.runtime.webmonitor.RestfulGateway;
    +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.FlinkException;
    +import org.apache.flink.util.Preconditions;
    +
    +import 
org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
    +import org.apache.flink.shaded.guava18.com.google.common.cache.CacheLoader;
    +import 
org.apache.flink.shaded.guava18.com.google.common.cache.LoadingCache;
    +import 
org.apache.flink.shaded.guava18.com.google.common.cache.RemovalNotification;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
    +import 
org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
    +import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent;
    +import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler;
    +import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedFile;
    +import org.apache.flink.shaded.netty4.io.netty.util.concurrent.Future;
    +import 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.GenericFutureListener;
    +
    +import javax.annotation.Nonnull;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.RandomAccessFile;
    +import java.nio.channels.FileChannel;
    +import java.util.Map;
    +import java.util.concurrent.CompletableFuture;
    +import java.util.concurrent.CompletionException;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.TimeUnit;
    +
    +import static 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
    +import static 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
    +import static 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK;
    +import static 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
    +
    +/**
    + * Base class for serving files from the {@link TaskExecutor}.
    + */
    +public abstract class AbstractTaskManagerFileHandler<M extends 
TaskManagerMessageParameters> extends AbstractHandler<RestfulGateway, 
EmptyRequestBody, M> {
    +
    +   private final GatewayRetriever<ResourceManagerGateway> 
resourceManagerGatewayRetriever;
    +   private final TransientBlobService transientBlobService;
    +
    +   private final LoadingCache<ResourceID, 
CompletableFuture<TransientBlobKey>> fileBlobKeys;
    +
    +   protected AbstractTaskManagerFileHandler(
    +                   @Nonnull CompletableFuture<String> localAddressFuture,
    +                   @Nonnull GatewayRetriever<? extends RestfulGateway> 
leaderRetriever,
    +                   @Nonnull Time timeout,
    +                   @Nonnull Map<String, String> responseHeaders,
    +                   @Nonnull 
UntypedResponseMessageHeaders<EmptyRequestBody, M> 
untypedResponseMessageHeaders,
    +                   @Nonnull GatewayRetriever<ResourceManagerGateway> 
resourceManagerGatewayRetriever,
    +                   @Nonnull TransientBlobService transientBlobService,
    +                   @Nonnull Time cacheEntryDuration) {
    +           super(localAddressFuture, leaderRetriever, timeout, 
responseHeaders, untypedResponseMessageHeaders);
    +
    +           this.resourceManagerGatewayRetriever = 
Preconditions.checkNotNull(resourceManagerGatewayRetriever);
    +
    +           this.transientBlobService = 
Preconditions.checkNotNull(transientBlobService);
    +
    +           this.fileBlobKeys = CacheBuilder
    +                   .newBuilder()
    +                   .expireAfterWrite(cacheEntryDuration.toMilliseconds(), 
TimeUnit.MILLISECONDS)
    +                   .removalListener(this::removeBlob)
    +                   .build(
    +                           new CacheLoader<ResourceID, 
CompletableFuture<TransientBlobKey>>() {
    +                                   @Override
    +                                   public 
CompletableFuture<TransientBlobKey> load(ResourceID resourceId) throws 
Exception {
    +                                           return 
loadTaskManagerFile(resourceId);
    +                                   }
    +                   });
    +   }
    +
    +   @Override
    +   protected void respondToRequest(ChannelHandlerContext ctx, HttpRequest 
httpRequest, HandlerRequest<EmptyRequestBody, M> handlerRequest, RestfulGateway 
gateway) throws RestHandlerException {
    +           final ResourceID taskManagerId = 
handlerRequest.getPathParameter(TaskManagerIdPathParameter.class);
    +
    +           final CompletableFuture<TransientBlobKey> blobKeyFuture;
    +           try {
    +                   blobKeyFuture = fileBlobKeys.get(taskManagerId);
    +           } catch (ExecutionException e) {
    +                   final Throwable cause = 
ExceptionUtils.stripExecutionException(e);
    +                   if (cause instanceof RestHandlerException) {
    +                           throw (RestHandlerException) cause;
    +                   } else {
    +                           throw new RestHandlerException("Could not 
retrieve file blob key future.", HttpResponseStatus.INTERNAL_SERVER_ERROR, e);
    +                   }
    +           }
    +
    +           final CompletableFuture<Void> resultFuture = 
blobKeyFuture.thenAcceptAsync(
    +                   (TransientBlobKey blobKey) -> {
    +                           final File file;
    +                           try {
    +                                   file = 
transientBlobService.getFile(blobKey);
    +                           } catch (IOException e) {
    +                                   throw new CompletionException(new 
FlinkException("Could not retrieve file from transient blob store.", e));
    +                           }
    +
    +                           try {
    +                                   transferFile(
    +                                           ctx,
    +                                           file,
    +                                           httpRequest);
    +                           } catch (FlinkException e) {
    +                                   throw new CompletionException(new 
FlinkException("Could not transfer file to client.", e));
    +                           }
    +                   },
    +                   ctx.executor());
    +
    +           resultFuture.whenComplete(
    +                   (Void ignored, Throwable throwable) -> {
    +                           if (throwable != null) {
    +                                   log.debug("Failed to transfer file from 
TaskExecutor {}.", taskManagerId, throwable);
    +                                   fileBlobKeys.invalidate(taskManagerId);
    --- End diff --
    
    It's not enough to invalidate the cache, you must also send an error back 
to the client, or the connection will not be closed.
    
    This will block forever if the TM is not registered:
    ```
    curl -v 
http://localhost:9067/taskmanagers/daecac46c3f0f13b945fd2bb94438204/log
    ```


> Port TaskManagerLogHandler to new REST endpoint
> -----------------------------------------------
>
>                 Key: FLINK-8503
>                 URL: https://issues.apache.org/jira/browse/FLINK-8503
>             Project: Flink
>          Issue Type: Sub-task
>          Components: REST
>    Affects Versions: 1.5.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>            Priority: Major
>              Labels: flip-6
>             Fix For: 1.5.0
>
>
> In order to serve {{TaskExecutor}} log stdout files, we have to port the 
> {{TaskManagerLogHandler}} to the new REST endpoint.
> In order to properly support serving of files, I propose to introduce an 
> {{AbstractHandler}} which takes a typed request but has not typed response. 
> That way we can easily output the file contents.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to