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

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

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.


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



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

Reply via email to