Details

    • Type: New Feature
    • Status: Closed
    • Priority: Major
    • Resolution: Implemented
    • Affects Version/s: 0.9
    • Fix Version/s: 1.3.0
    • Labels:
      None

      Description

      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

        Issue Links

          Activity

          Hide
          StephanEwen Stephan Ewen added a comment -

          How much is there in terms of Hadoop History server, or YARN history server?

          Show
          StephanEwen Stephan Ewen added a comment - How much is there in terms of Hadoop History server, or YARN history server?
          Show
          rmetzger Robert Metzger added a comment - We have to investigate on that: http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/TimelineServer.html
          Hide
          uce Ufuk Celebi added a comment -

          Just to ping this issue again. Another user asked for this.

          Show
          uce Ufuk Celebi added a comment - Just to ping this issue again. Another user asked for this.
          Hide
          eronwright Eron Wright added a comment -

          Something to consider before taking a dependency on the YARN timeline server, Flink on Mesos will suffer a similar situation.

          Show
          eronwright Eron Wright added a comment - Something to consider before taking a dependency on the YARN timeline server, Flink on Mesos will suffer a similar situation.
          Hide
          StephanEwen Stephan Ewen added a comment -

          The good thing in the current design is that all data (after the job is done) is simply JSON objects behind static REST-style URLs.
          All dynamic logic is client-side, in the browser.

          A simple server that the JobManager could push the JSON to (under the URLs) should actually do most of the job.
          That would, however, not cover authentication, and result in separate services for live and historic jobs.

          Show
          StephanEwen Stephan Ewen added a comment - The good thing in the current design is that all data (after the job is done) is simply JSON objects behind static REST-style URLs. All dynamic logic is client-side, in the browser. A simple server that the JobManager could push the JSON to (under the URLs) should actually do most of the job. That would, however, not cover authentication, and result in separate services for live and historic jobs.
          Hide
          StephanEwen Stephan Ewen added a comment -

          Here is a brief thread talking about that in the context of the YARN timeline server.
          The thread died off, because the solution with the YARN timeline server seemed not really simply and lightweight.

          http://mail-archives.apache.org/mod_mbox/hadoop-yarn-dev/201512.mbox/%3CD8A04025-2BF9-4FB2-8063-E93CDE7D1298@hortonworks.com%3E

          Show
          StephanEwen Stephan Ewen added a comment - Here is a brief thread talking about that in the context of the YARN timeline server. The thread died off, because the solution with the YARN timeline server seemed not really simply and lightweight. http://mail-archives.apache.org/mod_mbox/hadoop-yarn-dev/201512.mbox/%3CD8A04025-2BF9-4FB2-8063-E93CDE7D1298@hortonworks.com%3E
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user zentol opened a pull request:

          https://github.com/apache/flink/pull/3286

          FLINK-1579 [WIP] Implement Standalone HistoryServer

          This PR is a work-in-progress view over a standalone History Server (HS).

          JobManagers may send completed jobs to the HistoryServer for them to be archived. Upon receiving an ArchivedExecutionGraph the HS pre-computes all possible REST requests and writes them into files. The files are arranged in a directory structure corresponding to the REST API.

          The HS can be started by calling `./bin/historyserver.sh start`, similar to the JM/TM. Various config options exist for the HS that mostly mirror the web-ui/RPC options of the JM.

          The HS uses a slightly modified web-ui; basically it only shows the "Completed Jobs" page. To not duplicate the everything I've added 2 files, `index2.jade` and `index2.coffee`, to the build script. The resulting `index2.html` file will be loaded when the browser requests the `index.html`.

          In order to re-use the JSON generation code that previously was contained in various handlers a giant utility `JsonUtils` class was created. This class now contains a variety of static methods that generate the JSON responses. As a result most handlers were reduced to one-liners, bar some sanity-checks.

          In regard to tests we verify that the HS creates all expected files upon receiving an ExecutionGraph.
          Furthermore, the newly created JsonUtils are mostly tested (the new checkpoint stats aren't tested); so we have tests for the REST responses now, which is neat.

          I'm not opening a proper PR yet as i have to go through all changes once again in detail, but it works (locally and on a cluster) so i wanted people to try it out and get some feedback.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/zentol/flink 1579_history_server_b

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3286.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3286


          commit 0fdfeec0c86cba60d271d38cfbce7e4ae759b700
          Author: zentol <chesnay@apache.org>
          Date: 2016-10-17T10:55:19Z

          Add AccessExecutionVertex#getPriorExecutions()

          commit 18c4cc6a9e8f3c9b772bcfe8f866e07d2f7304ce
          Author: zentol <chesnay@apache.org>
          Date: 2017-01-30T15:06:13Z

          FLINK-5645 EG stuff

          commit fcc4def5251086d4e37901c58bc47785e1d90788
          Author: zentol <chesnay@apache.org>
          Date: 2017-01-24T09:13:24Z

          FLINK-1579 Implement History Server - Frontend

          commit 2cc6b736c0c5c78903b85f9c1a9ccde8c3ee70b8
          Author: zentol <chesnay@apache.org>
          Date: 2016-10-21T12:29:30Z

          FLINK-1579 Implement History Server - Backend

          commit 0047ae53b9f2f79eee9ec7e76195559b32dbeb20
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-08T08:58:01Z

          FLINK-1579 Implement History Server - Backend - Tests

          commit 730548a7d88c56a2cde235e3d7d92dbf676611b7
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-08T08:58:22Z

          Use JsonUtils in handlers

          commit adcc161e46f817e80301d1fb885cdef4a8679a71
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-08T10:23:56Z

          Rebuild web-frontend

          commit 3227fc2a12e8aeaaf111339833123da708ccea70
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-08T10:24:14Z

          tmp streaming example with checkpointing


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user zentol opened a pull request: https://github.com/apache/flink/pull/3286 FLINK-1579 [WIP] Implement Standalone HistoryServer This PR is a work-in-progress view over a standalone History Server (HS). JobManagers may send completed jobs to the HistoryServer for them to be archived. Upon receiving an ArchivedExecutionGraph the HS pre-computes all possible REST requests and writes them into files. The files are arranged in a directory structure corresponding to the REST API. The HS can be started by calling `./bin/historyserver.sh start`, similar to the JM/TM. Various config options exist for the HS that mostly mirror the web-ui/RPC options of the JM. The HS uses a slightly modified web-ui; basically it only shows the "Completed Jobs" page. To not duplicate the everything I've added 2 files, `index2.jade` and `index2.coffee`, to the build script. The resulting `index2.html` file will be loaded when the browser requests the `index.html`. In order to re-use the JSON generation code that previously was contained in various handlers a giant utility `JsonUtils` class was created. This class now contains a variety of static methods that generate the JSON responses. As a result most handlers were reduced to one-liners, bar some sanity-checks. In regard to tests we verify that the HS creates all expected files upon receiving an ExecutionGraph. Furthermore, the newly created JsonUtils are mostly tested (the new checkpoint stats aren't tested); so we have tests for the REST responses now, which is neat. I'm not opening a proper PR yet as i have to go through all changes once again in detail, but it works (locally and on a cluster) so i wanted people to try it out and get some feedback. You can merge this pull request into a Git repository by running: $ git pull https://github.com/zentol/flink 1579_history_server_b Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3286.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3286 commit 0fdfeec0c86cba60d271d38cfbce7e4ae759b700 Author: zentol <chesnay@apache.org> Date: 2016-10-17T10:55:19Z Add AccessExecutionVertex#getPriorExecutions() commit 18c4cc6a9e8f3c9b772bcfe8f866e07d2f7304ce Author: zentol <chesnay@apache.org> Date: 2017-01-30T15:06:13Z FLINK-5645 EG stuff commit fcc4def5251086d4e37901c58bc47785e1d90788 Author: zentol <chesnay@apache.org> Date: 2017-01-24T09:13:24Z FLINK-1579 Implement History Server - Frontend commit 2cc6b736c0c5c78903b85f9c1a9ccde8c3ee70b8 Author: zentol <chesnay@apache.org> Date: 2016-10-21T12:29:30Z FLINK-1579 Implement History Server - Backend commit 0047ae53b9f2f79eee9ec7e76195559b32dbeb20 Author: zentol <chesnay@apache.org> Date: 2017-02-08T08:58:01Z FLINK-1579 Implement History Server - Backend - Tests commit 730548a7d88c56a2cde235e3d7d92dbf676611b7 Author: zentol <chesnay@apache.org> Date: 2017-02-08T08:58:22Z Use JsonUtils in handlers commit adcc161e46f817e80301d1fb885cdef4a8679a71 Author: zentol <chesnay@apache.org> Date: 2017-02-08T10:23:56Z Rebuild web-frontend commit 3227fc2a12e8aeaaf111339833123da708ccea70 Author: zentol <chesnay@apache.org> Date: 2017-02-08T10:24:14Z tmp streaming example with checkpointing
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on the issue:

          https://github.com/apache/flink/pull/3286

          Thanks for this PR. I think it's super cool (and long overdue) to finally have this. I think the general approach is sound and has good things going for it.

          But I see two problems with maintaining the current implementation in the long run:

          • Changing a URL in the web dashboard requires that we also change it in the history server file generation. It's easy to accidentally break the history server and it's cumbersome to write tests when we extend it.
          • We have all JSON generation in the JsonUtil. That results in a giant class (as you say) that is really tightly coupled to the handlers but resides in a different place.

          I think we can word around this by moving the archival logic to the handlers. I think it's fair that handlers that want to take part in historical archiving implement their archival logic on their own. The handlers would need to return their path and also their JSON. That way we keep all the handler logic in one place for both the regular frontend and history server. A "historic execution graph" might be a simple collection of `path, JSON` entries that the history server persists on receive.

          Another minor thing is that the final status of the operators is missing on the history server. I don't know whether that is a front end issue or something is missing in the JSON.

          I think it's best to close this PR for now and think about how we want to extend this a bit for better maintainability.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on the issue: https://github.com/apache/flink/pull/3286 Thanks for this PR. I think it's super cool (and long overdue) to finally have this. I think the general approach is sound and has good things going for it. But I see two problems with maintaining the current implementation in the long run: Changing a URL in the web dashboard requires that we also change it in the history server file generation. It's easy to accidentally break the history server and it's cumbersome to write tests when we extend it. We have all JSON generation in the JsonUtil. That results in a giant class (as you say) that is really tightly coupled to the handlers but resides in a different place. I think we can word around this by moving the archival logic to the handlers. I think it's fair that handlers that want to take part in historical archiving implement their archival logic on their own. The handlers would need to return their path and also their JSON. That way we keep all the handler logic in one place for both the regular frontend and history server. A "historic execution graph" might be a simple collection of `path, JSON` entries that the history server persists on receive. Another minor thing is that the final status of the operators is missing on the history server. I don't know whether that is a front end issue or something is missing in the JSON. I think it's best to close this PR for now and think about how we want to extend this a bit for better maintainability.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol closed the pull request at:

          https://github.com/apache/flink/pull/3286

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol closed the pull request at: https://github.com/apache/flink/pull/3286
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user zentol opened a pull request:

          https://github.com/apache/flink/pull/3460

          FLINK-1579 Implement History Server

          This PR adds a slightly unpolished HistoryServer implementation. It is missing tests and some documentation, but is working.

          This PR builds on top of #3377.

          The basic idea is as follows:

          The ```MemoryArchivist```, upon receiving an ```ExecutionGraph```, writes a set of json files into a directory structure resembling the REST API using the features introduced in FLINK-5870, FLINK-5852 and FLINK-5941. The target location is configurable using ```job-manager.archive.dir```. Each job resides in it's own directory, using the job ID as the directory name. As such, each archive is consistent on it's own and multiple jobmanagers may use the same archive dir.

          The ```HistoryServer``` polls certain directories, configured via ```historyserver.archive.dirs```, in regular intervals, configured via ```historyserver.refresh-interval```, for new job archives. If a new archive is found it is downloaded and integrated into a cache of job archives in the local file system, configurable using ```historyserver.web.dir```. These files are served to a slightly modified WebFrontend using the ```HistoryServerStaticFileServerHandler```.

          In the end the HistoryServer is little more than an aggregator and archive viewer.

          None of the directory configuration options have defaults; as it stands the entire feature is opt-in.

          Should a file that the WebFrontend requests be missing a separate fetch routine kicks in which attempts to fetch the missing file. This is primarily aimed at eventually-consistent file-systems.

          The HistoryServer is started using the new historyserver.sh script, which works similarly to job- or taskmanager scripts: ```./bin/historyserver.sh [start|stop]```

          2 bigger refactorings were made to existing code to increase the amount of shared code:

          • the netty setup in the WebRuntimeMonitor was moved into a separate NettySetup class which the HistoryServer can use as well
          • an AbstractStaticFileServerHandler was added which the (HistoryServer)StaticFileServerHandler extend

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/zentol/flink 1579_history_server_pr

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3460.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3460


          commit 61a07456f151ac8f5418ac66629751e1a83ada3a
          Author: zentol <chesnay@apache.org>
          Date: 2017-01-24T09:13:24Z

          FLINK-1579 Implement History Server - Frontend

          commit e6316e544fea160f7d050dd1b087301a83345d31
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-21T11:36:17Z

          FLINK-5645 Store accumulators/metrics for canceled/failed tasks

          commit 84fd2746b09ce41c2d9bd5be7f6e8a8cc1a3291d
          Author: zentol <chesnay@apache.org>
          Date: 2017-03-02T12:31:56Z

          Refactor netty setup into separate class

          commit 81d7e6b92fe69326d6edf6b63f3f9c95f5ebd0ef
          Author: zentol <chesnay@apache.org>
          Date: 2017-02-22T14:47:07Z

          FLINK-1579 Implement History Server - Backend

          commit 8d1e8c59690ea97be4bbaf1a011c8ec4a68f5892
          Author: zentol <chesnay@apache.org>
          Date: 2017-03-02T11:09:36Z

          Rebuild frontend


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user zentol opened a pull request: https://github.com/apache/flink/pull/3460 FLINK-1579 Implement History Server This PR adds a slightly unpolished HistoryServer implementation. It is missing tests and some documentation, but is working. This PR builds on top of #3377. The basic idea is as follows: The ```MemoryArchivist```, upon receiving an ```ExecutionGraph```, writes a set of json files into a directory structure resembling the REST API using the features introduced in FLINK-5870 , FLINK-5852 and FLINK-5941 . The target location is configurable using ```job-manager.archive.dir```. Each job resides in it's own directory, using the job ID as the directory name. As such, each archive is consistent on it's own and multiple jobmanagers may use the same archive dir. The ```HistoryServer``` polls certain directories, configured via ```historyserver.archive.dirs```, in regular intervals, configured via ```historyserver.refresh-interval```, for new job archives. If a new archive is found it is downloaded and integrated into a cache of job archives in the local file system, configurable using ```historyserver.web.dir```. These files are served to a slightly modified WebFrontend using the ```HistoryServerStaticFileServerHandler```. In the end the HistoryServer is little more than an aggregator and archive viewer. None of the directory configuration options have defaults; as it stands the entire feature is opt-in. Should a file that the WebFrontend requests be missing a separate fetch routine kicks in which attempts to fetch the missing file. This is primarily aimed at eventually-consistent file-systems. The HistoryServer is started using the new historyserver.sh script, which works similarly to job- or taskmanager scripts: ```./bin/historyserver.sh [start|stop] ``` 2 bigger refactorings were made to existing code to increase the amount of shared code: the netty setup in the WebRuntimeMonitor was moved into a separate NettySetup class which the HistoryServer can use as well an AbstractStaticFileServerHandler was added which the (HistoryServer)StaticFileServerHandler extend You can merge this pull request into a Git repository by running: $ git pull https://github.com/zentol/flink 1579_history_server_pr Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3460.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3460 commit 61a07456f151ac8f5418ac66629751e1a83ada3a Author: zentol <chesnay@apache.org> Date: 2017-01-24T09:13:24Z FLINK-1579 Implement History Server - Frontend commit e6316e544fea160f7d050dd1b087301a83345d31 Author: zentol <chesnay@apache.org> Date: 2017-02-21T11:36:17Z FLINK-5645 Store accumulators/metrics for canceled/failed tasks commit 84fd2746b09ce41c2d9bd5be7f6e8a8cc1a3291d Author: zentol <chesnay@apache.org> Date: 2017-03-02T12:31:56Z Refactor netty setup into separate class commit 81d7e6b92fe69326d6edf6b63f3f9c95f5ebd0ef Author: zentol <chesnay@apache.org> Date: 2017-02-22T14:47:07Z FLINK-1579 Implement History Server - Backend commit 8d1e8c59690ea97be4bbaf1a011c8ec4a68f5892 Author: zentol <chesnay@apache.org> Date: 2017-03-02T11:09:36Z Rebuild frontend
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104407277

          — 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> {
          — End diff –

          Could you please update the comments to reflect your refactorings for the "request life cycle"? BTW You can remove the closing `</p>` tag, too (I know it's copied from `master`).

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104407277 — 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> { — End diff – Could you please update the comments to reflect your refactorings for the "request life cycle"? BTW You can remove the closing `</p>` tag, too (I know it's copied from `master`).
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104436179

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala —
          @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int)
          }
          }

          + private def archiveJsonFiles(graph: ArchivedExecutionGraph) {
          + future {
          + val rootPath = new Path(flinkConfiguration.getString(
          + JobManagerOptions.ARCHIVE_DIR))
          + val fs = rootPath.getFileSystem
          + val tmpArchivePath = new Path(rootPath, s"tmp_$

          {graph.getJobID.toString}")
          + for (archiver <- WebMonitorUtils.getArchivers) {
          + try {
          + for (archive <- archiver.archiveJsonWithPath(graph).asScala) {
          + val targetPath =
          + new Path(tmpArchivePath, s"${archive.getPath}.json")
          + val out = fs.create(targetPath, false)
          + out.write(archive.getJson.getBytes(StandardCharsets.UTF_8))
          + out.close()
          + }
          + } catch {
          + case ioe: IOException => { + log.error("Failed to archive job details.", ioe) + }
          + }
          + }
          + if (!fs.rename(tmpArchivePath, new Path(rootPath, s"${graph.getJobID.toString}

          "))) {
          — End diff –

          Do we have lingering files if the rename does not work as expected?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104436179 — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala — @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int) } } + private def archiveJsonFiles(graph: ArchivedExecutionGraph) { + future { + val rootPath = new Path(flinkConfiguration.getString( + JobManagerOptions.ARCHIVE_DIR)) + val fs = rootPath.getFileSystem + val tmpArchivePath = new Path(rootPath, s"tmp_$ {graph.getJobID.toString}") + for (archiver <- WebMonitorUtils.getArchivers) { + try { + for (archive <- archiver.archiveJsonWithPath(graph).asScala) { + val targetPath = + new Path(tmpArchivePath, s"${archive.getPath}.json") + val out = fs.create(targetPath, false) + out.write(archive.getJson.getBytes(StandardCharsets.UTF_8)) + out.close() + } + } catch { + case ioe: IOException => { + log.error("Failed to archive job details.", ioe) + } + } + } + if (!fs.rename(tmpArchivePath, new Path(rootPath, s"${graph.getJobID.toString} "))) { — End diff – Do we have lingering files if the rename does not work as expected?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104435195

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/NettySetup.java —
          @@ -0,0 +1,132 @@
          +/*
          + * 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.utils;
          +
          +import io.netty.bootstrap.ServerBootstrap;
          +import io.netty.channel.Channel;
          +import io.netty.channel.ChannelFuture;
          +import io.netty.channel.ChannelInitializer;
          +import io.netty.channel.nio.NioEventLoopGroup;
          +import io.netty.channel.socket.SocketChannel;
          +import io.netty.channel.socket.nio.NioServerSocketChannel;
          +import io.netty.handler.codec.http.HttpServerCodec;
          +import io.netty.handler.codec.http.router.Handler;
          +import io.netty.handler.codec.http.router.Router;
          +import io.netty.handler.ssl.SslHandler;
          +import io.netty.handler.stream.ChunkedWriteHandler;
          +import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
          +import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
          +import org.slf4j.Logger;
          +
          +import javax.net.ssl.SSLContext;
          +import javax.net.ssl.SSLEngine;
          +import java.io.File;
          +import java.net.InetSocketAddress;
          +
          +public class NettySetup {
          + private final Router router;
          + private final Logger LOG;
          — End diff –

          This is against common Java conventions. Let's go with the usual `log` or `logger`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104435195 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/NettySetup.java — @@ -0,0 +1,132 @@ +/* + * 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.utils; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.codec.http.router.Handler; +import io.netty.handler.codec.http.router.Router; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; +import org.apache.flink.runtime.webmonitor.HttpRequestHandler; +import org.apache.flink.runtime.webmonitor.PipelineErrorHandler; +import org.slf4j.Logger; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import java.io.File; +import java.net.InetSocketAddress; + +public class NettySetup { + private final Router router; + private final Logger LOG; — End diff – This is against common Java conventions. Let's go with the usual `log` or `logger`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104401759

          — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java —
          @@ -0,0 +1,54 @@
          +/*
          + * 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.configuration;
          +
          +import static org.apache.flink.configuration.ConfigOptions.key;
          +
          +public class HistoryServerOptions {
          — End diff –

          Missing stability annotation, I think this should be `@PublicEvolving`

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104401759 — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java — @@ -0,0 +1,54 @@ +/* + * 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.configuration; + +import static org.apache.flink.configuration.ConfigOptions.key; + +public class HistoryServerOptions { — End diff – Missing stability annotation, I think this should be `@PublicEvolving`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104435595

          — Diff: flink-runtime-web/web-dashboard/app/scripts/index_hs.coffee —
          @@ -0,0 +1,193 @@
          +#
          +# 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.
          +#
          +
          +angular.module('flinkApp', ['ui.router', 'angularMoment', 'dndLists'])
          +
          +# --------------------------------------
          +
          +.run ($rootScope) ->
          + $rootScope.sidebarVisible = false
          + $rootScope.showSidebar = ->
          + $rootScope.sidebarVisible = !$rootScope.sidebarVisible
          + $rootScope.sidebarClass = 'force-show'
          +
          +# --------------------------------------
          +
          +.value 'flinkConfig', {
          + jobServer: ''
          +# jobServer: 'http://localhost:8081/'
          + "refresh-interval": 10000
          — End diff –

          The default UI refresh interval is larger than the history server refresh interval. I think both should be the same. I would go with 10s for both.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104435595 — Diff: flink-runtime-web/web-dashboard/app/scripts/index_hs.coffee — @@ -0,0 +1,193 @@ +# +# 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. +# + +angular.module('flinkApp', ['ui.router', 'angularMoment', 'dndLists'] ) + +# -------------------------------------- + +.run ($rootScope) -> + $rootScope.sidebarVisible = false + $rootScope.showSidebar = -> + $rootScope.sidebarVisible = !$rootScope.sidebarVisible + $rootScope.sidebarClass = 'force-show' + +# -------------------------------------- + +.value 'flinkConfig', { + jobServer: '' +# jobServer: 'http://localhost:8081/' + "refresh-interval": 10000 — End diff – The default UI refresh interval is larger than the history server refresh interval. I think both should be the same. I would go with 10s for both.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104401622

          — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java —
          @@ -0,0 +1,54 @@
          +/*
          + * 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.configuration;
          +
          +import static org.apache.flink.configuration.ConfigOptions.key;
          +
          +public class HistoryServerOptions {
          — End diff –

          Can you please add a comment to this class and each config option. I know, some are obvious, but other are not.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104401622 — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java — @@ -0,0 +1,54 @@ +/* + * 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.configuration; + +import static org.apache.flink.configuration.ConfigOptions.key; + +public class HistoryServerOptions { — End diff – Can you please add a comment to this class and each config option. I know, some are obvious, but other are not.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104432749

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          — End diff –

          I would move the file fetch update logic to a separate outer class. The history server right now is very slim and essentially only a static file handler.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104432749 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { — End diff – I would move the file fetch update logic to a separate outer class. The history server right now is very slim and essentially only a static file handler.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104433287

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          — End diff –

          This is only relevant for the history server static file handler. I would move it there.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104433287 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { — End diff – This is only relevant for the history server static file handler. I would move it there.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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

          Show
          githubbot ASF GitHub Bot added a comment - 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
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104433910

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          + private final Path webDir;
          + private final Map<String, Path> cachedArchives;
          +
          + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives)

          { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + }

          +
          + public void fetchFileForRestRequest(String requestPath) throws IOException {
          + /**
          + * Here we extract the job ID from the request path. We are either missing a file that resides
          + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will
          + * return -1 so we manually set 'to' to the index before '.json'.
          + */
          + int from = 6;
          + int to = requestPath.indexOf('/', 7);
          + if (to == -1)

          { + to = requestPath.length() - 5; + }

          + String jobID = requestPath.substring(from, to);
          + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath);
          + try

          { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + }

          catch (Exception e) {
          + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e);
          + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID);
          + }
          + }
          + }
          +
          + // =================================================================================================================
          + // File generation
          + // =================================================================================================================
          + private static FileWriter createOrGetFile(File folder, String name) throws IOException {
          + File file = new File(folder, name + ".json");
          + if (!file.exists())

          { + Files.createFile(file.toPath()); + }

          + FileWriter fr = new FileWriter(file);
          + return fr;
          + }
          +
          + private void createDashboardConfigFile() throws IOException {
          — End diff –

          Does anything speak against using the actual handler?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104433910 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { + private final Path webDir; + private final Map<String, Path> cachedArchives; + + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives) { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + } + + public void fetchFileForRestRequest(String requestPath) throws IOException { + /** + * Here we extract the job ID from the request path. We are either missing a file that resides + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will + * return -1 so we manually set 'to' to the index before '.json'. + */ + int from = 6; + int to = requestPath.indexOf('/', 7); + if (to == -1) { + to = requestPath.length() - 5; + } + String jobID = requestPath.substring(from, to); + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath); + try { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + } catch (Exception e) { + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e); + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID); + } + } + } + + // ================================================================================================================= + // File generation + // ================================================================================================================= + private static FileWriter createOrGetFile(File folder, String name) throws IOException { + File file = new File(folder, name + ".json"); + if (!file.exists()) { + Files.createFile(file.toPath()); + } + FileWriter fr = new FileWriter(file); + return fr; + } + + private void createDashboardConfigFile() throws IOException { — End diff – Does anything speak against using the actual handler?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104409758

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java —
          @@ -0,0 +1,82 @@
          +/*
          + * 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.history;
          +
          +import io.netty.channel.ChannelHandler;
          +import org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.IOException;
          +
          +@ChannelHandler.Sharable
          +public class HistoryServerStaticFileServerHandler extends AbstractStaticFileServerHandler {
          — End diff –

          Do you have a unit test for this? With the new refactoring, it should be possible to guard the behaviour with a very specific unit test.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104409758 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java — @@ -0,0 +1,82 @@ +/* + * 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.history; + +import io.netty.channel.ChannelHandler; +import org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +@ChannelHandler.Sharable +public class HistoryServerStaticFileServerHandler extends AbstractStaticFileServerHandler { — End diff – Do you have a unit test for this? With the new refactoring, it should be possible to guard the behaviour with a very specific unit test.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104436093

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala —
          @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int)
          }
          }

          + private def archiveJsonFiles(graph: ArchivedExecutionGraph) {
          + future {
          + val rootPath = new Path(flinkConfiguration.getString(
          + JobManagerOptions.ARCHIVE_DIR))
          + val fs = rootPath.getFileSystem
          + val tmpArchivePath = new Path(rootPath, s"tmp_$

          {graph.getJobID.toString}

          ")
          + for (archiver <- WebMonitorUtils.getArchivers) {
          + try {
          + for (archive <- archiver.archiveJsonWithPath(graph).asScala) {
          + val targetPath =
          + new Path(tmpArchivePath, s"$

          {archive.getPath}

          .json")
          + val out = fs.create(targetPath, false)
          — End diff –

          Can we use try with resources here?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104436093 — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala — @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int) } } + private def archiveJsonFiles(graph: ArchivedExecutionGraph) { + future { + val rootPath = new Path(flinkConfiguration.getString( + JobManagerOptions.ARCHIVE_DIR)) + val fs = rootPath.getFileSystem + val tmpArchivePath = new Path(rootPath, s"tmp_$ {graph.getJobID.toString} ") + for (archiver <- WebMonitorUtils.getArchivers) { + try { + for (archive <- archiver.archiveJsonWithPath(graph).asScala) { + val targetPath = + new Path(tmpArchivePath, s"$ {archive.getPath} .json") + val out = fs.create(targetPath, false) — End diff – Can we use try with resources here?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104435068

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/NettySetup.java —
          @@ -0,0 +1,132 @@
          +/*
          + * 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.utils;
          +
          +import io.netty.bootstrap.ServerBootstrap;
          +import io.netty.channel.Channel;
          +import io.netty.channel.ChannelFuture;
          +import io.netty.channel.ChannelInitializer;
          +import io.netty.channel.nio.NioEventLoopGroup;
          +import io.netty.channel.socket.SocketChannel;
          +import io.netty.channel.socket.nio.NioServerSocketChannel;
          +import io.netty.handler.codec.http.HttpServerCodec;
          +import io.netty.handler.codec.http.router.Handler;
          +import io.netty.handler.codec.http.router.Router;
          +import io.netty.handler.ssl.SslHandler;
          +import io.netty.handler.stream.ChunkedWriteHandler;
          +import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
          +import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
          +import org.slf4j.Logger;
          +
          +import javax.net.ssl.SSLContext;
          +import javax.net.ssl.SSLEngine;
          +import java.io.File;
          +import java.net.InetSocketAddress;
          +
          +public class NettySetup {
          — End diff –

          Again missing comments.

          Also, the name is too generic in my opinion. I know the package gives the context, but still. What do you think about something like "WebFrontendBootstrap"?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104435068 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/NettySetup.java — @@ -0,0 +1,132 @@ +/* + * 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.utils; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.codec.http.router.Handler; +import io.netty.handler.codec.http.router.Router; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; +import org.apache.flink.runtime.webmonitor.HttpRequestHandler; +import org.apache.flink.runtime.webmonitor.PipelineErrorHandler; +import org.slf4j.Logger; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import java.io.File; +import java.net.InetSocketAddress; + +public class NettySetup { — End diff – Again missing comments. Also, the name is too generic in my opinion. I know the package gives the context, but still. What do you think about something like "WebFrontendBootstrap"?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104408644

          — 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 {
          — End diff –

          Could rename this to `respondWithFile` or something? With the new refactorings, it can become unclear what to overwrite etc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104408644 — 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 { — End diff – Could rename this to `respondWithFile` or something? With the new refactorings, it can become unclear what to overwrite etc.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104434129

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          + private final Path webDir;
          + private final Map<String, Path> cachedArchives;
          +
          + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives)

          { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + }

          +
          + public void fetchFileForRestRequest(String requestPath) throws IOException {
          + /**
          + * Here we extract the job ID from the request path. We are either missing a file that resides
          + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will
          + * return -1 so we manually set 'to' to the index before '.json'.
          + */
          + int from = 6;
          + int to = requestPath.indexOf('/', 7);
          + if (to == -1)

          { + to = requestPath.length() - 5; + }

          + String jobID = requestPath.substring(from, to);
          + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath);
          + try

          { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + }

          catch (Exception e) {
          + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e);
          + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID);
          + }
          + }
          + }
          +
          + // =================================================================================================================
          + // File generation
          + // =================================================================================================================
          + private static FileWriter createOrGetFile(File folder, String name) throws IOException {
          + File file = new File(folder, name + ".json");
          + if (!file.exists())

          { + Files.createFile(file.toPath()); + }

          + FileWriter fr = new FileWriter(file);
          + return fr;
          + }
          +
          + private void createDashboardConfigFile() throws IOException {
          + try (FileWriter fw = createOrGetFile(webDir, "config"))

          { + fw.write(DashboardConfigHandler.createConfigJson(webRefreshIntervalMillis)); + fw.flush(); + }

          catch (IOException ioe)

          { + LOG.error("Failed to write config file."); + throw ioe; + }

          + }
          +
          + /**
          + * This method replicates the JSON response that would be given by the

          {@link CurrentJobsOverviewHandler}

          when
          + * listing both running and finished jobs.
          + *
          + * Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on
          + * their own however the list of finished jobs only contains a single job.
          + *
          + * For the display in the HistoryServer WebFrontend we have to combine these overviews.
          + */
          + private static void updateJobOverview(File webDir) {
          — End diff –

          Does anything speak against using an actual handler that is updated with the counts when new jobs are fetched? I think that could be simpler than re-reading the file and re-wrting it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104434129 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { + private final Path webDir; + private final Map<String, Path> cachedArchives; + + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives) { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + } + + public void fetchFileForRestRequest(String requestPath) throws IOException { + /** + * Here we extract the job ID from the request path. We are either missing a file that resides + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will + * return -1 so we manually set 'to' to the index before '.json'. + */ + int from = 6; + int to = requestPath.indexOf('/', 7); + if (to == -1) { + to = requestPath.length() - 5; + } + String jobID = requestPath.substring(from, to); + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath); + try { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + } catch (Exception e) { + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e); + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID); + } + } + } + + // ================================================================================================================= + // File generation + // ================================================================================================================= + private static FileWriter createOrGetFile(File folder, String name) throws IOException { + File file = new File(folder, name + ".json"); + if (!file.exists()) { + Files.createFile(file.toPath()); + } + FileWriter fr = new FileWriter(file); + return fr; + } + + private void createDashboardConfigFile() throws IOException { + try (FileWriter fw = createOrGetFile(webDir, "config")) { + fw.write(DashboardConfigHandler.createConfigJson(webRefreshIntervalMillis)); + fw.flush(); + } catch (IOException ioe) { + LOG.error("Failed to write config file."); + throw ioe; + } + } + + /** + * This method replicates the JSON response that would be given by the {@link CurrentJobsOverviewHandler} when + * listing both running and finished jobs. + * + * Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on + * their own however the list of finished jobs only contains a single job. + * + * For the display in the HistoryServer WebFrontend we have to combine these overviews. + */ + private static void updateJobOverview(File webDir) { — End diff – Does anything speak against using an actual handler that is updated with the counts when new jobs are fetched? I think that could be simpler than re-reading the file and re-wrting it.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104435301

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/NettySetup.java —
          @@ -0,0 +1,132 @@
          +/*
          + * 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.utils;
          +
          +import io.netty.bootstrap.ServerBootstrap;
          +import io.netty.channel.Channel;
          +import io.netty.channel.ChannelFuture;
          +import io.netty.channel.ChannelInitializer;
          +import io.netty.channel.nio.NioEventLoopGroup;
          +import io.netty.channel.socket.SocketChannel;
          +import io.netty.channel.socket.nio.NioServerSocketChannel;
          +import io.netty.handler.codec.http.HttpServerCodec;
          +import io.netty.handler.codec.http.router.Handler;
          +import io.netty.handler.codec.http.router.Router;
          +import io.netty.handler.ssl.SslHandler;
          +import io.netty.handler.stream.ChunkedWriteHandler;
          +import org.apache.flink.runtime.webmonitor.HttpRequestHandler;
          +import org.apache.flink.runtime.webmonitor.PipelineErrorHandler;
          +import org.slf4j.Logger;
          +
          +import javax.net.ssl.SSLContext;
          +import javax.net.ssl.SSLEngine;
          +import java.io.File;
          +import java.net.InetSocketAddress;
          +
          +public class NettySetup {
          + private final Router router;
          + private final Logger LOG;
          + private final File uploadDir;
          + private final SSLContext serverSSLContext;
          + private final ServerBootstrap bootstrap;
          + private final Channel serverChannel;
          +
          + public NettySetup(Router router, Logger log, File directory, SSLContext sslContext, String configuredAddress, int configuredPort) throws InterruptedException {
          + this.router = router;
          — End diff –

          Add checks for not null?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104435301 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/utils/NettySetup.java — @@ -0,0 +1,132 @@ +/* + * 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.utils; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.codec.http.router.Handler; +import io.netty.handler.codec.http.router.Router; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; +import org.apache.flink.runtime.webmonitor.HttpRequestHandler; +import org.apache.flink.runtime.webmonitor.PipelineErrorHandler; +import org.slf4j.Logger; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import java.io.File; +import java.net.InetSocketAddress; + +public class NettySetup { + private final Router router; + private final Logger LOG; + private final File uploadDir; + private final SSLContext serverSSLContext; + private final ServerBootstrap bootstrap; + private final Channel serverChannel; + + public NettySetup(Router router, Logger log, File directory, SSLContext sslContext, String configuredAddress, int configuredPort) throws InterruptedException { + this.router = router; — End diff – Add checks for not null?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104407696

          — 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);
          — End diff –

          Please add comments for all `abstract` methods introduced making clear when they are called and what the returned values indicate etc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104407696 — 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); — End diff – Please add comments for all `abstract` methods introduced making clear when they are called and what the returned values indicate etc.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104432833

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          — End diff –

          The refresh dirs are static but we check them on every periodic invocation

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104432833 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { — End diff – The refresh dirs are static but we check them on every periodic invocation
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104431499

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java —
          @@ -205,215 +125,18 @@ public void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Except
          }
          }

          • /**
          • * Response when running with leading JobManager.
          • */
          • private void respondAsLeader(ChannelHandlerContext ctx, HttpRequest request, String requestPath)
          • throws IOException, ParseException, URISyntaxException {
            -
          • // 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();
            -
          • 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);
          • 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 (!(requestPath.contains("log") || requestPath.contains("out"))) { - 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);
            + @Override
            + public String preProcessRequestPath(String requestPath) {
            + // in case the files being accessed are logs or stdout files, find appropriate paths.
            + if (requestPath.equals("/jobmanager/log") || requestPath.equals("/jobmanager/stdout")) { + return ""; }

            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); + return 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
          • // ------------------------------------------------------------------------
            -
          • /**
          • * Writes a simple error response message.
          • *
          • * @param ctx The channel context to write the response to.
          • * @param status The response status.
          • */
          • private static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { - FullHttpResponse response = new DefaultFullHttpResponse( - HTTP_1_1, status, Unpooled.copiedBuffer("Failure: " + status + "\r\n", CharsetUtil.UTF_8)); - response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); - - // close the connection as soon as the error message is sent. - ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); - }

            -

          • /**
          • * Send the "304 Not Modified" response. This response can be used when the
          • * file timestamp is the same as what the browser is sending up.
          • *
          • * @param ctx The channel context to write the response to.
          • */
          • private static void sendNotModified(ChannelHandlerContext ctx) { - FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, NOT_MODIFIED); - setDateHeader(response); - - // close the connection as soon as the error message is sent. - ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); - }

            -

          • /**
          • * Sets the "date" header for the HTTP response.
          • *
          • * @param response HTTP response
          • */
          • private static void setDateHeader(FullHttpResponse response) { - SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US); - dateFormatter.setTimeZone(GMT_TIMEZONE); - - Calendar time = new GregorianCalendar(); - response.headers().set(DATE, dateFormatter.format(time.getTime())); - }

            -

          • /**
          • * Sets the "date" and "cache" headers for the HTTP Response.
          • *
          • * @param response The HTTP response object.
          • * @param fileToCache File to extract the modification timestamp from.
          • */
          • private static void setDateAndCacheHeaders(HttpResponse response, File fileToCache) { - SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US); - dateFormatter.setTimeZone(GMT_TIMEZONE); - - // date header - Calendar time = new GregorianCalendar(); - response.headers().set(DATE, dateFormatter.format(time.getTime())); - - // cache headers - time.add(Calendar.SECOND, HTTP_CACHE_SECONDS); - response.headers().set(EXPIRES, dateFormatter.format(time.getTime())); - response.headers().set(CACHE_CONTROL, "private, max-age=" + HTTP_CACHE_SECONDS); - response.headers().set(LAST_MODIFIED, dateFormatter.format(new Date(fileToCache.lastModified()))); - }

            -

          • /**
          • * Sets the content type header for the HTTP Response.
          • *
          • * @param response HTTP response
          • * @param file file to extract content type
          • */
          • private static void setContentTypeHeader(HttpResponse response, File file) {
          • String mimeType = MimeTypes.getMimeTypeForFileName(file.getName());
          • String mimeFinal = mimeType != null ? mimeType : MimeTypes.getDefaultMimeType();
          • response.headers().set(CONTENT_TYPE, mimeFinal);
            + protected boolean shouldCache(String requestPath) {
            + return !(requestPath.contains("log") || requestPath.contains("out"));
              • End diff –

          Should we make this more explicit in order to prevent accidental non-caching of requests that contain out or log for another reason?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104431499 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java — @@ -205,215 +125,18 @@ public void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Except } } /** * Response when running with leading JobManager. */ private void respondAsLeader(ChannelHandlerContext ctx, HttpRequest request, String requestPath) throws IOException, ParseException, URISyntaxException { - // 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(); - 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); 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 (!(requestPath.contains("log") || requestPath.contains("out"))) { - 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); + @Override + public String preProcessRequestPath(String requestPath) { + // in case the files being accessed are logs or stdout files, find appropriate paths. + if (requestPath.equals("/jobmanager/log") || requestPath.equals("/jobmanager/stdout")) { + return ""; } 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); + return 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 // ------------------------------------------------------------------------ - /** * Writes a simple error response message. * * @param ctx The channel context to write the response to. * @param status The response status. */ private static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { - FullHttpResponse response = new DefaultFullHttpResponse( - HTTP_1_1, status, Unpooled.copiedBuffer("Failure: " + status + "\r\n", CharsetUtil.UTF_8)); - response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); - - // close the connection as soon as the error message is sent. - ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); - } - /** * Send the "304 Not Modified" response. This response can be used when the * file timestamp is the same as what the browser is sending up. * * @param ctx The channel context to write the response to. */ private static void sendNotModified(ChannelHandlerContext ctx) { - FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, NOT_MODIFIED); - setDateHeader(response); - - // close the connection as soon as the error message is sent. - ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); - } - /** * Sets the "date" header for the HTTP response. * * @param response HTTP response */ private static void setDateHeader(FullHttpResponse response) { - SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US); - dateFormatter.setTimeZone(GMT_TIMEZONE); - - Calendar time = new GregorianCalendar(); - response.headers().set(DATE, dateFormatter.format(time.getTime())); - } - /** * Sets the "date" and "cache" headers for the HTTP Response. * * @param response The HTTP response object. * @param fileToCache File to extract the modification timestamp from. */ private static void setDateAndCacheHeaders(HttpResponse response, File fileToCache) { - SimpleDateFormat dateFormatter = new SimpleDateFormat(HTTP_DATE_FORMAT, Locale.US); - dateFormatter.setTimeZone(GMT_TIMEZONE); - - // date header - Calendar time = new GregorianCalendar(); - response.headers().set(DATE, dateFormatter.format(time.getTime())); - - // cache headers - time.add(Calendar.SECOND, HTTP_CACHE_SECONDS); - response.headers().set(EXPIRES, dateFormatter.format(time.getTime())); - response.headers().set(CACHE_CONTROL, "private, max-age=" + HTTP_CACHE_SECONDS); - response.headers().set(LAST_MODIFIED, dateFormatter.format(new Date(fileToCache.lastModified()))); - } - /** * Sets the content type header for the HTTP Response. * * @param response HTTP response * @param file file to extract content type */ private static void setContentTypeHeader(HttpResponse response, File file) { String mimeType = MimeTypes.getMimeTypeForFileName(file.getName()); String mimeFinal = mimeType != null ? mimeType : MimeTypes.getDefaultMimeType(); response.headers().set(CONTENT_TYPE, mimeFinal); + protected boolean shouldCache(String requestPath) { + return !(requestPath.contains("log") || requestPath.contains("out")); End diff – Should we make this more explicit in order to prevent accidental non-caching of requests that contain out or log for another reason?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104436518

          — Diff: flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerOptions.java —
          @@ -32,6 +32,10 @@
          public static final ConfigOption<Integer> MAX_ATTEMPTS_HISTORY_SIZE =
          key("job-manager.max-attempts-history-size").defaultValue(16);

          + public static final ConfigOption<String> ARCHIVE_DIR =
          + key("job-manager.archive.dir")
          — End diff –

          I know that you are following this classes example, but we have `jobmanager` as the regular config option spelling in the shipped default config. We need to change this and create a issue to rename the max attempts key accordingly.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104436518 — Diff: flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerOptions.java — @@ -32,6 +32,10 @@ public static final ConfigOption<Integer> MAX_ATTEMPTS_HISTORY_SIZE = key("job-manager.max-attempts-history-size").defaultValue(16); + public static final ConfigOption<String> ARCHIVE_DIR = + key("job-manager.archive.dir") — End diff – I know that you are following this classes example, but we have `jobmanager` as the regular config option spelling in the shipped default config. We need to change this and create a issue to rename the max attempts key accordingly.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104401428

          — Diff: flink-dist/src/main/flink-bin/bin/historyserver.sh —
          @@ -0,0 +1,37 @@
          +#!/usr/bin/env bash
          +################################################################################
          +# 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.
          +################################################################################
          +
          +# Start/stop a Flink HistoryServer
          +USAGE="Usage: historyserver.sh (start|stop)"
          +
          +STARTSTOP=$1
          +
          +bin=`dirname "$0"`
          +bin=`cd "$bin"; pwd`
          +
          +. "$bin"/config.sh
          +
          +if [[ $STARTSTOP == "start" ]]; then
          + # export HS specific env opts
          +
          — End diff –

          Empty line here and in line 34

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104401428 — Diff: flink-dist/src/main/flink-bin/bin/historyserver.sh — @@ -0,0 +1,37 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +# Start/stop a Flink HistoryServer +USAGE="Usage: historyserver.sh (start|stop)" + +STARTSTOP=$1 + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +. "$bin"/config.sh + +if [[ $STARTSTOP == "start" ]]; then + # export HS specific env opts + — End diff – Empty line here and in line 34
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104436017

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala —
          @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int)
          }
          }

          + private def archiveJsonFiles(graph: ArchivedExecutionGraph) {
          + future {
          + val rootPath = new Path(flinkConfiguration.getString(
          — End diff –

          Since this is executed async, we should add a safety try-catch around it that logs any exceptions

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104436017 — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala — @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int) } } + private def archiveJsonFiles(graph: ArchivedExecutionGraph) { + future { + val rootPath = new Path(flinkConfiguration.getString( — End diff – Since this is executed async, we should add a safety try-catch around it that logs any exceptions
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104409593

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java —
          @@ -0,0 +1,82 @@
          +/*
          + * 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.history;
          +
          +import io.netty.channel.ChannelHandler;
          +import org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.IOException;
          +
          +@ChannelHandler.Sharable
          +public class HistoryServerStaticFileServerHandler extends AbstractStaticFileServerHandler {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerStaticFileServerHandler.class);
          +
          + // ------------------------------------------------------------------------
          +
          + private final HistoryServer.JobFileFetcher fileFetcher;
          +
          + public HistoryServerStaticFileServerHandler(File rootPath, HistoryServer.JobFileFetcher fileFetcher) throws IOException {
          — End diff –

          Can be package private

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104409593 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java — @@ -0,0 +1,82 @@ +/* + * 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.history; + +import io.netty.channel.ChannelHandler; +import org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +@ChannelHandler.Sharable +public class HistoryServerStaticFileServerHandler extends AbstractStaticFileServerHandler { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerStaticFileServerHandler.class); + + // ------------------------------------------------------------------------ + + private final HistoryServer.JobFileFetcher fileFetcher; + + public HistoryServerStaticFileServerHandler(File rootPath, HistoryServer.JobFileFetcher fileFetcher) throws IOException { — End diff – Can be package private
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104431925

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          — End diff –

          Should we add the job manager dirs as well for convenience? Otherwise, users always have to specify the directories twice. Once for the history server and once for the job manager.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104431925 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); — End diff – Should we add the job manager dirs as well for convenience? Otherwise, users always have to specify the directories twice. Once for the history server and once for the job manager.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104433142

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          — End diff –

          The first execution could be immediate in order to not have a high startup delay until you see historic jobs.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104433142 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); — End diff – The first execution could be immediate in order to not have a high startup delay until you see historic jobs.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104401703

          — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java —
          @@ -0,0 +1,54 @@
          +/*
          + * 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.configuration;
          +
          +import static org.apache.flink.configuration.ConfigOptions.key;
          +
          +public class HistoryServerOptions {
          +
          + public static final ConfigOption<Long> HISTORY_SERVER_REFRESH_INTERVAL =
          + key("historyserver.refresh-interval")
          + .defaultValue(3000L);
          +
          + public static final ConfigOption<String> HISTORY_SERVER_DIR =
          + key("historyserver.archive.dirs")
          + .noDefaultValue();
          +
          + public static final ConfigOption<String> HISTORY_SERVER_WEB_DIR =
          + key("historyserver.web.dir")
          + .noDefaultValue();
          +
          + public static final ConfigOption<String> HISTRY_SERVER_WEB_ADDRESS =
          — End diff –

          typo `HISTRY` => `HISTORY`

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104401703 — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java — @@ -0,0 +1,54 @@ +/* + * 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.configuration; + +import static org.apache.flink.configuration.ConfigOptions.key; + +public class HistoryServerOptions { + + public static final ConfigOption<Long> HISTORY_SERVER_REFRESH_INTERVAL = + key("historyserver.refresh-interval") + .defaultValue(3000L); + + public static final ConfigOption<String> HISTORY_SERVER_DIR = + key("historyserver.archive.dirs") + .noDefaultValue(); + + public static final ConfigOption<String> HISTORY_SERVER_WEB_DIR = + key("historyserver.web.dir") + .noDefaultValue(); + + public static final ConfigOption<String> HISTRY_SERVER_WEB_ADDRESS = — End diff – typo `HISTRY` => `HISTORY`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104401910

          — Diff: flink-dist/src/main/flink-bin/bin/historyserver.sh —
          @@ -0,0 +1,37 @@
          +#!/usr/bin/env bash
          +################################################################################
          +# 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.
          +################################################################################
          +
          +# Start/stop a Flink HistoryServer
          +USAGE="Usage: historyserver.sh (start|stop)"
          +
          +STARTSTOP=$1
          +
          +bin=`dirname "$0"`
          +bin=`cd "$bin"; pwd`
          +
          +. "$bin"/config.sh
          +
          +if [[ $STARTSTOP == "start" ]]; then
          + # export HS specific env opts
          — End diff –

          We are not exporting anything here. Is this a left over comment?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104401910 — Diff: flink-dist/src/main/flink-bin/bin/historyserver.sh — @@ -0,0 +1,37 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +# Start/stop a Flink HistoryServer +USAGE="Usage: historyserver.sh (start|stop)" + +STARTSTOP=$1 + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +. "$bin"/config.sh + +if [[ $STARTSTOP == "start" ]]; then + # export HS specific env opts — End diff – We are not exporting anything here. Is this a left over comment?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104431734

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          — End diff –

          I've noticed that the startup fails silently with output in *.out only when required configuration options are missing. I think we should make sure that they appear in the logs.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104431734 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { — End diff – I've noticed that the startup fails silently with output in *.out only when required configuration options are missing. I think we should make sure that they appear in the logs.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104432442

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          — End diff –

          Can you please check the `BlobCache` for a safe shut down hook pattern although currently it's not too bad because the history server runs in a separate process (but it might change in the future).

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104432442 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { — End diff – Can you please check the `BlobCache` for a safe shut down hook pattern although currently it's not too bad because the history server runs in a separate process (but it might change in the future).
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104433701

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          + private final Path webDir;
          + private final Map<String, Path> cachedArchives;
          +
          + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives)

          { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + }

          +
          + public void fetchFileForRestRequest(String requestPath) throws IOException {
          + /**
          + * Here we extract the job ID from the request path. We are either missing a file that resides
          + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will
          + * return -1 so we manually set 'to' to the index before '.json'.
          + */
          + int from = 6;
          + int to = requestPath.indexOf('/', 7);
          + if (to == -1)

          { + to = requestPath.length() - 5; + }

          + String jobID = requestPath.substring(from, to);
          + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath);
          + try

          { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + }

          catch (Exception e) {
          + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e);
          — End diff –

          I think the better pattern is
          ```java
          if (isDebug())

          { LOG.error(msg, path, jobId, e); }

          else

          { LOG.error(msg, path, jobId); }

          ```
          Furthermore, this line is missing the jobId replacement.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104433701 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { + private final Path webDir; + private final Map<String, Path> cachedArchives; + + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives) { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + } + + public void fetchFileForRestRequest(String requestPath) throws IOException { + /** + * Here we extract the job ID from the request path. We are either missing a file that resides + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will + * return -1 so we manually set 'to' to the index before '.json'. + */ + int from = 6; + int to = requestPath.indexOf('/', 7); + if (to == -1) { + to = requestPath.length() - 5; + } + String jobID = requestPath.substring(from, to); + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath); + try { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + } catch (Exception e) { + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e); — End diff – I think the better pattern is ```java if (isDebug()) { LOG.error(msg, path, jobId, e); } else { LOG.error(msg, path, jobId); } ``` Furthermore, this line is missing the jobId replacement.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104408311

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java —
          @@ -120,15 +59,9 @@

          private final FiniteDuration timeout;

          • /** The path in which the static documents are */
          • private final File rootPath;
            -
            /** Whether the web service has https enabled */
            private final boolean httpsEnabled;
          • /** The log for all error reporting */
          • private final Logger logger;
            -
            private String localJobManagerAddress;

          public StaticFileServerHandler(
          — End diff –

          Could you remove this constructor and only use the other one? This one is only used via the other one and the logger is never overwritten.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104408311 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java — @@ -120,15 +59,9 @@ private final FiniteDuration timeout; /** The path in which the static documents are */ private final File rootPath; - /** Whether the web service has https enabled */ private final boolean httpsEnabled; /** The log for all error reporting */ private final Logger logger; - private String localJobManagerAddress; public StaticFileServerHandler( — End diff – Could you remove this constructor and only use the other one? This one is only used via the other one and the logger is never overwritten.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104634061

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          + private final Path webDir;
          + private final Map<String, Path> cachedArchives;
          +
          + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives)

          { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + }

          +
          + public void fetchFileForRestRequest(String requestPath) throws IOException {
          + /**
          + * Here we extract the job ID from the request path. We are either missing a file that resides
          + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will
          + * return -1 so we manually set 'to' to the index before '.json'.
          + */
          + int from = 6;
          + int to = requestPath.indexOf('/', 7);
          + if (to == -1)

          { + to = requestPath.length() - 5; + }

          + String jobID = requestPath.substring(from, to);
          + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath);
          + try

          { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + }

          catch (Exception e) {
          + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e);
          + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID);
          + }
          + }
          + }
          +
          + // =================================================================================================================
          + // File generation
          + // =================================================================================================================
          + private static FileWriter createOrGetFile(File folder, String name) throws IOException {
          + File file = new File(folder, name + ".json");
          + if (!file.exists())

          { + Files.createFile(file.toPath()); + }

          + FileWriter fr = new FileWriter(file);
          + return fr;
          + }
          +
          + private void createDashboardConfigFile() throws IOException {
          — End diff –

          yes. In order to use the DBConfigHandler you have to wrap it in a RuntimeMonitorHandler which require a JobManagerRetriever and JobManagerAddressFuture.

          I could add a wrapper that doesn't enforce this requirement; then we could totally use it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104634061 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { + private final Path webDir; + private final Map<String, Path> cachedArchives; + + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives) { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + } + + public void fetchFileForRestRequest(String requestPath) throws IOException { + /** + * Here we extract the job ID from the request path. We are either missing a file that resides + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will + * return -1 so we manually set 'to' to the index before '.json'. + */ + int from = 6; + int to = requestPath.indexOf('/', 7); + if (to == -1) { + to = requestPath.length() - 5; + } + String jobID = requestPath.substring(from, to); + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath); + try { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + } catch (Exception e) { + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e); + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID); + } + } + } + + // ================================================================================================================= + // File generation + // ================================================================================================================= + private static FileWriter createOrGetFile(File folder, String name) throws IOException { + File file = new File(folder, name + ".json"); + if (!file.exists()) { + Files.createFile(file.toPath()); + } + FileWriter fr = new FileWriter(file); + return fr; + } + + private void createDashboardConfigFile() throws IOException { — End diff – yes. In order to use the DBConfigHandler you have to wrap it in a RuntimeMonitorHandler which require a JobManagerRetriever and JobManagerAddressFuture. I could add a wrapper that doesn't enforce this requirement; then we could totally use it.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104634695

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          + private final Path webDir;
          + private final Map<String, Path> cachedArchives;
          +
          + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives)

          { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + }

          +
          + public void fetchFileForRestRequest(String requestPath) throws IOException {
          + /**
          + * Here we extract the job ID from the request path. We are either missing a file that resides
          + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will
          + * return -1 so we manually set 'to' to the index before '.json'.
          + */
          + int from = 6;
          + int to = requestPath.indexOf('/', 7);
          + if (to == -1)

          { + to = requestPath.length() - 5; + }

          + String jobID = requestPath.substring(from, to);
          + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath);
          + try

          { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + }

          catch (Exception e) {
          + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e);
          + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID);
          + }
          + }
          + }
          +
          + // =================================================================================================================
          + // File generation
          + // =================================================================================================================
          + private static FileWriter createOrGetFile(File folder, String name) throws IOException {
          + File file = new File(folder, name + ".json");
          + if (!file.exists())

          { + Files.createFile(file.toPath()); + }

          + FileWriter fr = new FileWriter(file);
          + return fr;
          + }
          +
          + private void createDashboardConfigFile() throws IOException {
          — End diff –

          Actually, turning around the inheritancs of ```RuntimeMonitorHandler``` and ```RuntimeMonitorHandlerBase``` would do the trick.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104634695 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { + private final Path webDir; + private final Map<String, Path> cachedArchives; + + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives) { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + } + + public void fetchFileForRestRequest(String requestPath) throws IOException { + /** + * Here we extract the job ID from the request path. We are either missing a file that resides + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will + * return -1 so we manually set 'to' to the index before '.json'. + */ + int from = 6; + int to = requestPath.indexOf('/', 7); + if (to == -1) { + to = requestPath.length() - 5; + } + String jobID = requestPath.substring(from, to); + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath); + try { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + } catch (Exception e) { + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e); + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID); + } + } + } + + // ================================================================================================================= + // File generation + // ================================================================================================================= + private static FileWriter createOrGetFile(File folder, String name) throws IOException { + File file = new File(folder, name + ".json"); + if (!file.exists()) { + Files.createFile(file.toPath()); + } + FileWriter fr = new FileWriter(file); + return fr; + } + + private void createDashboardConfigFile() throws IOException { — End diff – Actually, turning around the inheritancs of ```RuntimeMonitorHandler``` and ```RuntimeMonitorHandlerBase``` would do the trick.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104635328

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          + private final Path webDir;
          + private final Map<String, Path> cachedArchives;
          +
          + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives)

          { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + }

          +
          + public void fetchFileForRestRequest(String requestPath) throws IOException {
          + /**
          + * Here we extract the job ID from the request path. We are either missing a file that resides
          + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will
          + * return -1 so we manually set 'to' to the index before '.json'.
          + */
          + int from = 6;
          + int to = requestPath.indexOf('/', 7);
          + if (to == -1)

          { + to = requestPath.length() - 5; + }

          + String jobID = requestPath.substring(from, to);
          + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath);
          + try

          { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + }

          catch (Exception e) {
          + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e);
          + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID);
          + }
          + }
          + }
          +
          + // =================================================================================================================
          + // File generation
          + // =================================================================================================================
          + private static FileWriter createOrGetFile(File folder, String name) throws IOException {
          + File file = new File(folder, name + ".json");
          + if (!file.exists())

          { + Files.createFile(file.toPath()); + }

          + FileWriter fr = new FileWriter(file);
          + return fr;
          + }
          +
          + private void createDashboardConfigFile() throws IOException {
          + try (FileWriter fw = createOrGetFile(webDir, "config"))

          { + fw.write(DashboardConfigHandler.createConfigJson(webRefreshIntervalMillis)); + fw.flush(); + }

          catch (IOException ioe)

          { + LOG.error("Failed to write config file."); + throw ioe; + }

          + }
          +
          + /**
          + * This method replicates the JSON response that would be given by the

          {@link CurrentJobsOverviewHandler}

          when
          + * listing both running and finished jobs.
          + *
          + * Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on
          + * their own however the list of finished jobs only contains a single job.
          + *
          + * For the display in the HistoryServer WebFrontend we have to combine these overviews.
          + */
          + private static void updateJobOverview(File webDir) {
          — End diff –

          The CurrentJobsOverhandler is hard-wired to use akka; it sends a ```RequestJobDetails``` request. Dealing with that is more complicated than the current approach.

          We could add a new in-memory handler for this though. The entire implementation would be basically the ```updateJobOverview``` method; without the file writing.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104635328 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { + private final Path webDir; + private final Map<String, Path> cachedArchives; + + public JobFileFetcher(Path webDir, Map<String, Path> cachedArchives) { + this.webDir = webDir; + this.cachedArchives = cachedArchives; + } + + public void fetchFileForRestRequest(String requestPath) throws IOException { + /** + * Here we extract the job ID from the request path. We are either missing a file that resides + * under /jobs/:jobid/* or /jobs/:jobid.json. In the latter case indexOf will + * return -1 so we manually set 'to' to the index before '.json'. + */ + int from = 6; + int to = requestPath.indexOf('/', 7); + if (to == -1) { + to = requestPath.length() - 5; + } + String jobID = requestPath.substring(from, to); + Path sourcePath = new Path(new Path(cachedArchives.get(jobID), jobID), requestPath); + try { + FileCache.copy(sourcePath, new Path(webDir, requestPath), false); + } catch (Exception e) { + LOG.debug("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, e); + LOG.error("Failed to retrieve file {} for job {}. This may indicate an incomplete or corrupt job archive.", sourcePath, jobID); + } + } + } + + // ================================================================================================================= + // File generation + // ================================================================================================================= + private static FileWriter createOrGetFile(File folder, String name) throws IOException { + File file = new File(folder, name + ".json"); + if (!file.exists()) { + Files.createFile(file.toPath()); + } + FileWriter fr = new FileWriter(file); + return fr; + } + + private void createDashboardConfigFile() throws IOException { + try (FileWriter fw = createOrGetFile(webDir, "config")) { + fw.write(DashboardConfigHandler.createConfigJson(webRefreshIntervalMillis)); + fw.flush(); + } catch (IOException ioe) { + LOG.error("Failed to write config file."); + throw ioe; + } + } + + /** + * This method replicates the JSON response that would be given by the {@link CurrentJobsOverviewHandler} when + * listing both running and finished jobs. + * + * Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on + * their own however the list of finished jobs only contains a single job. + * + * For the display in the HistoryServer WebFrontend we have to combine these overviews. + */ + private static void updateJobOverview(File webDir) { — End diff – The CurrentJobsOverhandler is hard-wired to use akka; it sends a ```RequestJobDetails``` request. Dealing with that is more complicated than the current approach. We could add a new in-memory handler for this though. The entire implementation would be basically the ```updateJobOverview``` method; without the file writing.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on the issue:

          https://github.com/apache/flink/pull/3460

          @uce I've addressed a few of your comments (all that i gave a Thumbs up)

          I will implement that we write out only 1 file instead of multiple ones; this removes plenty of edge cases and makes the JobFileFetcher irrelevant.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on the issue: https://github.com/apache/flink/pull/3460 @uce I've addressed a few of your comments (all that i gave a Thumbs up) I will implement that we write out only 1 file instead of multiple ones; this removes plenty of edge cases and makes the JobFileFetcher irrelevant.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104640380

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          — End diff –

          that's ood, will look into it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104640380 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { — End diff – that's ood, will look into it.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104640447

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          + if (refreshDirectories == null)

          { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + }

          + for (String refreshDirectory : refreshDirectories.split(","))

          { + refreshDirs.add(new Path(refreshDirectory)); + }

          + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          +
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir);
          +
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try {
          + Runtime.getRuntime().addShutdownHook(new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + });
          + } catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + private void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          + if (LOG.isInfoEnabled()) {
          + for (Path refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir);
          + }
          + }
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives)));
          +
          + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          +
          + createDashboardConfigFile();
          + updateJobOverview(webDir);
          +
          + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS);
          + }
          + }
          +
          + private void stop() {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          + netty.shutdown();
          + shutdownExecutor();
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          + }
          + }
          +
          + private void shutdownExecutor() {
          + if (executor != null) {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          + }
          +
          + // =================================================================================================================
          + // File-fetching
          + // =================================================================================================================
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIR}

          for
          + * new job archives.
          + */
          + private static class JobArchiveFetcherTask extends TimerTask {
          + private final List<Path> refreshDirs;
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final Path webJobDir;
          + private final Path webOverviewDir;
          +
          + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + }

          +
          + @Override
          + public void run() {
          + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() {
          + Path refreshDir = iterator.next();
          + FileSystem fs;
          + try

          { + fs = refreshDir.getFileSystem(); + }

          catch (IOException e) {
          + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e);
          + // there's most likely something wrong with the path itself
          + try

          { + iterator.remove(); + }

          catch (Exception ignored)

          { + //well, we tried + }

          + continue;
          + }
          +
          + /**
          + * The expected directory structure is as follows:
          + * /:refreshDir/:jobid/jobs/:jobid/...
          + * /:refreshDir/:jobid/jobs/:jobid.json
          + * /:refreshDir/:jobid/joboverview.json
          + */
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = fs.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + try {
          + // contents of /:refreshDir/:jobid
          + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath);
          + for (FileStatus archiveFile : archiveFiles) {
          + if (archiveFile.isDir()) {
          + // contents of /:refreshDir/:jobid/jobs
          + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath()))

          { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + }

          + } else

          { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + }

          + }
          + } catch (IOException e)

          { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + }

          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + }
          + }
          +
          + /**
          + * Backup file fetcher that is used by the

          {@link HistoryServerStaticFileServerHandler}

          if it cannot find a file
          + * within a job archive. This class does not share code with the

          {@link JobArchiveFetcherTask}

          since the former
          + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file
          + * based on a REST URL.
          + */
          + public static class JobFileFetcher {
          — End diff –

          This class will be removed once we only write out 1 file.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104640447 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); + if (refreshDirectories == null) { + throw new IOException(HistoryServerOptions.HISTORY_SERVER_DIR + " was not configured."); + } + for (String refreshDirectory : refreshDirectories.split(",")) { + refreshDirs.add(new Path(refreshDirectory)); + } + refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, cachedArchives, webDir); + + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + private void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + if (LOG.isInfoEnabled()) { + for (Path refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir); + } + } + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir, new JobFileFetcher(new Path(webDir.getAbsolutePath()), cachedArchives))); + + netty = new NettySetup(router, LOG, webDir, serverSSLContext, webAddress, webPort); + + createDashboardConfigFile(); + updateJobOverview(webDir); + + executor.scheduleWithFixedDelay(fetcherTask, refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + } + + private void stop() { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + netty.shutdown(); + shutdownExecutor(); + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + } + + // ================================================================================================================= + // File-fetching + // ================================================================================================================= + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIR} for + * new job archives. + */ + private static class JobArchiveFetcherTask extends TimerTask { + private final List<Path> refreshDirs; + private final Map<String, Path> cachedArchives; + private final File webDir; + private final Path webJobDir; + private final Path webOverviewDir; + + public JobArchiveFetcherTask(List<Path> refreshDirs, Map<String, Path> cachedArchives, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = cachedArchives; + this.webDir = webDir; + this.webJobDir = new Path(webDir.getAbsolutePath(), "jobs"); + this.webOverviewDir = new Path(webDir.getAbsolutePath(), OVERVIEWS_FOLDER_NAME); + } + + @Override + public void run() { + for (Iterator<Path> iterator = refreshDirs.iterator(); iterator.hasNext() { + Path refreshDir = iterator.next(); + FileSystem fs; + try { + fs = refreshDir.getFileSystem(); + } catch (IOException e) { + LOG.error("Failed to create FileSystem for path {}.", refreshDir, e); + // there's most likely something wrong with the path itself + try { + iterator.remove(); + } catch (Exception ignored) { + //well, we tried + } + continue; + } + + /** + * The expected directory structure is as follows: + * /:refreshDir/:jobid/jobs/:jobid/... + * /:refreshDir/:jobid/jobs/:jobid.json + * /:refreshDir/:jobid/joboverview.json + */ + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = fs.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + try { + // contents of /:refreshDir/:jobid + FileStatus[] archiveFiles = fs.listStatus(jobArchivePath); + for (FileStatus archiveFile : archiveFiles) { + if (archiveFile.isDir()) { + // contents of /:refreshDir/:jobid/jobs + for (FileStatus jobFile : fs.listStatus(archiveFile.getPath())) { + /** + * src: /:refreshDir/:jobid/jobs/X + * dst: /:webDir/jobs/X + * + * X is either :jobid.json or the :jobid directory. + */ + FileCache.copy(jobFile.getPath(), new Path(webJobDir + "/" + jobFile.getPath().getName()), false); + } + } else { + /** + * src: /:refreshDir/:jobid/joboverview.json + * dst: /:webDir/joboverviews/:jobid.json + */ + FileCache.copy(archiveFile.getPath(), new Path(webOverviewDir, jobID + ".json"), false); + updateOverview = true; + } + } + } catch (IOException e) { + LOG.error("Failed to fetch archive file.", e); + cachedArchives.remove(jobID); + continue; + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } + } + + /** + * Backup file fetcher that is used by the {@link HistoryServerStaticFileServerHandler} if it cannot find a file + * within a job archive. This class does not share code with the {@link JobArchiveFetcherTask} since the former + * fetches entire archives at a given location based on a path whereas this class only fetches one specific file + * based on a REST URL. + */ + public static class JobFileFetcher { — End diff – This class will be removed once we only write out 1 file.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104640647

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala —
          @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int)
          }
          }

          + private def archiveJsonFiles(graph: ArchivedExecutionGraph) {
          + future {
          + val rootPath = new Path(flinkConfiguration.getString(
          + JobManagerOptions.ARCHIVE_DIR))
          + val fs = rootPath.getFileSystem
          + val tmpArchivePath = new Path(rootPath, s"tmp_$

          {graph.getJobID.toString}")
          + for (archiver <- WebMonitorUtils.getArchivers) {
          + try {
          + for (archive <- archiver.archiveJsonWithPath(graph).asScala) {
          + val targetPath =
          + new Path(tmpArchivePath, s"${archive.getPath}.json")
          + val out = fs.create(targetPath, false)
          + out.write(archive.getJson.getBytes(StandardCharsets.UTF_8))
          + out.close()
          + }
          + } catch {
          + case ioe: IOException => { + log.error("Failed to archive job details.", ioe) + }
          + }
          + }
          + if (!fs.rename(tmpArchivePath, new Path(rootPath, s"${graph.getJobID.toString}

          "))) {
          — End diff –

          yes. Once we write out 1 file this should no longer be an issue.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104640647 — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala — @@ -183,6 +193,33 @@ class MemoryArchivist(private val max_entries: Int) } } + private def archiveJsonFiles(graph: ArchivedExecutionGraph) { + future { + val rootPath = new Path(flinkConfiguration.getString( + JobManagerOptions.ARCHIVE_DIR)) + val fs = rootPath.getFileSystem + val tmpArchivePath = new Path(rootPath, s"tmp_$ {graph.getJobID.toString}") + for (archiver <- WebMonitorUtils.getArchivers) { + try { + for (archive <- archiver.archiveJsonWithPath(graph).asScala) { + val targetPath = + new Path(tmpArchivePath, s"${archive.getPath}.json") + val out = fs.create(targetPath, false) + out.write(archive.getJson.getBytes(StandardCharsets.UTF_8)) + out.close() + } + } catch { + case ioe: IOException => { + log.error("Failed to archive job details.", ioe) + } + } + } + if (!fs.rename(tmpArchivePath, new Path(rootPath, s"${graph.getJobID.toString} "))) { — End diff – yes. Once we write out 1 file this should no longer be an issue.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r104643908

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,411 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.NettySetup;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.Iterator;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.UUID;
          +import java.util.concurrent.ConcurrentHashMap;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private static final String OVERVIEWS_FOLDER_NAME = "overviews";
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final long refreshIntervalMillis;
          + private final List<Path> refreshDirs = new ArrayList<>();
          +
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>();
          +
          + private final SSLContext serverSSLContext;
          + private NettySetup netty;
          + private final Object startupShutdownLock = new Object();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + HistoryServer hs = new HistoryServer(flinkConfig);
          + hs.run();
          + }
          +
          + public HistoryServer(Configuration config) throws IOException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR);
          — End diff –

          I don't know, sounds a bit inconsistent. The HS configuration would be different depending on whether you have 1 or multiple JM. With 1 JM only setting ARCHIVE_DIR is enough; for multiple HISTORY_SERVER_DIRS *must* be set.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r104643908 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,411 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.NettySetup; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private static final String OVERVIEWS_FOLDER_NAME = "overviews"; + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final long refreshIntervalMillis; + private final List<Path> refreshDirs = new ArrayList<>(); + + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives = new ConcurrentHashMap<>(); + + private final SSLContext serverSSLContext; + private NettySetup netty; + private final Object startupShutdownLock = new Object(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-FileFetcher")); + private final JobArchiveFetcherTask fetcherTask; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + } + + public HistoryServer(Configuration config) throws IOException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTRY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIR); — End diff – I don't know, sounds a bit inconsistent. The HS configuration would be different depending on whether you have 1 or multiple JM. With 1 JM only setting ARCHIVE_DIR is enough; for multiple HISTORY_SERVER_DIRS * must * be set.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on the issue:

          https://github.com/apache/flink/pull/3460

          @uce I've addressed the remaining comments. Still undecided on the handler re-use; tests/documentation still missing.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on the issue: https://github.com/apache/flink/pull/3460 @uce I've addressed the remaining comments. Still undecided on the handler re-use; tests/documentation still missing.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105877694

          — Diff: flink-runtime-web/web-dashboard/app/index_hs.jade —
          @@ -0,0 +1,60 @@
          +//
          + 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.
          +
          +doctype html
          +html(lang='en')
          + head
          + meta(charset='utf-8')
          + meta(http-equiv='X-UA-Compatible', content='IE=edge')
          + meta(name='viewport', content='width=device-width, initial-scale=1')
          +
          + title Apache Flink Web Dashboard
          — End diff –

          Should we adjust the title to `Apache Flink History Server`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105877694 — Diff: flink-runtime-web/web-dashboard/app/index_hs.jade — @@ -0,0 +1,60 @@ +// + 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. + +doctype html +html(lang='en') + head + meta(charset='utf-8') + meta(http-equiv='X-UA-Compatible', content='IE=edge') + meta(name='viewport', content='width=device-width, initial-scale=1') + + title Apache Flink Web Dashboard — End diff – Should we adjust the title to `Apache Flink History Server`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105868993

          — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java —
          @@ -0,0 +1,82 @@
          +/*
          + * 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.configuration;
          +
          +import org.apache.flink.annotation.PublicEvolving;
          +
          +import static org.apache.flink.configuration.ConfigOptions.key;
          +
          +/**
          + * The set of configuration options relating to the HistoryServer.
          + */
          +@PublicEvolving
          +public class HistoryServerOptions {
          +
          + /**
          + * The interval at which the HistoryServer polls

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for new archives.
          + */
          + public static final ConfigOption<Long> HISTORY_SERVER_REFRESH_INTERVAL =
          + key("historyserver.refresh-interval")
          + .defaultValue(3000L);
          +
          + /**
          + * Comma-separated list of directories which the HistoryServer polls for new archives.
          + */
          + public static final ConfigOption<String> HISTORY_SERVER_DIRS =
          + key("historyserver.archive.dirs")
          + .noDefaultValue();
          +
          + /**
          + * The local directory used by the HistoryServer web-frontend.
          + */
          + public static final ConfigOption<String> HISTORY_SERVER_WEB_DIR =
          + key("historyserver.web.dir")
          — End diff –

          Should we call this `historyserver.web.tmpDir` to better distinguish against the other `fs` config options?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105868993 — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java — @@ -0,0 +1,82 @@ +/* + * 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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * The set of configuration options relating to the HistoryServer. + */ +@PublicEvolving +public class HistoryServerOptions { + + /** + * The interval at which the HistoryServer polls {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for new archives. + */ + public static final ConfigOption<Long> HISTORY_SERVER_REFRESH_INTERVAL = + key("historyserver.refresh-interval") + .defaultValue(3000L); + + /** + * Comma-separated list of directories which the HistoryServer polls for new archives. + */ + public static final ConfigOption<String> HISTORY_SERVER_DIRS = + key("historyserver.archive.dirs") + .noDefaultValue(); + + /** + * The local directory used by the HistoryServer web-frontend. + */ + public static final ConfigOption<String> HISTORY_SERVER_WEB_DIR = + key("historyserver.web.dir") — End diff – Should we call this `historyserver.web.tmpDir` to better distinguish against the other `fs` config options?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105898216

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
          +
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          + private final long refreshIntervalMillis;
          +
          + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) {
          + this.refreshIntervalMillis = refreshIntervalMillis;
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir);
          + if (LOG.isInfoEnabled()) {
          + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath());
          + }
          + }
          + }
          +
          + void start()

          { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + }

          +
          + void stop() {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for
          + * new job archives.
          + */
          + static class JobArchiveFetcherTask extends TimerTask {
          + private final List<HistoryServer.RefreshLocation> refreshDirs;
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final File webTmpDir;
          + private final File webJobDir;
          + private final File webOverviewDir;
          +
          + private static final String JSON_FILE_ENDING = ".json";
          +
          + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + }

          +
          + @Override
          + public void run() {
          + try {
          + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
          + Path refreshDir = refreshLocation.getPath();
          + FileSystem refreshFS = refreshLocation.getFs();
          +
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = refreshFS.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + if (jobArchives == null)

          { + continue; + }

          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + File tmpLocalFile = new File(webTmpDir, jobID);
          + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID);
          + try {
          + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false);
          +
          + JsonNode archive = mapper.readTree(tmpLocalFile);
          + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) {
          + String path = archivePart.get(ArchivedJson.PATH).asText();
          + String json = archivePart.get(ArchivedJson.JSON).asText();
          +
          + File target;
          + if (path.equals("/joboverview"))

          { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + }

          else

          { + target = new File(webDir, path + JSON_FILE_ENDING); + }

          +
          + java.nio.file.Path parent = target.getParentFile().toPath();
          +
          + try

          { + Files.createDirectories(parent); + }

          catch (FileAlreadyExistsException ignored)

          { + // odd, but OK + }

          +
          + java.nio.file.Path targetPath = target.toPath();
          + /**
          — End diff –

          I think we should use `//` comments for inline comments.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105898216 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); + + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher")); + private final JobArchiveFetcherTask fetcherTask; + private final long refreshIntervalMillis; + + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshIntervalMillis = refreshIntervalMillis; + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir); + if (LOG.isInfoEnabled()) { + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); + } + } + } + + void start() { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + + void stop() { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for + * new job archives. + */ + static class JobArchiveFetcherTask extends TimerTask { + private final List<HistoryServer.RefreshLocation> refreshDirs; + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives; + private final File webDir; + private final File webTmpDir; + private final File webJobDir; + private final File webOverviewDir; + + private static final String JSON_FILE_ENDING = ".json"; + + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + } + + @Override + public void run() { + try { + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { + Path refreshDir = refreshLocation.getPath(); + FileSystem refreshFS = refreshLocation.getFs(); + + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = refreshFS.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + if (jobArchives == null) { + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + File tmpLocalFile = new File(webTmpDir, jobID); + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID); + try { + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false); + + JsonNode archive = mapper.readTree(tmpLocalFile); + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) { + String path = archivePart.get(ArchivedJson.PATH).asText(); + String json = archivePart.get(ArchivedJson.JSON).asText(); + + File target; + if (path.equals("/joboverview")) { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + } else { + target = new File(webDir, path + JSON_FILE_ENDING); + } + + java.nio.file.Path parent = target.getParentFile().toPath(); + + try { + Files.createDirectories(parent); + } catch (FileAlreadyExistsException ignored) { + // odd, but OK + } + + java.nio.file.Path targetPath = target.toPath(); + /** — End diff – I think we should use `//` comments for inline comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105868354

          — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java —
          @@ -0,0 +1,82 @@
          +/*
          + * 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.configuration;
          +
          +import org.apache.flink.annotation.PublicEvolving;
          +
          +import static org.apache.flink.configuration.ConfigOptions.key;
          +
          +/**
          + * The set of configuration options relating to the HistoryServer.
          + */
          +@PublicEvolving
          +public class HistoryServerOptions {
          — End diff –

          Should we rename the following keys to make them filesystem-specific, leaving the option to implement other ways of data exchange between the JM and history server.
          ```
          historyserver.refresh-interval -> historyserver.archive.fs.refresh-interval
          historyserver.archive.dirs -> historyserver.archive.fs.dirs
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105868354 — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java — @@ -0,0 +1,82 @@ +/* + * 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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * The set of configuration options relating to the HistoryServer. + */ +@PublicEvolving +public class HistoryServerOptions { — End diff – Should we rename the following keys to make them filesystem-specific, leaving the option to implement other ways of data exchange between the JM and history server. ``` historyserver.refresh-interval -> historyserver.archive.fs.refresh-interval historyserver.archive.dirs -> historyserver.archive.fs.dirs ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105876698

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,276 @@
          +/*
          + * 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.history;
          +
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.security.SecurityUtils;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
          +import org.apache.flink.util.FileUtils;
          +import org.apache.flink.util.FlinkException;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.lang.reflect.UndeclaredThrowableException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.UUID;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final HistoryServerArchiveFetcher archiveFetcher;
          +
          + private final SSLContext serverSSLContext;
          + private WebFrontendBootstrap netty;
          +
          + private final Object startupShutdownLock = new Object();
          + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
          + private final Thread shutdownHook;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + // run the job manager
          + SecurityUtils.install(new SecurityUtils.SecurityConfiguration(flinkConfig));
          +
          + try {
          + SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
          + @Override
          + public Integer call() throws Exception

          { + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + return 0; + }

          + });
          + System.exit(0);
          + } catch (UndeclaredThrowableException ute)

          { + Throwable cause = ute. getUndeclaredThrowable(); + LOG.error("Failed to run HistoryServer.", cause); + cause.printStackTrace(); + System.exit(1); + }

          catch (Exception e)

          { + LOG.error("Failed to run HistoryServer.", e); + e.printStackTrace(); + System.exit(1); + }

          + }
          +
          + public HistoryServer(Configuration config) throws IOException, FlinkException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIRS);
          + if (refreshDirectories == null)

          { + throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_DIRS + " was not configured."); + }

          + List<RefreshLocation> refreshDirs = new ArrayList<>();
          + for (String refreshDirectory : refreshDirectories.split(",")) {
          + try

          { + Path refreshPath = new Path(refreshDirectory); + FileSystem refreshFS = refreshPath.getFileSystem(); + refreshDirs.add(new RefreshLocation(refreshPath, refreshFS)); + }

          catch (Exception e) {
          + // there's most likely something wrong with the path itself, so we ignore it from here on
          + LOG.error("Failed to create Path or FileSystem for directory {}.", refreshDirectory, e);
          + }
          + }
          +
          + long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir);
          +
          + this.shutdownHook = new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + };
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try

          { + Runtime.getRuntime().addShutdownHook(shutdownHook); + }

          catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir));
          +
          + webDir.mkdirs();
          +
          + createDashboardConfigFile();
          +
          + archiveFetcher.start();
          +
          + netty = new WebFrontendBootstrap(router, LOG, webDir, serverSSLContext, webAddress, webPort);
          + }
          + }
          +
          + void stop() {
          + if (shutdownRequested.compareAndSet(false, true)) {
          + synchronized (startupShutdownLock) {
          + LOG.info("Stopping history server.");
          +
          + try

          { + netty.shutdown(); + }

          catch (Throwable t)

          { + LOG.warn("Error while shutting down WebFrontendBootstrap.", t); + }

          +
          + archiveFetcher.stop();
          +
          + try {
          + LOG.info("Removing web dashboard root cache directory {}", webDir);
          + FileUtils.deleteDirectory(webDir);
          + } catch (Throwable t) {
          + LOG.warn("Error while deleting web root directory {}", webDir, t);
          + }
          +
          + LOG.info("Stopped history server.");
          +
          + // Remove shutdown hook to prevent resource leaks, unless this is invoked by the shutdown hook itself
          + if (shutdownHook != null && shutdownHook != Thread.currentThread()) {
          + try

          { + Runtime.getRuntime().removeShutdownHook(shutdownHook); + }

          catch (IllegalStateException ignored)

          { + // race, JVM is in shutdown already, we can safely ignore this + }

          catch (Throwable t)

          { + LOG.warn("Exception while unregistering HistoryServer cleanup shutdown hook."); + }

          + }
          + }
          + }
          + }
          +
          + // =================================================================================================================
          + // File generation
          — End diff –

          Same here about formatting and empty line

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105876698 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,276 @@ +/* + * 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.history; + +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final HistoryServerArchiveFetcher archiveFetcher; + + private final SSLContext serverSSLContext; + private WebFrontendBootstrap netty; + + private final Object startupShutdownLock = new Object(); + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + private final Thread shutdownHook; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + // run the job manager + SecurityUtils.install(new SecurityUtils.SecurityConfiguration(flinkConfig)); + + try { + SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() { + @Override + public Integer call() throws Exception { + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + return 0; + } + }); + System.exit(0); + } catch (UndeclaredThrowableException ute) { + Throwable cause = ute. getUndeclaredThrowable(); + LOG.error("Failed to run HistoryServer.", cause); + cause.printStackTrace(); + System.exit(1); + } catch (Exception e) { + LOG.error("Failed to run HistoryServer.", e); + e.printStackTrace(); + System.exit(1); + } + } + + public HistoryServer(Configuration config) throws IOException, FlinkException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIRS); + if (refreshDirectories == null) { + throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_DIRS + " was not configured."); + } + List<RefreshLocation> refreshDirs = new ArrayList<>(); + for (String refreshDirectory : refreshDirectories.split(",")) { + try { + Path refreshPath = new Path(refreshDirectory); + FileSystem refreshFS = refreshPath.getFileSystem(); + refreshDirs.add(new RefreshLocation(refreshPath, refreshFS)); + } catch (Exception e) { + // there's most likely something wrong with the path itself, so we ignore it from here on + LOG.error("Failed to create Path or FileSystem for directory {}.", refreshDirectory, e); + } + } + + long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir); + + this.shutdownHook = new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }; + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(shutdownHook); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir)); + + webDir.mkdirs(); + + createDashboardConfigFile(); + + archiveFetcher.start(); + + netty = new WebFrontendBootstrap(router, LOG, webDir, serverSSLContext, webAddress, webPort); + } + } + + void stop() { + if (shutdownRequested.compareAndSet(false, true)) { + synchronized (startupShutdownLock) { + LOG.info("Stopping history server."); + + try { + netty.shutdown(); + } catch (Throwable t) { + LOG.warn("Error while shutting down WebFrontendBootstrap.", t); + } + + archiveFetcher.stop(); + + try { + LOG.info("Removing web dashboard root cache directory {}", webDir); + FileUtils.deleteDirectory(webDir); + } catch (Throwable t) { + LOG.warn("Error while deleting web root directory {}", webDir, t); + } + + LOG.info("Stopped history server."); + + // Remove shutdown hook to prevent resource leaks, unless this is invoked by the shutdown hook itself + if (shutdownHook != null && shutdownHook != Thread.currentThread()) { + try { + Runtime.getRuntime().removeShutdownHook(shutdownHook); + } catch (IllegalStateException ignored) { + // race, JVM is in shutdown already, we can safely ignore this + } catch (Throwable t) { + LOG.warn("Exception while unregistering HistoryServer cleanup shutdown hook."); + } + } + } + } + } + + // ================================================================================================================= + // File generation — End diff – Same here about formatting and empty line
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105918471

          — Diff: flink-runtime-web/web-dashboard/app/index_hs.jade —
          @@ -0,0 +1,60 @@
          +//
          + 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.
          +
          +doctype html
          +html(lang='en')
          + head
          + meta(charset='utf-8')
          + meta(http-equiv='X-UA-Compatible', content='IE=edge')
          + meta(name='viewport', content='width=device-width, initial-scale=1')
          +
          + title Apache Flink Web Dashboard
          +
          + link(rel="apple-touch-icon", sizes="180x180", href="images/apple-touch-icon.png")
          + link(rel="icon", type="image/png", href="images/favicon-32x32.png", sizes="32x32")
          + link(rel="icon", type="image/png", href="images/favicon-16x16.png", sizes="16x16")
          + link(rel="manifest", href="images/manifest.json")
          + link(rel="mask-icon", href="images/safari-pinned-tab.svg", color="#aa1919")
          + link(rel="shortcut icon", href="images/favicon.ico")
          + meta(name="msapplication-config", content="images/browserconfig.xml")
          + meta(name="theme-color", content="#ffffff")
          +
          + link(rel='stylesheet', href='css/vendor.css', type='text/css')
          + link(rel='stylesheet', href='css/index.css', type='text/css')
          +
          + script(src="js/vendor.js")
          + script(src="js/hs/index.js")
          +
          + body(ng-app="flinkApp" ng-strict-di)
          + #sidebar(ng-class="

          { 'sidebar-visible': sidebarVisible }

          ")
          + nav.navbar.navbar-inverse.navbar-static-top
          + .navbar-header
          + a.navbar-brand(ui-sref="completed-jobs")
          + img.logo(alt="Apache Flink Dashboard" src="images/flink-logo.png")
          + a.navbar-brand.navbar-brand-text(ui-sref="completed-jobs")
          + | Apache Flink Dashboard
          — End diff –

          Also rename to `Apache Flink History Server`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105918471 — Diff: flink-runtime-web/web-dashboard/app/index_hs.jade — @@ -0,0 +1,60 @@ +// + 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. + +doctype html +html(lang='en') + head + meta(charset='utf-8') + meta(http-equiv='X-UA-Compatible', content='IE=edge') + meta(name='viewport', content='width=device-width, initial-scale=1') + + title Apache Flink Web Dashboard + + link(rel="apple-touch-icon", sizes="180x180", href="images/apple-touch-icon.png") + link(rel="icon", type="image/png", href="images/favicon-32x32.png", sizes="32x32") + link(rel="icon", type="image/png", href="images/favicon-16x16.png", sizes="16x16") + link(rel="manifest", href="images/manifest.json") + link(rel="mask-icon", href="images/safari-pinned-tab.svg", color="#aa1919") + link(rel="shortcut icon", href="images/favicon.ico") + meta(name="msapplication-config", content="images/browserconfig.xml") + meta(name="theme-color", content="#ffffff") + + link(rel='stylesheet', href='css/vendor.css', type='text/css') + link(rel='stylesheet', href='css/index.css', type='text/css') + + script(src="js/vendor.js") + script(src="js/hs/index.js") + + body(ng-app="flinkApp" ng-strict-di) + #sidebar(ng-class=" { 'sidebar-visible': sidebarVisible } ") + nav.navbar.navbar-inverse.navbar-static-top + .navbar-header + a.navbar-brand(ui-sref="completed-jobs") + img.logo(alt="Apache Flink Dashboard" src="images/flink-logo.png") + a.navbar-brand.navbar-brand-text(ui-sref="completed-jobs") + | Apache Flink Dashboard — End diff – Also rename to `Apache Flink History Server`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105894735

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
          +
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          + private final long refreshIntervalMillis;
          +
          + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) {
          + this.refreshIntervalMillis = refreshIntervalMillis;
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir);
          + if (LOG.isInfoEnabled()) {
          + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath());
          + }
          + }
          + }
          +
          + void start()

          { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + }

          +
          + void stop() {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for
          + * new job archives.
          + */
          + static class JobArchiveFetcherTask extends TimerTask {
          + private final List<HistoryServer.RefreshLocation> refreshDirs;
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final File webTmpDir;
          + private final File webJobDir;
          + private final File webOverviewDir;
          +
          + private static final String JSON_FILE_ENDING = ".json";
          +
          + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + }

          +
          + @Override
          + public void run() {
          + try {
          + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
          + Path refreshDir = refreshLocation.getPath();
          + FileSystem refreshFS = refreshLocation.getFs();
          +
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = refreshFS.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + if (jobArchives == null)

          { + continue; + }

          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + File tmpLocalFile = new File(webTmpDir, jobID);
          + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID);
          + try {
          + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false);
          +
          + JsonNode archive = mapper.readTree(tmpLocalFile);
          + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) {
          + String path = archivePart.get(ArchivedJson.PATH).asText();
          + String json = archivePart.get(ArchivedJson.JSON).asText();
          +
          + File target;
          + if (path.equals("/joboverview"))

          { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + }

          else

          { + target = new File(webDir, path + JSON_FILE_ENDING); + }

          +
          + java.nio.file.Path parent = target.getParentFile().toPath();
          +
          + try

          { + Files.createDirectories(parent); + }

          catch (FileAlreadyExistsException ignored)

          { + // odd, but OK + }

          +
          + java.nio.file.Path targetPath = target.toPath();
          + /**
          + * We overwrite existing files since this may be another attempt at fetching this archive.
          + * Existing files may be incomplete/corrupt.
          + */
          + if (Files.exists(targetPath))

          { + Files.delete(targetPath); + }

          +
          + Files.createFile(target.toPath());
          + try (FileWriter fw = new FileWriter(target))

          { + fw.write(json); + fw.flush(); + }

          + }
          + updateOverview = true;
          + } catch (IOException e) {
          + LOG.error("Failure while fetching/processing job archive.", e);
          — End diff –

          Should we add the job archive file to the error message?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105894735 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); + + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher")); + private final JobArchiveFetcherTask fetcherTask; + private final long refreshIntervalMillis; + + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshIntervalMillis = refreshIntervalMillis; + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir); + if (LOG.isInfoEnabled()) { + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); + } + } + } + + void start() { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + + void stop() { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for + * new job archives. + */ + static class JobArchiveFetcherTask extends TimerTask { + private final List<HistoryServer.RefreshLocation> refreshDirs; + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives; + private final File webDir; + private final File webTmpDir; + private final File webJobDir; + private final File webOverviewDir; + + private static final String JSON_FILE_ENDING = ".json"; + + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + } + + @Override + public void run() { + try { + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { + Path refreshDir = refreshLocation.getPath(); + FileSystem refreshFS = refreshLocation.getFs(); + + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = refreshFS.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + if (jobArchives == null) { + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + File tmpLocalFile = new File(webTmpDir, jobID); + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID); + try { + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false); + + JsonNode archive = mapper.readTree(tmpLocalFile); + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) { + String path = archivePart.get(ArchivedJson.PATH).asText(); + String json = archivePart.get(ArchivedJson.JSON).asText(); + + File target; + if (path.equals("/joboverview")) { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + } else { + target = new File(webDir, path + JSON_FILE_ENDING); + } + + java.nio.file.Path parent = target.getParentFile().toPath(); + + try { + Files.createDirectories(parent); + } catch (FileAlreadyExistsException ignored) { + // odd, but OK + } + + java.nio.file.Path targetPath = target.toPath(); + /** + * We overwrite existing files since this may be another attempt at fetching this archive. + * Existing files may be incomplete/corrupt. + */ + if (Files.exists(targetPath)) { + Files.delete(targetPath); + } + + Files.createFile(target.toPath()); + try (FileWriter fw = new FileWriter(target)) { + fw.write(json); + fw.flush(); + } + } + updateOverview = true; + } catch (IOException e) { + LOG.error("Failure while fetching/processing job archive.", e); — End diff – Should we add the job archive file to the error message?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105870285

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,276 @@
          +/*
          + * 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.history;
          +
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.security.SecurityUtils;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
          +import org.apache.flink.util.FileUtils;
          +import org.apache.flink.util.FlinkException;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.lang.reflect.UndeclaredThrowableException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.UUID;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final HistoryServerArchiveFetcher archiveFetcher;
          +
          + private final SSLContext serverSSLContext;
          + private WebFrontendBootstrap netty;
          +
          + private final Object startupShutdownLock = new Object();
          + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
          + private final Thread shutdownHook;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + // run the job manager
          — End diff –

          Copy paste comment, please adjust

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105870285 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,276 @@ +/* + * 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.history; + +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final HistoryServerArchiveFetcher archiveFetcher; + + private final SSLContext serverSSLContext; + private WebFrontendBootstrap netty; + + private final Object startupShutdownLock = new Object(); + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + private final Thread shutdownHook; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + // run the job manager — End diff – Copy paste comment, please adjust
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105869478

          — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java —
          @@ -0,0 +1,82 @@
          +/*
          + * 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.configuration;
          +
          +import org.apache.flink.annotation.PublicEvolving;
          +
          +import static org.apache.flink.configuration.ConfigOptions.key;
          +
          +/**
          + * The set of configuration options relating to the HistoryServer.
          + */
          +@PublicEvolving
          +public class HistoryServerOptions {
          +
          + /**
          + * The interval at which the HistoryServer polls

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for new archives.
          + */
          + public static final ConfigOption<Long> HISTORY_SERVER_REFRESH_INTERVAL =
          + key("historyserver.refresh-interval")
          + .defaultValue(3000L);
          +
          + /**
          + * Comma-separated list of directories which the HistoryServer polls for new archives.
          + */
          + public static final ConfigOption<String> HISTORY_SERVER_DIRS =
          + key("historyserver.archive.dirs")
          + .noDefaultValue();
          +
          + /**
          + * The local directory used by the HistoryServer web-frontend.
          + */
          + public static final ConfigOption<String> HISTORY_SERVER_WEB_DIR =
          + key("historyserver.web.dir")
          + .noDefaultValue();
          +
          + /**
          + * The address under which the HistoryServer web-frontend is accessible.
          + */
          + public static final ConfigOption<String> HISTORY_SERVER_WEB_ADDRESS =
          + key("historyserver.web.address")
          + .noDefaultValue();
          +
          + /**
          + * The port under which the HistoryServer web-frontend is accessible.
          + */
          + public static final ConfigOption<Integer> HISTORY_SERVER_WEB_PORT =
          + key("historyserver.web.port")
          + .defaultValue(8082);
          +
          + /**
          + * The refresh interval for the HistoryServer web-frontend in milliseconds.
          + */
          + public static final ConfigOption<Long> HISTORY_SERVER_WEB_REFRESH_INTERVAL =
          — End diff –

          I've seen this in the web frontend as well, but I'm wondering what it is exactly. Is it the same interval that is configured in the `index.coffee` file?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105869478 — Diff: flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java — @@ -0,0 +1,82 @@ +/* + * 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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * The set of configuration options relating to the HistoryServer. + */ +@PublicEvolving +public class HistoryServerOptions { + + /** + * The interval at which the HistoryServer polls {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for new archives. + */ + public static final ConfigOption<Long> HISTORY_SERVER_REFRESH_INTERVAL = + key("historyserver.refresh-interval") + .defaultValue(3000L); + + /** + * Comma-separated list of directories which the HistoryServer polls for new archives. + */ + public static final ConfigOption<String> HISTORY_SERVER_DIRS = + key("historyserver.archive.dirs") + .noDefaultValue(); + + /** + * The local directory used by the HistoryServer web-frontend. + */ + public static final ConfigOption<String> HISTORY_SERVER_WEB_DIR = + key("historyserver.web.dir") + .noDefaultValue(); + + /** + * The address under which the HistoryServer web-frontend is accessible. + */ + public static final ConfigOption<String> HISTORY_SERVER_WEB_ADDRESS = + key("historyserver.web.address") + .noDefaultValue(); + + /** + * The port under which the HistoryServer web-frontend is accessible. + */ + public static final ConfigOption<Integer> HISTORY_SERVER_WEB_PORT = + key("historyserver.web.port") + .defaultValue(8082); + + /** + * The refresh interval for the HistoryServer web-frontend in milliseconds. + */ + public static final ConfigOption<Long> HISTORY_SERVER_WEB_REFRESH_INTERVAL = — End diff – I've seen this in the web frontend as well, but I'm wondering what it is exactly. Is it the same interval that is configured in the `index.coffee` file?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105870188

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,276 @@
          +/*
          + * 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.history;
          +
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.security.SecurityUtils;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
          +import org.apache.flink.util.FileUtils;
          +import org.apache.flink.util.FlinkException;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.lang.reflect.UndeclaredThrowableException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.UUID;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +public class HistoryServer {
          — End diff –

          Please add a high level comment describing what the history server does.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105870188 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,276 @@ +/* + * 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.history; + +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HistoryServer { — End diff – Please add a high level comment describing what the history server does.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105898166

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
          +
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          + private final long refreshIntervalMillis;
          +
          + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) {
          + this.refreshIntervalMillis = refreshIntervalMillis;
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir);
          + if (LOG.isInfoEnabled()) {
          + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath());
          + }
          + }
          + }
          +
          + void start()

          { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + }

          +
          + void stop() {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for
          + * new job archives.
          + */
          + static class JobArchiveFetcherTask extends TimerTask {
          + private final List<HistoryServer.RefreshLocation> refreshDirs;
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final File webTmpDir;
          + private final File webJobDir;
          + private final File webOverviewDir;
          +
          + private static final String JSON_FILE_ENDING = ".json";
          +
          + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + }

          +
          + @Override
          + public void run() {
          + try {
          + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
          + Path refreshDir = refreshLocation.getPath();
          + FileSystem refreshFS = refreshLocation.getFs();
          +
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = refreshFS.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + if (jobArchives == null)

          { + continue; + }

          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + File tmpLocalFile = new File(webTmpDir, jobID);
          + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID);
          + try {
          + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false);
          +
          + JsonNode archive = mapper.readTree(tmpLocalFile);
          + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) {
          + String path = archivePart.get(ArchivedJson.PATH).asText();
          + String json = archivePart.get(ArchivedJson.JSON).asText();
          +
          + File target;
          + if (path.equals("/joboverview"))

          { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + }

          else

          { + target = new File(webDir, path + JSON_FILE_ENDING); + }

          +
          + java.nio.file.Path parent = target.getParentFile().toPath();
          +
          + try

          { + Files.createDirectories(parent); + }

          catch (FileAlreadyExistsException ignored) {
          + // odd, but OK
          — End diff –

          :smile: Can you adjust the comment to say that we allow this because it might be a previously created attempt or the tmpDir is out of sync. `Odd` sounds like we don't know what's going on here.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105898166 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); + + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher")); + private final JobArchiveFetcherTask fetcherTask; + private final long refreshIntervalMillis; + + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshIntervalMillis = refreshIntervalMillis; + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir); + if (LOG.isInfoEnabled()) { + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); + } + } + } + + void start() { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + + void stop() { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for + * new job archives. + */ + static class JobArchiveFetcherTask extends TimerTask { + private final List<HistoryServer.RefreshLocation> refreshDirs; + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives; + private final File webDir; + private final File webTmpDir; + private final File webJobDir; + private final File webOverviewDir; + + private static final String JSON_FILE_ENDING = ".json"; + + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + } + + @Override + public void run() { + try { + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { + Path refreshDir = refreshLocation.getPath(); + FileSystem refreshFS = refreshLocation.getFs(); + + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = refreshFS.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + if (jobArchives == null) { + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + File tmpLocalFile = new File(webTmpDir, jobID); + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID); + try { + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false); + + JsonNode archive = mapper.readTree(tmpLocalFile); + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) { + String path = archivePart.get(ArchivedJson.PATH).asText(); + String json = archivePart.get(ArchivedJson.JSON).asText(); + + File target; + if (path.equals("/joboverview")) { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + } else { + target = new File(webDir, path + JSON_FILE_ENDING); + } + + java.nio.file.Path parent = target.getParentFile().toPath(); + + try { + Files.createDirectories(parent); + } catch (FileAlreadyExistsException ignored) { + // odd, but OK — End diff – :smile: Can you adjust the comment to say that we allow this because it might be a previously created attempt or the tmpDir is out of sync. `Odd` sounds like we don't know what's going on here.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105906846

          — Diff: flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerOptions.java —
          @@ -32,6 +32,13 @@
          public static final ConfigOption<Integer> MAX_ATTEMPTS_HISTORY_SIZE =
          key("job-manager.max-attempts-history-size").defaultValue(16);

          + /**
          + * The location where the

          {@link JobManager}

          stores the archives for finished jobs.
          + */
          + public static final ConfigOption<String> ARCHIVE_DIR =
          + key("jobmanager.archive.dir")
          — End diff –

          Should we rename this to be in line with my other proposal to allow future changes? E.g. `jobmanager.archive.fs.dir`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105906846 — Diff: flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerOptions.java — @@ -32,6 +32,13 @@ public static final ConfigOption<Integer> MAX_ATTEMPTS_HISTORY_SIZE = key("job-manager.max-attempts-history-size").defaultValue(16); + /** + * The location where the {@link JobManager} stores the archives for finished jobs. + */ + public static final ConfigOption<String> ARCHIVE_DIR = + key("jobmanager.archive.dir") — End diff – Should we rename this to be in line with my other proposal to allow future changes? E.g. `jobmanager.archive.fs.dir`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105876054

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,276 @@
          +/*
          + * 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.history;
          +
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.security.SecurityUtils;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
          +import org.apache.flink.util.FileUtils;
          +import org.apache.flink.util.FlinkException;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.lang.reflect.UndeclaredThrowableException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.UUID;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final HistoryServerArchiveFetcher archiveFetcher;
          +
          + private final SSLContext serverSSLContext;
          + private WebFrontendBootstrap netty;
          +
          + private final Object startupShutdownLock = new Object();
          + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
          + private final Thread shutdownHook;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + // run the job manager
          + SecurityUtils.install(new SecurityUtils.SecurityConfiguration(flinkConfig));
          +
          + try {
          + SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
          + @Override
          + public Integer call() throws Exception

          { + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + return 0; + }

          + });
          + System.exit(0);
          + } catch (UndeclaredThrowableException ute)

          { + Throwable cause = ute. getUndeclaredThrowable(); + LOG.error("Failed to run HistoryServer.", cause); + cause.printStackTrace(); + System.exit(1); + }

          catch (Exception e)

          { + LOG.error("Failed to run HistoryServer.", e); + e.printStackTrace(); + System.exit(1); + }

          + }
          +
          + public HistoryServer(Configuration config) throws IOException, FlinkException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIRS);
          + if (refreshDirectories == null)

          { + throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_DIRS + " was not configured."); + }

          + List<RefreshLocation> refreshDirs = new ArrayList<>();
          + for (String refreshDirectory : refreshDirectories.split(",")) {
          + try

          { + Path refreshPath = new Path(refreshDirectory); + FileSystem refreshFS = refreshPath.getFileSystem(); + refreshDirs.add(new RefreshLocation(refreshPath, refreshFS)); + }

          catch (Exception e) {
          + // there's most likely something wrong with the path itself, so we ignore it from here on
          + LOG.error("Failed to create Path or FileSystem for directory {}.", refreshDirectory, e);
          + }
          + }
          +
          + long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir);
          +
          + this.shutdownHook = new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + };
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try

          { + Runtime.getRuntime().addShutdownHook(shutdownHook); + }

          catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          + try

          { + new CountDownLatch(1).await(); + }

          catch (InterruptedException e)

          { + Thread.currentThread().interrupt(); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Failure while running HistoryServer.", e); + }

          finally

          { + stop(); + }

          + }
          +
          + // =================================================================================================================
          + // Life-cycle
          + // =================================================================================================================
          + void start() throws IOException, InterruptedException {
          + synchronized (startupShutdownLock) {
          + LOG.info("Starting history server.");
          +
          + Files.createDirectories(webDir.toPath());
          + LOG.info("Using directory {} as local cache.", webDir);
          +
          + Router router = new Router();
          + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir));
          +
          + webDir.mkdirs();
          — End diff –

          Should we make this
          ```
          if (!webDir.exists && !webDir.mkdirs)

          { throw new IOException("Failed to create local directory $webDir"); }

          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105876054 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,276 @@ +/* + * 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.history; + +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final HistoryServerArchiveFetcher archiveFetcher; + + private final SSLContext serverSSLContext; + private WebFrontendBootstrap netty; + + private final Object startupShutdownLock = new Object(); + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + private final Thread shutdownHook; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + // run the job manager + SecurityUtils.install(new SecurityUtils.SecurityConfiguration(flinkConfig)); + + try { + SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() { + @Override + public Integer call() throws Exception { + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + return 0; + } + }); + System.exit(0); + } catch (UndeclaredThrowableException ute) { + Throwable cause = ute. getUndeclaredThrowable(); + LOG.error("Failed to run HistoryServer.", cause); + cause.printStackTrace(); + System.exit(1); + } catch (Exception e) { + LOG.error("Failed to run HistoryServer.", e); + e.printStackTrace(); + System.exit(1); + } + } + + public HistoryServer(Configuration config) throws IOException, FlinkException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIRS); + if (refreshDirectories == null) { + throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_DIRS + " was not configured."); + } + List<RefreshLocation> refreshDirs = new ArrayList<>(); + for (String refreshDirectory : refreshDirectories.split(",")) { + try { + Path refreshPath = new Path(refreshDirectory); + FileSystem refreshFS = refreshPath.getFileSystem(); + refreshDirs.add(new RefreshLocation(refreshPath, refreshFS)); + } catch (Exception e) { + // there's most likely something wrong with the path itself, so we ignore it from here on + LOG.error("Failed to create Path or FileSystem for directory {}.", refreshDirectory, e); + } + } + + long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir); + + this.shutdownHook = new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }; + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(shutdownHook); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { + try { + new CountDownLatch(1).await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (Exception e) { + LOG.error("Failure while running HistoryServer.", e); + } finally { + stop(); + } + } + + // ================================================================================================================= + // Life-cycle + // ================================================================================================================= + void start() throws IOException, InterruptedException { + synchronized (startupShutdownLock) { + LOG.info("Starting history server."); + + Files.createDirectories(webDir.toPath()); + LOG.info("Using directory {} as local cache.", webDir); + + Router router = new Router(); + router.GET("/:*", new HistoryServerStaticFileServerHandler(webDir)); + + webDir.mkdirs(); — End diff – Should we make this ``` if (!webDir.exists && !webDir.mkdirs) { throw new IOException("Failed to create local directory $webDir"); } ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105893756

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
          +
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          + private final long refreshIntervalMillis;
          +
          + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) {
          + this.refreshIntervalMillis = refreshIntervalMillis;
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir);
          + if (LOG.isInfoEnabled()) {
          + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath());
          + }
          + }
          + }
          +
          + void start()

          { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + }

          +
          + void stop() {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for
          + * new job archives.
          + */
          + static class JobArchiveFetcherTask extends TimerTask {
          + private final List<HistoryServer.RefreshLocation> refreshDirs;
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final File webTmpDir;
          + private final File webJobDir;
          + private final File webOverviewDir;
          +
          + private static final String JSON_FILE_ENDING = ".json";
          +
          + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + }

          +
          + @Override
          + public void run() {
          + try {
          + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
          + Path refreshDir = refreshLocation.getPath();
          + FileSystem refreshFS = refreshLocation.getFs();
          +
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = refreshFS.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + if (jobArchives == null)

          { + continue; + }

          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          — End diff –

          We could make the map `<JobID, Path>` and try to parse the file name via `JobID.fromHexString` to make sure that this is an expected file.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105893756 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); + + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher")); + private final JobArchiveFetcherTask fetcherTask; + private final long refreshIntervalMillis; + + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshIntervalMillis = refreshIntervalMillis; + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir); + if (LOG.isInfoEnabled()) { + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); + } + } + } + + void start() { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + + void stop() { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for + * new job archives. + */ + static class JobArchiveFetcherTask extends TimerTask { + private final List<HistoryServer.RefreshLocation> refreshDirs; + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives; + private final File webDir; + private final File webTmpDir; + private final File webJobDir; + private final File webOverviewDir; + + private static final String JSON_FILE_ENDING = ".json"; + + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + } + + @Override + public void run() { + try { + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { + Path refreshDir = refreshLocation.getPath(); + FileSystem refreshFS = refreshLocation.getFs(); + + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = refreshFS.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + if (jobArchives == null) { + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); — End diff – We could make the map `<JobID, Path>` and try to parse the file name via `JobID.fromHexString` to make sure that this is an expected file.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105907455

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala —
          @@ -183,6 +198,45 @@ class MemoryArchivist(private val max_entries: Int)
          }
          }

          + private def archiveJsonFiles(graph: ArchivedExecutionGraph) {
          + future {
          + try {
          + val rootPath = new Path(flinkConfiguration.getString(
          + JobManagerOptions.ARCHIVE_DIR))
          + val fs = rootPath.getFileSystem
          + val path = new Path(rootPath, s"$

          {graph.getJobID.toString}

          ")
          + val out = fs.create(path, WriteMode.NO_OVERWRITE)
          +
          + try {
          + val gen = jacksonFactory.createGenerator(out, JsonEncoding.UTF8)
          + try {
          + gen.writeStartObject()
          + gen.writeArrayFieldStart(ArchivedJson.ARCHIVE)
          + for (archiver <- WebMonitorUtils.getJsonArchivists) {
          + for (archive <- archiver.archiveJsonWithPath(graph).asScala)

          { + gen.writeStartObject() + gen.writeStringField(ArchivedJson.PATH, archive.getPath) + gen.writeStringField(ArchivedJson.JSON, archive.getJson) + gen.writeEndObject() + }

          + }
          + gen.writeEndArray()
          + gen.writeEndObject()
          — End diff –

          Should we add a log message that the job has been archived in the path XYZ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105907455 — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala — @@ -183,6 +198,45 @@ class MemoryArchivist(private val max_entries: Int) } } + private def archiveJsonFiles(graph: ArchivedExecutionGraph) { + future { + try { + val rootPath = new Path(flinkConfiguration.getString( + JobManagerOptions.ARCHIVE_DIR)) + val fs = rootPath.getFileSystem + val path = new Path(rootPath, s"$ {graph.getJobID.toString} ") + val out = fs.create(path, WriteMode.NO_OVERWRITE) + + try { + val gen = jacksonFactory.createGenerator(out, JsonEncoding.UTF8) + try { + gen.writeStartObject() + gen.writeArrayFieldStart(ArchivedJson.ARCHIVE) + for (archiver <- WebMonitorUtils.getJsonArchivists) { + for (archive <- archiver.archiveJsonWithPath(graph).asScala) { + gen.writeStartObject() + gen.writeStringField(ArchivedJson.PATH, archive.getPath) + gen.writeStringField(ArchivedJson.JSON, archive.getJson) + gen.writeEndObject() + } + } + gen.writeEndArray() + gen.writeEndObject() — End diff – Should we add a log message that the job has been archived in the path XYZ?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105893041

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
          +
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          + private final long refreshIntervalMillis;
          +
          + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) {
          + this.refreshIntervalMillis = refreshIntervalMillis;
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir);
          + if (LOG.isInfoEnabled()) {
          + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath());
          + }
          + }
          + }
          +
          + void start()

          { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + }

          +
          + void stop() {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for
          + * new job archives.
          + */
          + static class JobArchiveFetcherTask extends TimerTask {
          + private final List<HistoryServer.RefreshLocation> refreshDirs;
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives;
          — End diff –

          In the long run, we should make the size of this map bounded with a high default to bound the memory consumption.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105893041 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); + + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher")); + private final JobArchiveFetcherTask fetcherTask; + private final long refreshIntervalMillis; + + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshIntervalMillis = refreshIntervalMillis; + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir); + if (LOG.isInfoEnabled()) { + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); + } + } + } + + void start() { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + + void stop() { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for + * new job archives. + */ + static class JobArchiveFetcherTask extends TimerTask { + private final List<HistoryServer.RefreshLocation> refreshDirs; + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives; — End diff – In the long run, we should make the size of this map bounded with a high default to bound the memory consumption.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105904946

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/AbstractStaticFileServerHandler.java —
          @@ -0,0 +1,372 @@
          +/*
          + * 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 org.slf4j.LoggerFactory;
          +
          +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.
          + *
          + * This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server
          + * example.
          + *
          + * For every incoming requests the

          {@link Routed#path()}

          is pre-processed in
          + *

          {@link AbstractStaticFileServerHandler#preProcessRequestPath(String)}

          .
          + *
          + * This path is then interpreted as a relative file path, with the configured rootDir being the parent.
          + *
          + * If no file exists for this path, another (optional) pre-processing step is executed in
          + *

          {@link AbstractStaticFileServerHandler#preProcessFilePath(String)}

          .
          + * We then try to load a file for the resulting file path using the

          {@link ClassLoader}

          .
          + */
          +public abstract class AbstractStaticFileServerHandler extends SimpleChannelInboundHandler<Routed> {
          — End diff –

          Something subtle must have changed here calling `http://localhost:8081/index.html` works but `http://localhost:8081` does not. Please add a specific test after fixing.

          I get the following Exception:

          ```
          2017-03-14 14:21:24,635 ERROR org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler - error while responding
          java.nio.file.FileAlreadyExistsException: /private/var/folders/py/s_1l8vln6f19ygc77m8c4zhr0000gn/T/flink-web-13c5ce04-53d8-4a50-9f67-5b126f02a81e
          at sun.nio.fs.UnixException.translateToIOException(UnixException.java:88)
          at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
          at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
          at sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:214)
          at java.nio.file.spi.FileSystemProvider.newOutputStream(FileSystemProvider.java:434)
          at java.nio.file.Files.newOutputStream(Files.java:216)
          at java.nio.file.Files.copy(Files.java:3016)
          at org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler.respondWithFile(AbstractStaticFileServerHandler.java:157)
          at org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.handleRequest(StaticFileServerHandler.java:102)
          at org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler.channelRead0(AbstractStaticFileServerHandler.java:123)
          at org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler.channelRead0(AbstractStaticFileServerHandler.java:96)
          at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
          at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
          at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
          at io.netty.handler.codec.http.router.Handler.routed(Handler.java:62)
          at io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:57)
          at io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:20)
          at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
          at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
          at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
          at org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(HttpRequestHandler.java:105)
          at org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(HttpRequestHandler.java:65)
          at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
          at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
          at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
          at io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
          at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
          at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
          at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242)
          at io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:147)
          at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
          at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
          at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847)
          at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
          at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
          at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
          at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
          at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
          at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
          at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
          at java.lang.Thread.run(Thread.java:745)
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105904946 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/AbstractStaticFileServerHandler.java — @@ -0,0 +1,372 @@ +/* + * 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 org.slf4j.LoggerFactory; + +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. + * + * This code is based on the "HttpStaticFileServerHandler" from the Netty project's HTTP server + * example. + * + * For every incoming requests the {@link Routed#path()} is pre-processed in + * {@link AbstractStaticFileServerHandler#preProcessRequestPath(String)} . + * + * This path is then interpreted as a relative file path, with the configured rootDir being the parent. + * + * If no file exists for this path, another (optional) pre-processing step is executed in + * {@link AbstractStaticFileServerHandler#preProcessFilePath(String)} . + * We then try to load a file for the resulting file path using the {@link ClassLoader} . + */ +public abstract class AbstractStaticFileServerHandler extends SimpleChannelInboundHandler<Routed> { — End diff – Something subtle must have changed here calling ` http://localhost:8081/index.html ` works but ` http://localhost:8081 ` does not. Please add a specific test after fixing. I get the following Exception: ``` 2017-03-14 14:21:24,635 ERROR org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler - error while responding java.nio.file.FileAlreadyExistsException: /private/var/folders/py/s_1l8vln6f19ygc77m8c4zhr0000gn/T/flink-web-13c5ce04-53d8-4a50-9f67-5b126f02a81e at sun.nio.fs.UnixException.translateToIOException(UnixException.java:88) at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) at sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:214) at java.nio.file.spi.FileSystemProvider.newOutputStream(FileSystemProvider.java:434) at java.nio.file.Files.newOutputStream(Files.java:216) at java.nio.file.Files.copy(Files.java:3016) at org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler.respondWithFile(AbstractStaticFileServerHandler.java:157) at org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.handleRequest(StaticFileServerHandler.java:102) at org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler.channelRead0(AbstractStaticFileServerHandler.java:123) at org.apache.flink.runtime.webmonitor.files.AbstractStaticFileServerHandler.channelRead0(AbstractStaticFileServerHandler.java:96) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) at io.netty.handler.codec.http.router.Handler.routed(Handler.java:62) at io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:57) at io.netty.handler.codec.http.router.DualAbstractHandler.channelRead0(DualAbstractHandler.java:20) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) at org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(HttpRequestHandler.java:105) at org.apache.flink.runtime.webmonitor.HttpRequestHandler.channelRead0(HttpRequestHandler.java:65) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) at io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242) at io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:147) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111) at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137) at java.lang.Thread.run(Thread.java:745) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105878628

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala —
          @@ -247,8 +248,11 @@ class LocalFlinkMiniCluster(
          // Props for the distributed components
          //------------------------------------------------------------------------------------------------

          • def getArchiveProps(archiveClass: Class[_ <: MemoryArchivist], archiveCount: Int): Props = {
          • JobManager.getArchiveProps(archiveClass, archiveCount)
            + def getArchiveProps(
            + archiveClass: Class[_ <: MemoryArchivist],
              • End diff –

          I think the arguments should have one more level of indentation

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105878628 — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala — @@ -247,8 +248,11 @@ class LocalFlinkMiniCluster( // Props for the distributed components //------------------------------------------------------------------------------------------------ def getArchiveProps(archiveClass: Class [_ <: MemoryArchivist] , archiveCount: Int): Props = { JobManager.getArchiveProps(archiveClass, archiveCount) + def getArchiveProps( + archiveClass: Class [_ <: MemoryArchivist] , End diff – I think the arguments should have one more level of indentation
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105899831

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
          +
          + private static final JsonFactory jacksonFactory = new JsonFactory();
          + private static final ObjectMapper mapper = new ObjectMapper();
          +
          + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(
          + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
          + private final JobArchiveFetcherTask fetcherTask;
          + private final long refreshIntervalMillis;
          +
          + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) {
          + this.refreshIntervalMillis = refreshIntervalMillis;
          + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir);
          + if (LOG.isInfoEnabled()) {
          + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
          + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath());
          + }
          + }
          + }
          +
          + void start()

          { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + }

          +
          + void stop() {
          + executor.shutdown();
          +
          + try {
          + if (!executor.awaitTermination(1, TimeUnit.SECONDS))

          { + executor.shutdownNow(); + }

          + } catch (InterruptedException ignored)

          { + executor.shutdownNow(); + }

          + }
          +
          + /**
          + *

          {@link TimerTask}

          that polls the directories configured as

          {@link HistoryServerOptions#HISTORY_SERVER_DIRS}

          for
          + * new job archives.
          + */
          + static class JobArchiveFetcherTask extends TimerTask {
          + private final List<HistoryServer.RefreshLocation> refreshDirs;
          + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */
          + private final Map<String, Path> cachedArchives;
          + private final File webDir;
          + private final File webTmpDir;
          + private final File webJobDir;
          + private final File webOverviewDir;
          +
          + private static final String JSON_FILE_ENDING = ".json";
          +
          + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir)

          { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + }

          +
          + @Override
          + public void run() {
          + try {
          + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
          + Path refreshDir = refreshLocation.getPath();
          + FileSystem refreshFS = refreshLocation.getFs();
          +
          + // contents of /:refreshDir
          + FileStatus[] jobArchives;
          + try

          { + jobArchives = refreshFS.listStatus(refreshDir); + }

          catch (IOException e) {
          + LOG.error("Failed to access job archive location for path {}.", refreshDir, e);
          + continue;
          + }
          + if (jobArchives == null)

          { + continue; + }

          + boolean updateOverview = false;
          + for (FileStatus jobArchive : jobArchives) {
          + Path jobArchivePath = jobArchive.getPath();
          + String jobID = jobArchivePath.getName();
          + if (cachedArchives.put(jobID, refreshDir) == null) {
          + File tmpLocalFile = new File(webTmpDir, jobID);
          + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID);
          + try {
          + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false);
          +
          + JsonNode archive = mapper.readTree(tmpLocalFile);
          + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) {
          + String path = archivePart.get(ArchivedJson.PATH).asText();
          + String json = archivePart.get(ArchivedJson.JSON).asText();
          +
          + File target;
          + if (path.equals("/joboverview"))

          { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + }

          else

          { + target = new File(webDir, path + JSON_FILE_ENDING); + }

          +
          + java.nio.file.Path parent = target.getParentFile().toPath();
          +
          + try

          { + Files.createDirectories(parent); + }

          catch (FileAlreadyExistsException ignored)

          { + // odd, but OK + }

          +
          + java.nio.file.Path targetPath = target.toPath();
          + /**
          + * We overwrite existing files since this may be another attempt at fetching this archive.
          + * Existing files may be incomplete/corrupt.
          + */
          + if (Files.exists(targetPath))

          { + Files.delete(targetPath); + }

          +
          + Files.createFile(target.toPath());
          + try (FileWriter fw = new FileWriter(target))

          { + fw.write(json); + fw.flush(); + }

          + }
          + updateOverview = true;
          + } catch (IOException e) {
          + LOG.error("Failure while fetching/processing job archive.", e);
          + /** Make sure we attempt to fetch the archive again */
          + cachedArchives.remove(jobID);
          + /** Make sure we do not include this job in the overview */
          + Files.delete(new File(webOverviewDir, jobID + JSON_FILE_ENDING).toPath());
          + /** Clean up job files we may have created */
          + File jobDirectory = new File(webJobDir, jobID);
          + try

          { + FileUtils.deleteDirectory(jobDirectory); + }

          catch (IOException ioe)

          { + LOG.debug("Could not clean up job-directory.", ioe); + }

          + } finally {
          + try

          { + Files.delete(tmpLocalFile.toPath()); + }

          + catch (NoSuchFileException e)

          { + // if the file is already gone (concurrently), we don't mind + }

          + }
          + }
          + }
          + if (updateOverview)

          { + updateJobOverview(webDir); + }

          + }
          + } catch (Exception e)

          { + LOG.error("Critical failure while fetching/processing job archives.", e); + }

          + }
          + }
          +
          + /**
          + * This method replicates the JSON response that would be given by the

          {@link CurrentJobsOverviewHandler}

          when
          + * listing both running and finished jobs.
          + *
          + * Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on
          + * their own however the list of finished jobs only contains a single job.
          + *
          + * For the display in the HistoryServer WebFrontend we have to combine these overviews.
          + */
          + private static void updateJobOverview(File webDir) {
          — End diff –

          Why don't we simply count the number of jobs that we successfully add to the cached map? Then we don't need to write a file for each update. We would need to add a simple handler for this though.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105899831 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); + + private static final JsonFactory jacksonFactory = new JsonFactory(); + private static final ObjectMapper mapper = new ObjectMapper(); + + private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher")); + private final JobArchiveFetcherTask fetcherTask; + private final long refreshIntervalMillis; + + HistoryServerArchiveFetcher(long refreshIntervalMillis, List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshIntervalMillis = refreshIntervalMillis; + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir); + if (LOG.isInfoEnabled()) { + for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { + LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); + } + } + } + + void start() { + executor.scheduleWithFixedDelay(fetcherTask, 0, refreshIntervalMillis, TimeUnit.MILLISECONDS); + } + + void stop() { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException ignored) { + executor.shutdownNow(); + } + } + + /** + * {@link TimerTask} that polls the directories configured as {@link HistoryServerOptions#HISTORY_SERVER_DIRS} for + * new job archives. + */ + static class JobArchiveFetcherTask extends TimerTask { + private final List<HistoryServer.RefreshLocation> refreshDirs; + /** Map containing the JobID of all fetched jobs and the refreshDir from with they originate. */ + private final Map<String, Path> cachedArchives; + private final File webDir; + private final File webTmpDir; + private final File webJobDir; + private final File webOverviewDir; + + private static final String JSON_FILE_ENDING = ".json"; + + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + } + + @Override + public void run() { + try { + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { + Path refreshDir = refreshLocation.getPath(); + FileSystem refreshFS = refreshLocation.getFs(); + + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = refreshFS.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + if (jobArchives == null) { + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + File tmpLocalFile = new File(webTmpDir, jobID); + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID); + try { + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false); + + JsonNode archive = mapper.readTree(tmpLocalFile); + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) { + String path = archivePart.get(ArchivedJson.PATH).asText(); + String json = archivePart.get(ArchivedJson.JSON).asText(); + + File target; + if (path.equals("/joboverview")) { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + } else { + target = new File(webDir, path + JSON_FILE_ENDING); + } + + java.nio.file.Path parent = target.getParentFile().toPath(); + + try { + Files.createDirectories(parent); + } catch (FileAlreadyExistsException ignored) { + // odd, but OK + } + + java.nio.file.Path targetPath = target.toPath(); + /** + * We overwrite existing files since this may be another attempt at fetching this archive. + * Existing files may be incomplete/corrupt. + */ + if (Files.exists(targetPath)) { + Files.delete(targetPath); + } + + Files.createFile(target.toPath()); + try (FileWriter fw = new FileWriter(target)) { + fw.write(json); + fw.flush(); + } + } + updateOverview = true; + } catch (IOException e) { + LOG.error("Failure while fetching/processing job archive.", e); + /** Make sure we attempt to fetch the archive again */ + cachedArchives.remove(jobID); + /** Make sure we do not include this job in the overview */ + Files.delete(new File(webOverviewDir, jobID + JSON_FILE_ENDING).toPath()); + /** Clean up job files we may have created */ + File jobDirectory = new File(webJobDir, jobID); + try { + FileUtils.deleteDirectory(jobDirectory); + } catch (IOException ioe) { + LOG.debug("Could not clean up job-directory.", ioe); + } + } finally { + try { + Files.delete(tmpLocalFile.toPath()); + } + catch (NoSuchFileException e) { + // if the file is already gone (concurrently), we don't mind + } + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } catch (Exception e) { + LOG.error("Critical failure while fetching/processing job archives.", e); + } + } + } + + /** + * This method replicates the JSON response that would be given by the {@link CurrentJobsOverviewHandler} when + * listing both running and finished jobs. + * + * Every job archive contains a joboverview.json file containing the same structure. Since jobs are archived on + * their own however the list of finished jobs only contains a single job. + * + * For the display in the HistoryServer WebFrontend we have to combine these overviews. + */ + private static void updateJobOverview(File webDir) { — End diff – Why don't we simply count the number of jobs that we successfully add to the cached map? Then we don't need to write a file for each update. We would need to add a simple handler for this though.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105898701

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java —
          @@ -0,0 +1,244 @@
          +/*
          + * 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.history;
          +
          +import com.fasterxml.jackson.core.JsonFactory;
          +import com.fasterxml.jackson.core.JsonGenerator;
          +import com.fasterxml.jackson.databind.JsonNode;
          +import com.fasterxml.jackson.databind.ObjectMapper;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileStatus;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.filecache.FileCache;
          +import org.apache.flink.runtime.util.ExecutorThreadFactory;
          +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler;
          +import org.apache.flink.util.FileUtils;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.nio.file.FileAlreadyExistsException;
          +import java.nio.file.Files;
          +import java.nio.file.NoSuchFileException;
          +import java.util.HashMap;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.TimerTask;
          +import java.util.concurrent.Executors;
          +import java.util.concurrent.ScheduledExecutorService;
          +import java.util.concurrent.TimeUnit;
          +
          +public class HistoryServerArchiveFetcher {
          — End diff –

          Please add a high level comment describing what the fetcher does mentioning that we copy files from the archive dirs to the local dirs etc.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105898701 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java — @@ -0,0 +1,244 @@ +/* + * 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.history; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.webmonitor.handlers.CurrentJobsOverviewHandler; +import org.apache.flink.util.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class HistoryServerArchiveFetcher { — End diff – Please add a high level comment describing what the fetcher does mentioning that we copy files from the archive dirs to the local dirs etc.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105874859

          — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java —
          @@ -0,0 +1,276 @@
          +/*
          + * 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.history;
          +
          +import io.netty.handler.codec.http.router.Router;
          +import org.apache.flink.api.java.utils.ParameterTool;
          +import org.apache.flink.configuration.Configuration;
          +import org.apache.flink.configuration.GlobalConfiguration;
          +import org.apache.flink.configuration.HistoryServerOptions;
          +import org.apache.flink.core.fs.FileSystem;
          +import org.apache.flink.core.fs.Path;
          +import org.apache.flink.runtime.net.SSLUtils;
          +import org.apache.flink.runtime.security.SecurityUtils;
          +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler;
          +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap;
          +import org.apache.flink.util.FileUtils;
          +import org.apache.flink.util.FlinkException;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import javax.net.ssl.SSLContext;
          +import java.io.File;
          +import java.io.FileWriter;
          +import java.io.IOException;
          +import java.lang.reflect.UndeclaredThrowableException;
          +import java.nio.file.Files;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.UUID;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.CountDownLatch;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +public class HistoryServer {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
          +
          + private final String webAddress;
          + private final int webPort;
          + private final long webRefreshIntervalMillis;
          + private final File webDir;
          +
          + private final HistoryServerArchiveFetcher archiveFetcher;
          +
          + private final SSLContext serverSSLContext;
          + private WebFrontendBootstrap netty;
          +
          + private final Object startupShutdownLock = new Object();
          + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
          + private final Thread shutdownHook;
          +
          + public static void main(String[] args) throws Exception {
          + ParameterTool pt = ParameterTool.fromArgs(args);
          + String configDir = pt.getRequired("configDir");
          +
          + LOG.info("Loading configuration from {}", configDir);
          + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir);
          +
          + // run the job manager
          + SecurityUtils.install(new SecurityUtils.SecurityConfiguration(flinkConfig));
          +
          + try {
          + SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
          + @Override
          + public Integer call() throws Exception

          { + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + return 0; + }

          + });
          + System.exit(0);
          + } catch (UndeclaredThrowableException ute)

          { + Throwable cause = ute. getUndeclaredThrowable(); + LOG.error("Failed to run HistoryServer.", cause); + cause.printStackTrace(); + System.exit(1); + }

          catch (Exception e)

          { + LOG.error("Failed to run HistoryServer.", e); + e.printStackTrace(); + System.exit(1); + }

          + }
          +
          + public HistoryServer(Configuration config) throws IOException, FlinkException {
          + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) {
          + LOG.info("Enabling ssl for the history server.");
          + try

          { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + }

          catch (Exception e)

          { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + }

          + } else

          { + this.serverSSLContext = null; + }

          +
          + webAddress = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_ADDRESS);
          + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT);
          + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL);
          +
          + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR);
          + if (webDirectory == null)

          { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + }

          + webDir = new File(webDirectory);
          +
          + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIRS);
          + if (refreshDirectories == null)

          { + throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_DIRS + " was not configured."); + }

          + List<RefreshLocation> refreshDirs = new ArrayList<>();
          + for (String refreshDirectory : refreshDirectories.split(",")) {
          + try

          { + Path refreshPath = new Path(refreshDirectory); + FileSystem refreshFS = refreshPath.getFileSystem(); + refreshDirs.add(new RefreshLocation(refreshPath, refreshFS)); + }

          catch (Exception e) {
          + // there's most likely something wrong with the path itself, so we ignore it from here on
          + LOG.error("Failed to create Path or FileSystem for directory {}.", refreshDirectory, e);
          + }
          + }
          +
          + long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL);
          + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir);
          +
          + this.shutdownHook = new Thread() {
          + @Override
          + public void run()

          { + HistoryServer.this.stop(); + }

          + };
          + // add shutdown hook for deleting the directories and remaining temp files on shutdown
          + try

          { + Runtime.getRuntime().addShutdownHook(shutdownHook); + }

          catch (IllegalStateException e)

          { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + }

          catch (Throwable t)

          { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + }

          + }
          +
          + public void run() {
          + try {
          + start();
          + while (true) {
          — End diff –

          We don't need the while loop and catch block here. There are no spurious wakeups with the `CountDownLatch` and reacting to an interrupt in the main Thread is good I think.

          The `while-catch-interrupt` pattern here leads to an endless loop.

          ```java
          Thread t = new Thread() {
          @Override
          public void run() {
          CountDownLatch latch = new CountDownLatch(1);
          while (true) {
          try

          { latch.await(); }

          catch (InterruptedException e)

          { e.printStackTrace(); Thread.currentThread().interrupt(); }

          }
          }
          };

          t.start();

          while (!t.isAlive()) {
          }

          t.interrupt();

          t.join(); // Thread never terminates interruption loop
          ```

          The one thing to remember is that interruptions are cooperative and you have to manually check and respect the interrupted flag.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://github.com/apache/flink/pull/3460#discussion_r105874859 — Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java — @@ -0,0 +1,276 @@ +/* + * 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.history; + +import io.netty.handler.codec.http.router.Router; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HistoryServerOptions; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.net.SSLUtils; +import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.webmonitor.handlers.DashboardConfigHandler; +import org.apache.flink.runtime.webmonitor.utils.WebFrontendBootstrap; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; + +public class HistoryServer { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class); + + private final String webAddress; + private final int webPort; + private final long webRefreshIntervalMillis; + private final File webDir; + + private final HistoryServerArchiveFetcher archiveFetcher; + + private final SSLContext serverSSLContext; + private WebFrontendBootstrap netty; + + private final Object startupShutdownLock = new Object(); + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + private final Thread shutdownHook; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + String configDir = pt.getRequired("configDir"); + + LOG.info("Loading configuration from {}", configDir); + final Configuration flinkConfig = GlobalConfiguration.loadConfiguration(configDir); + + // run the job manager + SecurityUtils.install(new SecurityUtils.SecurityConfiguration(flinkConfig)); + + try { + SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() { + @Override + public Integer call() throws Exception { + HistoryServer hs = new HistoryServer(flinkConfig); + hs.run(); + return 0; + } + }); + System.exit(0); + } catch (UndeclaredThrowableException ute) { + Throwable cause = ute. getUndeclaredThrowable(); + LOG.error("Failed to run HistoryServer.", cause); + cause.printStackTrace(); + System.exit(1); + } catch (Exception e) { + LOG.error("Failed to run HistoryServer.", e); + e.printStackTrace(); + System.exit(1); + } + } + + public HistoryServer(Configuration config) throws IOException, FlinkException { + if (config.getBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED) && SSLUtils.getSSLEnabled(config)) { + LOG.info("Enabling ssl for the history server."); + try { + this.serverSSLContext = SSLUtils.createSSLServerContext(config); + } catch (Exception e) { + throw new IOException("Failed to initialize SSLContext for the history server.", e); + } + } else { + this.serverSSLContext = null; + } + + webAddress = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_ADDRESS); + webPort = config.getInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT); + webRefreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_WEB_REFRESH_INTERVAL); + + String webDirectory = config.getString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR); + if (webDirectory == null) { + webDirectory = System.getProperty("java.io.tmpdir") + "flink-web-history-" + UUID.randomUUID(); + } + webDir = new File(webDirectory); + + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_DIRS); + if (refreshDirectories == null) { + throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_DIRS + " was not configured."); + } + List<RefreshLocation> refreshDirs = new ArrayList<>(); + for (String refreshDirectory : refreshDirectories.split(",")) { + try { + Path refreshPath = new Path(refreshDirectory); + FileSystem refreshFS = refreshPath.getFileSystem(); + refreshDirs.add(new RefreshLocation(refreshPath, refreshFS)); + } catch (Exception e) { + // there's most likely something wrong with the path itself, so we ignore it from here on + LOG.error("Failed to create Path or FileSystem for directory {}.", refreshDirectory, e); + } + } + + long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_REFRESH_INTERVAL); + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir); + + this.shutdownHook = new Thread() { + @Override + public void run() { + HistoryServer.this.stop(); + } + }; + // add shutdown hook for deleting the directories and remaining temp files on shutdown + try { + Runtime.getRuntime().addShutdownHook(shutdownHook); + } catch (IllegalStateException e) { + // race, JVM is in shutdown already, we can safely ignore this + LOG.debug("Unable to add shutdown hook, shutdown already in progress", e); + } catch (Throwable t) { + // these errors usually happen when the shutdown is already in progress + LOG.warn("Error while adding shutdown hook", t); + } + } + + public void run() { + try { + start(); + while (true) { — End diff – We don't need the while loop and catch block here. There are no spurious wakeups with the `CountDownLatch` and reacting to an interrupt in the main Thread is good I think. The `while-catch-interrupt` pattern here leads to an endless loop. ```java Thread t = new Thread() { @Override public void run() { CountDownLatch latch = new CountDownLatch(1); while (true) { try { latch.await(); } catch (InterruptedException e) { e.printStackTrace(); Thread.currentThread().interrupt(); } } } }; t.start(); while (!t.isAlive()) { } t.interrupt(); t.join(); // Thread never terminates interruption loop ``` The one thing to remember is that interruptions are cooperative and you have to manually check and respect the interrupted flag.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3460#discussion_r105905806

          — Diff: flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala —
          @@ -183,6 +198,45 @@ class MemoryArchivist(private val max_entries: Int)
          }
          }

          + private def archiveJsonFiles(graph: ArchivedExecutionGraph) {
          + future {
          + try {
          + val rootPath = new Path(flinkConfiguration.getString(
          — End diff –

          Since this has no default value, we need to check whether the path is null. You should check this before submitting the future to safe work when no directory is configured.

          I get the following Exception in the logs:

          ```
          2017-03-14 14:24:41,953 ERROR org.apache.flink.runtime.jobmanager.MemoryArchivist - Failed to archive job.
          java.lang.IllegalArgumentException: Can not create a Path from a null string
          at org.apache.flink.core.fs.Path.checkAndTrimPathArg(Path.java:159)
          at org.apache.flink.core.fs.Path.<init>(Path.java:176)
          at org.apache.flink.runtime.jobmanager.MemoryArchivist$$anonfun$org$apache$flink$runtime$jobmanager$MemoryArchivist$$archiveJsonFiles$1.apply$mcV$sp(MemoryArchivist.scala:204)
          at org.apache.flink.runtime.jobmanager.MemoryArchivist$$anonfun$org$apache$flink$runtime$jobmanager$MemoryArchivist$$archiveJsonFiles$1.apply(MemoryArchivist.scala:203)
          at org.apache.flink.runtime.jobmanager.MemoryArchivist$$anonfun$org$apache$flink$runtime$jobmanager$MemoryArchivist$$archiveJsonFiles$1.apply(MemoryArchivist.scala:203)
          at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
          at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
          at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
          at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
          at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
          at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
          at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
          at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on a diff in the pull request: https://gith