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.
--- 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. ---