[
https://issues.apache.org/jira/browse/FLINK-1579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15899236#comment-15899236
]
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_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.
> 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)