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

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

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

    https://github.com/apache/flink/pull/3460#discussion_r104407909
  
    --- Diff: 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/AbstractStaticFileServerHandler.java
 ---
    @@ -0,0 +1,350 @@
    +/*
    + * 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.webmonitor.files;
    +
    
+/*****************************************************************************
    + * This code is based on the "HttpStaticFileServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * 
https://github.com/netty/netty/blob/4.0/example/src/main/java/io/netty/example/http/file/HttpStaticFileServerHandler.java
    + 
*****************************************************************************/
    +
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.channel.ChannelFutureListener;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.DefaultFileRegion;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.DefaultFullHttpResponse;
    +import io.netty.handler.codec.http.DefaultHttpResponse;
    +import io.netty.handler.codec.http.FullHttpResponse;
    +import io.netty.handler.codec.http.HttpChunkedInput;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.HttpResponse;
    +import io.netty.handler.codec.http.HttpResponseStatus;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.router.Routed;
    +import io.netty.handler.ssl.SslHandler;
    +import io.netty.handler.stream.ChunkedFile;
    +import io.netty.util.CharsetUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.slf4j.Logger;
    +
    +import java.io.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.RandomAccessFile;
    +import java.net.URI;
    +import java.net.URL;
    +import java.nio.file.Files;
    +import java.text.ParseException;
    +import java.text.SimpleDateFormat;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.GregorianCalendar;
    +import java.util.Locale;
    +import java.util.TimeZone;
    +
    +import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
    +import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
    +import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
    +import static io.netty.handler.codec.http.HttpHeaders.Names.DATE;
    +import static io.netty.handler.codec.http.HttpHeaders.Names.EXPIRES;
    +import static 
io.netty.handler.codec.http.HttpHeaders.Names.IF_MODIFIED_SINCE;
    +import static io.netty.handler.codec.http.HttpHeaders.Names.LAST_MODIFIED;
    +import static 
io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
    +import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
    +import static io.netty.handler.codec.http.HttpResponseStatus.NOT_MODIFIED;
    +import static io.netty.handler.codec.http.HttpResponseStatus.OK;
    +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
    +
    +/**
    + * Simple file server handler that serves requests to web frontend's 
static files, such as
    + * HTML, CSS, or JS files.
    + *
    + * <p>This code is based on the "HttpStaticFileServerHandler" from the 
Netty project's HTTP server
    + * example.</p>
    + */
    +public abstract class AbstractStaticFileServerHandler extends 
SimpleChannelInboundHandler<Routed> {
    +
    +   /** Timezone in which this server answers its "if-modified" requests */
    +   private static final TimeZone GMT_TIMEZONE = 
TimeZone.getTimeZone("GMT");
    +
    +   /** Date format for HTTP */
    +   public static final String HTTP_DATE_FORMAT = "EEE, dd MMM yyyy 
HH:mm:ss zzz";
    +
    +   /** Be default, we allow files to be cached for 5 minutes */
    +   private static final int HTTP_CACHE_SECONDS = 300;
    +
    +   /** The log for all error reporting */
    +   protected final Logger logger;
    +
    +   /** The path in which the static documents are */
    +   private final File rootPath;
    +
    +   public AbstractStaticFileServerHandler(Logger logger, File rootPath) 
throws IOException {
    +           this.logger = Preconditions.checkNotNull(logger);
    +           this.rootPath = 
Preconditions.checkNotNull(rootPath).getCanonicalFile();
    +   }
    +
    +   @Override
    +   public void channelRead0(ChannelHandlerContext ctx, Routed routed) 
throws Exception {
    +           String requestPath = routed.path();
    +
    +           // make sure we request the "index.html" in case there is a 
directory request
    +           if (requestPath.endsWith("/")) {
    +                   requestPath = requestPath + "index.html";
    +           }
    +
    +           requestPath = preProcessRequestPath(requestPath);
    +           
    +           handleRequest(ctx, routed, requestPath);
    +   }
    +
    +   protected void handleRequest(ChannelHandlerContext ctx, Routed routed, 
String requestPath) throws Exception {
    +           respond(ctx, routed.request(), requestPath);
    +   }
    +
    +   protected void respond(ChannelHandlerContext ctx, HttpRequest request, 
String requestPath) throws IOException, ParseException {
    +           // convert to absolute path
    +           final File file = new File(rootPath, requestPath);
    +
    +           if (!file.exists()) {
    +                   // file does not exist. Try to load it with the 
classloader
    +                   ClassLoader cl = 
StaticFileServerHandler.class.getClassLoader();
    +
    +                   requestPath = preProcessFilePath(requestPath);
    +
    +                   try(InputStream resourceStream = 
cl.getResourceAsStream("web" + requestPath)) {
    +                           boolean success = false;
    +                           try {
    +                                   if (resourceStream != null) {
    +                                           URL root = 
cl.getResource("web");
    +                                           URL requested = 
cl.getResource("web" + requestPath);
    +
    +                                           if (root != null && requested 
!= null) {
    +                                                   URI rootURI = new 
URI(root.getPath()).normalize();
    +                                                   URI requestedURI = new 
URI(requested.getPath()).normalize();
    +
    +                                                   // Check that we don't 
load anything from outside of the
    +                                                   // expected scope.
    +                                                   if 
(!rootURI.relativize(requestedURI).equals(requestedURI)) {
    +                                                           
logger.debug("Loading missing file from classloader: {}", requestPath);
    +                                                           // ensure that 
directory to file exists.
    +                                                           
file.getParentFile().mkdirs();
    +                                                           
Files.copy(resourceStream, file.toPath());
    +
    +                                                           success = true;
    +                                                   }
    +                                           }
    +                                   }
    +                           } catch (Throwable t) {
    +                                   logger.error("error while responding", 
t);
    +                           } finally {
    +                                   if (!success) {
    +                                           logger.debug("Unable to load 
requested file {} from classloader", requestPath);
    +                                           
handleUnavailableFile(requestPath);
    +                                           sendError(ctx, NOT_FOUND);
    +                                           return;
    +                                   }
    +                           }
    +                   }
    +           }
    +
    +           if (!file.exists() || file.isHidden() || file.isDirectory() || 
!file.isFile()) {
    +                   sendError(ctx, NOT_FOUND);
    +                   return;
    +           }
    +
    +           if 
(!file.getCanonicalFile().toPath().startsWith(rootPath.toPath())) {
    +                   sendError(ctx, NOT_FOUND);
    +                   return;
    +           }
    +
    +           // cache validation
    +           final String ifModifiedSince = 
request.headers().get(IF_MODIFIED_SINCE);
    +           if (ifModifiedSince != null && !ifModifiedSince.isEmpty()) {
    +                   SimpleDateFormat dateFormatter = new 
SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US);
    +                   Date ifModifiedSinceDate = 
dateFormatter.parse(ifModifiedSince);
    +
    +                   // Only compare up to the second because the datetime 
format we send to the client
    +                   // does not have milliseconds
    +                   long ifModifiedSinceDateSeconds = 
ifModifiedSinceDate.getTime() / 1000;
    +                   long fileLastModifiedSeconds = file.lastModified() / 
1000;
    +                   if (ifModifiedSinceDateSeconds == 
fileLastModifiedSeconds) {
    +                           if (logger.isDebugEnabled()) {
    +                                   logger.debug("Responding 'NOT MODIFIED' 
for file '" + file.getAbsolutePath() + '\'');
    +                           }
    +
    +                           sendNotModified(ctx);
    +                           return;
    +                   }
    +           }
    +
    +           if (logger.isDebugEnabled()) {
    +                   logger.debug("Responding with file '" + 
file.getAbsolutePath() + '\'');
    +           }
    +
    +           // Don't need to close this manually. Netty's DefaultFileRegion 
will take care of it.
    +           final RandomAccessFile raf;
    +           try {
    +                   raf = new RandomAccessFile(file, "r");
    +           }
    +           catch (FileNotFoundException e) {
    +                   sendError(ctx, NOT_FOUND);
    +                   return;
    +           }
    +           long fileLength = raf.length();
    +
    +           HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
    +           setContentTypeHeader(response, file);
    +
    +           // since the log and out files are rapidly changing, we don't 
want to browser to cache them
    +           if (shouldCache(requestPath)) {
    +                   setDateAndCacheHeaders(response, file);
    +           }
    +           if (HttpHeaders.isKeepAlive(request)) {
    +                   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.
    +           ChannelFuture lastContentFuture;
    +           if (ctx.pipeline().get(SslHandler.class) == null) {
    +                   ctx.write(new DefaultFileRegion(raf.getChannel(), 0, 
fileLength), ctx.newProgressivePromise());
    +                   lastContentFuture = 
ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
    +           } else {
    +                   lastContentFuture = ctx.writeAndFlush(new 
HttpChunkedInput(new ChunkedFile(raf, 0, fileLength, 8192)),
    +                           ctx.newProgressivePromise());
    +                   // HttpChunkedInput will write the end marker 
(LastHttpContent) for us.
    +           }
    +
    +           // close the connection, if no keep-alive is needed
    +           if (!HttpHeaders.isKeepAlive(request)) {
    +                   
lastContentFuture.addListener(ChannelFutureListener.CLOSE);
    +           }
    +   }
    +
    +   protected abstract String preProcessRequestPath(String requestPath);
    +
    +   protected abstract boolean shouldCache(String requestPath);
    +
    +   protected String preProcessFilePath(String requestPath) {
    +           return requestPath;
    +   }
    +
    +   protected void handleUnavailableFile(String requestPath) {
    +   }
    +
    +   @Override
    +   public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) 
{
    +           if (ctx.channel().isActive()) {
    +                   logger.error("Caught exception", cause);
    +                   sendError(ctx, INTERNAL_SERVER_ERROR);
    +           }
    +   }
    +
    +   // 
------------------------------------------------------------------------
    +   //  Utilities to encode headers and responses
    +   // 
------------------------------------------------------------------------
    +
    +   public static Date convertTimestampToDate(String timestamp) throws 
ParseException {
    --- End diff --
    
    Unused method


> Create a Flink History Server
> -----------------------------
>
>                 Key: FLINK-1579
>                 URL: https://issues.apache.org/jira/browse/FLINK-1579
>             Project: Flink
>          Issue Type: New Feature
>          Components: Distributed Coordination
>    Affects Versions: 0.9
>            Reporter: Robert Metzger
>            Assignee: Chesnay Schepler
>
> Right now its not possible to analyze the job results for jobs that ran on 
> YARN, because we'll loose the information once the JobManager has stopped.
> Therefore, I propose to implement a "Flink History Server" which serves  the 
> results from these jobs.
> I haven't started thinking about the implementation, but I suspect it 
> involves some JSON files stored in HDFS :)



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to