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.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---