[ 
https://issues.apache.org/jira/browse/FLINK-1579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15924354#comment-15924354
 ] 

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_r105929511
  
    --- 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);
    +                   }
    +           }
    +   }
    +
    +   /**
    +    * This method replicates the JSON response that would be given by the 
{@link CurrentJobsOverviewHandler} when
    +    * listing both running and finished jobs.
    +    *
    +    * Every job archive contains a joboverview.json file containing the 
same structure. Since jobs are archived on
    +    * their own however the list of finished jobs only contains a single 
job.
    +    *
    +    * For the display in the HistoryServer WebFrontend we have to combine 
these overviews.
    +    */
    +   private static void updateJobOverview(File webDir) {
    --- End diff --
    
    Ok sorry! I forgot about my previous comment. What do you think about 
adding a handler? If it's not too much overhead, it might lead to less moving 
parts. We can also do this as a follow up if you don't want to do it 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)

Reply via email to