From issues-return-151040-archive-asf-public=cust-asf.ponee.io@flink.apache.org Mon Feb 5 14:38:05 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id 69CD0180647 for ; Mon, 5 Feb 2018 14:38:05 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 5A0DA160C4B; Mon, 5 Feb 2018 13:38:05 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 77FC0160C46 for ; Mon, 5 Feb 2018 14:38:04 +0100 (CET) Received: (qmail 39719 invoked by uid 500); 5 Feb 2018 13:38:03 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 39709 invoked by uid 99); 5 Feb 2018 13:38:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 05 Feb 2018 13:38:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 359E3C0118 for ; Mon, 5 Feb 2018 13:38:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.311 X-Spam-Level: X-Spam-Status: No, score=-110.311 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id L5x5varMnT4z for ; Mon, 5 Feb 2018 13:38:01 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id F310F5F232 for ; Mon, 5 Feb 2018 13:38:00 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 58844E008D for ; Mon, 5 Feb 2018 13:38:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 14EDA21E82 for ; Mon, 5 Feb 2018 13:38:00 +0000 (UTC) Date: Mon, 5 Feb 2018 13:38:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-8503) Port TaskManagerLogHandler to new REST endpoint MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ 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 extends AbstractHandler { + + private final GatewayRetriever resourceManagerGatewayRetriever; + private final TransientBlobService transientBlobService; + + private final LoadingCache> fileBlobKeys; + + protected AbstractTaskManagerFileHandler( + @Nonnull CompletableFuture localAddressFuture, + @Nonnull GatewayRetriever leaderRetriever, + @Nonnull Time timeout, + @Nonnull Map responseHeaders, + @Nonnull UntypedResponseMessageHeaders untypedResponseMessageHeaders, + @Nonnull GatewayRetriever 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>() { + @Override + public CompletableFuture load(ResourceID resourceId) throws Exception { + return loadTaskManagerFile(resourceId); + } + }); + } + + @Override + protected void respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest handlerRequest, RestfulGateway gateway) throws RestHandlerException { + final ResourceID taskManagerId = handlerRequest.getPathParameter(TaskManagerIdPathParameter.class); + + final CompletableFuture 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 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)