You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/28 12:02:01 UTC

[GitHub] [flink] xintongsong commented on a diff in pull request #21185: [FLINK-28643][runtime-web] HistoryServer support lazy unzip

xintongsong commented on code in PR #21185:
URL: https://github.com/apache/flink/pull/21185#discussion_r1033366920


##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java:
##########
@@ -340,15 +364,24 @@ void stop() {
                     LOG.warn("Error while shutting down WebFrontendBootstrap.", t);
                 }
 
-                ExecutorUtils.gracefulShutdown(1, TimeUnit.SECONDS, executor);
-
+                ExecutorUtils.gracefulShutdown(1, TimeUnit.MINUTES, fetcherExecutor, unzipExecutor);
                 try {
-                    LOG.info("Removing web dashboard root cache directory {}", webDir);
-                    FileUtils.deleteDirectory(webDir);
+                    LOG.info("Removing web dashboard cached WebFrontend files in dir {}", webDir);
+                    for (java.nio.file.Path path : FileUtils.listDirectory(webDir.toPath())) {
+                        if ((Files.isDirectory(path)
+                                        && path.toFile().getName().equals(ARCHIVED_JOBS_DIR))
+                                || (Files.isDirectory(path)
+                                        && path.toFile().getName().equals(JOBS_DIR))
+                                || (Files.isDirectory(path)
+                                        && path.toFile().getName().equals(OVERVIEWS_DIR))) {
+                            continue;
+                        }

Review Comment:
   Why do we want to skip cleaning the cache files on termination?



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java:
##########
@@ -116,9 +123,17 @@ public class HistoryServer {
     private WebFrontendBootstrap netty;
 
     private final long refreshIntervalMillis;
-    private final ScheduledExecutorService executor =
+    private final ScheduledExecutorService fetcherExecutor =
             Executors.newSingleThreadScheduledExecutor(
                     new ExecutorThreadFactory("Flink-HistoryServer-ArchiveFetcher"));
+    private final ExecutorService unzipExecutor =
+            new ThreadPoolExecutor(
+                    8,
+                    32,

Review Comment:
   I wonder if it makes sense to make these configurable.



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java:
##########
@@ -75,16 +86,26 @@ class HistoryServerArchiveFetcher {
 
     /** Possible job archive operations in history-server. */
     public enum ArchiveEventType {
-        /** Job archive was found in one refresh location and created in history server. */
+        /** Archived job file was found in one refresh location and downloaded in history server. */
+        DOWNLOADED,
+        /** Unzipped Job archive was reloaded. */
+        RELOADED,

Review Comment:
   What are the differences between downloaded and reloaded?



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java:
##########
@@ -102,6 +106,9 @@ public class HistoryServer {
 
     private static final Logger LOG = LoggerFactory.getLogger(HistoryServer.class);
     private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper();
+    private static final String ARCHIVED_JOBS_DIR = "archivedJobs";
+    private static final String JOBS_DIR = "jobs";
+    private static final String OVERVIEWS_DIR = "overviews";

Review Comment:
   It was unclear to me what are these directories for, until diving deeper into the codes. The readability can be improved by documenting them explicitly. 



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java:
##########
@@ -152,10 +202,68 @@ public ArchiveEventType getType() {
         }
     }
 
+    private void initJobCache() {
+        initArchivedJobCache();
+        initUnzippedJobCache();
+    }
+
+    private void initArchivedJobCache() {
+        if (this.webArchivedDir.list() == null) {
+            LOG.info("No legacy archived jobs");
+            return;
+        }
+        Set<String> jobInLocal =
+                Arrays.stream(this.webArchivedDir.list()).collect(Collectors.toSet());
+        LOG.info("Reload left archived jobs : [{}]", String.join(",", jobInLocal));
+
+        for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) {
+            Path refreshDir = refreshLocation.getPath();
+            try {
+                FileStatus[] jobArchives = listArchives(refreshLocation.getFs(), refreshDir);
+                Set<String> jobInRefreshLocation =
+                        Arrays.stream(jobArchives)
+                                .map(FileStatus::getPath)
+                                .map(Path::getName)
+                                .collect(Collectors.toSet());
+                jobInRefreshLocation.retainAll(jobInLocal);
+                this.cachedArchivesPerRefreshDirectory.get(refreshDir).addAll(jobInRefreshLocation);
+            } catch (IOException e) {
+                LOG.error("Failed to reload archivedJobs in {}.", refreshDir, refreshDir, e);
+            }
+        }
+
+        for (String jobId : Objects.requireNonNull(this.webArchivedDir.list())) {
+            this.cachedArchivesPerRefreshDirectory.forEach((path, archives) -> archives.add(jobId));
+        }

Review Comment:
   Why do we want to add all local archives to caches of all refresh directoreis?



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java:
##########
@@ -75,16 +86,26 @@ class HistoryServerArchiveFetcher {
 
     /** Possible job archive operations in history-server. */
     public enum ArchiveEventType {
-        /** Job archive was found in one refresh location and created in history server. */
+        /** Archived job file was found in one refresh location and downloaded in history server. */
+        DOWNLOADED,
+        /** Unzipped Job archive was reloaded. */
+        RELOADED,
+        /**
+         * Archived job file was unzipped and Unzipped Job archive was created in history server.
+         */
         CREATED,
+        /** Unzipped Job archive was deleted in history server. */
+        CLEANED,
         /**
-         * Job archive was deleted from one of refresh locations and deleted from history server.
+         * Archived job file and Unzipped Job archive was deleted from one of refresh locations and
+         * deleted from history server.
          */
-        DELETED
+        DELETED,
     }
 
     /** Representation of job archive event. */
     public static class ArchiveEvent {
+

Review Comment:
   It seems this event is no long being used. We probably can just get rid of it.



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java:
##########
@@ -103,47 +124,76 @@ public ArchiveEventType getType() {
     }
 
     private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
-
     private static final JsonFactory jacksonFactory = new JsonFactory();
     private static final ObjectMapper mapper = JacksonMapperFactory.createObjectMapper();
 
     private static final String JSON_FILE_ENDING = ".json";
 
     private final List<HistoryServer.RefreshLocation> refreshDirs;
+    private final List<ArchiveEvent> events;
     private final Consumer<ArchiveEvent> jobArchiveEventListener;
     private final boolean processExpiredArchiveDeletion;
     private final boolean processBeyondLimitArchiveDeletion;
     private final int maxHistorySize;
 
     /** Cache of all available jobs identified by their id. */
     private final Map<Path, Set<String>> cachedArchivesPerRefreshDirectory;
+    /** Cache of all unzipped jobs. key: jobID */
+    private final LoadingCache<String, Boolean> unzippedJobCache;
 
     private final File webDir;
     private final File webJobDir;
+    private final File webArchivedDir;
     private final File webOverviewDir;
 
     HistoryServerArchiveFetcher(
             List<HistoryServer.RefreshLocation> refreshDirs,
             File webDir,
             Consumer<ArchiveEvent> jobArchiveEventListener,
             boolean cleanupExpiredArchives,
-            int maxHistorySize)
+            int maxHistorySize,
+            int maxCachedJobSize)
             throws IOException {
         this.refreshDirs = checkNotNull(refreshDirs);
+        this.events = Collections.synchronizedList(new ArrayList<>());
         this.jobArchiveEventListener = jobArchiveEventListener;
         this.processExpiredArchiveDeletion = cleanupExpiredArchives;
         this.maxHistorySize = maxHistorySize;
         this.processBeyondLimitArchiveDeletion = this.maxHistorySize > 0;
         this.cachedArchivesPerRefreshDirectory = new HashMap<>();
+        this.unzippedJobCache =
+                CacheBuilder.newBuilder()
+                        .concurrencyLevel(10)
+                        .initialCapacity(10)
+                        .maximumSize(maxCachedJobSize)
+                        .expireAfterAccess(7L, TimeUnit.DAYS)
+                        .removalListener(
+                                notification -> {
+                                    LOG.info(
+                                            "Job:{} is removed from cache with reason [{}]",
+                                            notification.getKey(),
+                                            notification.getCause());
+                                    deleteJobFiles((String) notification.getKey());
+                                })
+                        .build(
+                                new CacheLoader<String, Boolean>() {
+                                    @Override
+                                    public Boolean load(String s) throws IOException {
+                                        return unzipArchive(s);
+                                    }
+                                });
         for (HistoryServer.RefreshLocation refreshDir : refreshDirs) {
             cachedArchivesPerRefreshDirectory.put(refreshDir.getPath(), new HashSet<>());
         }
         this.webDir = checkNotNull(webDir);
+        this.webArchivedDir = new File(webDir, "archivedJobs");
+        Files.createDirectories(webArchivedDir.toPath());
         this.webJobDir = new File(webDir, "jobs");
         Files.createDirectories(webJobDir.toPath());
         this.webOverviewDir = new File(webDir, "overviews");
         Files.createDirectories(webOverviewDir.toPath());

Review Comment:
   These should refer to the same constants as in `HistoryServer`, rather than string literals.



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java:
##########
@@ -103,47 +124,76 @@ public ArchiveEventType getType() {
     }
 
     private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class);
-
     private static final JsonFactory jacksonFactory = new JsonFactory();
     private static final ObjectMapper mapper = JacksonMapperFactory.createObjectMapper();
 
     private static final String JSON_FILE_ENDING = ".json";
 
     private final List<HistoryServer.RefreshLocation> refreshDirs;
+    private final List<ArchiveEvent> events;
     private final Consumer<ArchiveEvent> jobArchiveEventListener;
     private final boolean processExpiredArchiveDeletion;
     private final boolean processBeyondLimitArchiveDeletion;
     private final int maxHistorySize;
 
     /** Cache of all available jobs identified by their id. */
     private final Map<Path, Set<String>> cachedArchivesPerRefreshDirectory;
+    /** Cache of all unzipped jobs. key: jobID */
+    private final LoadingCache<String, Boolean> unzippedJobCache;

Review Comment:
   What does the boolean value stand for?



##########
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerStaticFileServerHandler.java:
##########
@@ -166,10 +195,38 @@ private void respondWithFile(ChannelHandlerContext ctx, HttpRequest request, Str
                             }
                         }
                     }
+                    // try to unzip archived files
+                    if (enableUnzip && !success) {
+                        // extract jobid from requestPath
+                        String jobId = extractJobId(requestPath);
+                        if (!StringUtils.isNullOrWhitespaceOnly(jobId)) {
+                            // submit unzip Task and get future
+                            Boolean unzipped =
+                                    CompletableFuture.supplyAsync(
+                                                    unzipTask.apply(jobId), unzipExecutor)

Review Comment:
   It's weird some unzipping happens on the unzipExecutor, while others (those in the fetcher) are not.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org