[ https://issues.apache.org/jira/browse/FLINK-1579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15924351#comment-15924351 ]
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_r105929296 --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java --- @@ -0,0 +1,244 @@ +/* + * 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.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.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.nio.file.NoSuchFileException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +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_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. */ + private final Map<String, Path> cachedArchives; + private final File webDir; + private final File webTmpDir; + private final File webJobDir; + private final File webOverviewDir; + + private static final String JSON_FILE_ENDING = ".json"; + + JobArchiveFetcherTask(List<HistoryServer.RefreshLocation> refreshDirs, File webDir) { + this.refreshDirs = refreshDirs; + this.cachedArchives = new HashMap<>(); + this.webDir = webDir; + this.webTmpDir = new File(webDir, "tmp"); + webTmpDir.mkdir(); + this.webJobDir = new File(webDir, "jobs"); + webJobDir.mkdir(); + this.webOverviewDir = new File(webDir, "overviews"); + webOverviewDir.mkdir(); + } + + @Override + public void run() { + try { + for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { + Path refreshDir = refreshLocation.getPath(); + FileSystem refreshFS = refreshLocation.getFs(); + + // contents of /:refreshDir + FileStatus[] jobArchives; + try { + jobArchives = refreshFS.listStatus(refreshDir); + } catch (IOException e) { + LOG.error("Failed to access job archive location for path {}.", refreshDir, e); + continue; + } + if (jobArchives == null) { + continue; + } + boolean updateOverview = false; + for (FileStatus jobArchive : jobArchives) { + Path jobArchivePath = jobArchive.getPath(); + String jobID = jobArchivePath.getName(); + if (cachedArchives.put(jobID, refreshDir) == null) { + File tmpLocalFile = new File(webTmpDir, jobID); + Path tmpLocalPath = new Path(webTmpDir.getAbsolutePath(), jobID); + try { + FileCache.copy(jobArchive.getPath(), tmpLocalPath, false); + + JsonNode archive = mapper.readTree(tmpLocalFile); + for (JsonNode archivePart : archive.get(ArchivedJson.ARCHIVE)) { + String path = archivePart.get(ArchivedJson.PATH).asText(); + String json = archivePart.get(ArchivedJson.JSON).asText(); + + File target; + if (path.equals("/joboverview")) { + target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); + } else { + target = new File(webDir, path + JSON_FILE_ENDING); + } + + java.nio.file.Path parent = target.getParentFile().toPath(); + + try { + Files.createDirectories(parent); + } catch (FileAlreadyExistsException ignored) { + // odd, but OK + } + + java.nio.file.Path targetPath = target.toPath(); + /** + * We overwrite existing files since this may be another attempt at fetching this archive. + * Existing files may be incomplete/corrupt. + */ + if (Files.exists(targetPath)) { + Files.delete(targetPath); + } + + Files.createFile(target.toPath()); + try (FileWriter fw = new FileWriter(target)) { + fw.write(json); + fw.flush(); + } + } + updateOverview = true; + } catch (IOException e) { + LOG.error("Failure while fetching/processing job archive.", e); + /** Make sure we attempt to fetch the archive again */ + cachedArchives.remove(jobID); + /** Make sure we do not include this job in the overview */ + Files.delete(new File(webOverviewDir, jobID + JSON_FILE_ENDING).toPath()); + /** Clean up job files we may have created */ + File jobDirectory = new File(webJobDir, jobID); + try { + FileUtils.deleteDirectory(jobDirectory); + } catch (IOException ioe) { + LOG.debug("Could not clean up job-directory.", ioe); + } + } finally { + try { + Files.delete(tmpLocalFile.toPath()); + } + catch (NoSuchFileException e) { + // if the file is already gone (concurrently), we don't mind + } + } + } + } + if (updateOverview) { + updateJobOverview(webDir); + } + } + } catch (Exception e) { + LOG.error("Critical failure while fetching/processing job archives.", e); --- End diff -- I thought that the Exception affects the task state, which it doesn't. 👍 > 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)