You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2018/05/15 07:52:11 UTC
[04/12] flink git commit: [FLINK-9246][HS] Adjust HistoryServer for
job overview changes
[FLINK-9246][HS] Adjust HistoryServer for job overview changes
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2cef5fde
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2cef5fde
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2cef5fde
Branch: refs/heads/master
Commit: 2cef5fded65575eb16511271c5820a992890195b
Parents: bcd028d
Author: hzyuqi1 <hz...@corp.netease.com>
Authored: Tue Apr 24 10:25:10 2018 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Mon May 14 23:40:48 2018 +0200
----------------------------------------------------------------------
.../history/HistoryServerArchiveFetcher.java | 16 +++++++---------
1 file changed, 7 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2cef5fde/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
index 450436f..413473b 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java
@@ -25,6 +25,7 @@ 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.rest.handler.legacy.JobsOverviewHandler;
+import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.util.FileUtils;
@@ -162,7 +163,7 @@ class HistoryServerArchiveFetcher {
String json = archive.getJson();
File target;
- if (path.equals("/joboverview")) {
+ if (path.equals(JobsOverviewHeaders.URL)) {
target = new File(webOverviewDir, jobID + JSON_FILE_ENDING);
} else {
target = new File(webDir, path + JSON_FILE_ENDING);
@@ -211,7 +212,7 @@ class HistoryServerArchiveFetcher {
}
}
if (updateOverview) {
- updateJobOverview(webDir);
+ updateJobOverview(webOverviewDir, webDir);
}
}
} catch (Exception e) {
@@ -230,19 +231,16 @@ class HistoryServerArchiveFetcher {
*
* <p>For the display in the HistoryServer WebFrontend we have to combine these overviews.
*/
- private static void updateJobOverview(File webDir) {
- File webOverviewDir = new File(webDir, "overviews");
- try (JsonGenerator gen = jacksonFactory.createGenerator(HistoryServer.createOrGetFile(webDir, "joboverview"))) {
+ private static void updateJobOverview(File webOverviewDir, File webDir) {
+ try (JsonGenerator gen = jacksonFactory.createGenerator(HistoryServer.createOrGetFile(webDir, JobsOverviewHeaders.URL))) {
gen.writeStartObject();
- gen.writeArrayFieldStart("running");
- gen.writeEndArray();
- gen.writeArrayFieldStart("finished");
+ gen.writeArrayFieldStart("jobs");
File[] overviews = new File(webOverviewDir.getPath()).listFiles();
if (overviews != null) {
for (File overview : overviews) {
JsonNode root = mapper.readTree(overview);
- JsonNode finished = root.get("finished");
+ JsonNode finished = root.get("jobs");
JsonNode job = finished.get(0);
mapper.writeTree(gen, job);
}