[
https://issues.apache.org/jira/browse/FLINK-1579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15932970#comment-15932970
]
ASF GitHub Bot commented on FLINK-1579:
---------------------------------------
Github user uce commented on a diff in the pull request:
https://github.com/apache/flink/pull/3460#discussion_r106948356
--- Diff:
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
---
@@ -0,0 +1,242 @@
+/*
+ * 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.api.common.JobID;
+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.history.FsJobArchivist;
+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.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TimerTask;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class is used by the {@link HistoryServer} to fetch the job
archives that are located at
+ * {@link HistoryServerOptions#HISTORY_SERVER_ARCHIVE_DIRS}. The
directories are polled in regular intervals, defined
+ * by {@link HistoryServerOptions#HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL}.
+ *
+ * The archives are downloaded and expanded into a file structure analog
to the REST API defined in the WebRuntimeMonitor.
+ */
+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_ARCHIVE_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. */
--- End diff --
This comment is outdated now
> 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)