You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@streampark.apache.org by GitBox <gi...@apache.org> on 2022/10/26 14:49:24 UTC

[GitHub] [incubator-streampark] 1996fanrui commented on a diff in pull request #1910: [improve]Precision state judgment based on archive logs

1996fanrui commented on code in PR #1910:
URL: https://github.com/apache/incubator-streampark/pull/1910#discussion_r1005738232


##########
streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ApplicationServiceImpl.java:
##########
@@ -1361,6 +1368,11 @@ public void start(Application appParam, boolean auto) throws Exception {
         } else {
             if (ExecutionMode.isKubernetesApplicationMode(application.getExecutionMode())) {
                 AssertUtils.state(buildResult != null);
+                String flinkConfPath = String.format("%s/conf/flink-conf.yaml", flinkEnv.getFlinkHome());
+                ParameterTool parameter = ParameterTool.fromPropertiesFile(flinkConfPath);

Review Comment:
   Just check `flink-conf.yaml`? Should we check properties as well? 



##########
streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ApplicationServiceImpl.java:
##########
@@ -1361,6 +1368,11 @@ public void start(Application appParam, boolean auto) throws Exception {
         } else {
             if (ExecutionMode.isKubernetesApplicationMode(application.getExecutionMode())) {
                 AssertUtils.state(buildResult != null);
+                String flinkConfPath = String.format("%s/conf/flink-conf.yaml", flinkEnv.getFlinkHome());
+                ParameterTool parameter = ParameterTool.fromPropertiesFile(flinkConfPath);
+                String fsPath = Optional.ofNullable(parameter.get("jobmanager.archive.fs.dir")).orElse(archivePath);

Review Comment:
   Should we use `JobManagerOptions.ARCHIVE_DIR.key()` here?



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {

Review Comment:
   How about rename `fetchArchives`  to `getJobStateFromArchiveFile`?



##########
streampark-common/src/main/scala/org/apache/streampark/common/conf/ConfigConst.scala:
##########
@@ -112,6 +112,8 @@ object ConfigConst {
 
   val KEY_FLINK_DEPLOYMENT_OPTION_PREFIX = "flink.deployment.option."
 
+  val KEY_JOBMANAGER_ARCHIVE_FS_DIR = "jobmanager.archive.fs.dir"

Review Comment:
   This code can be deleted, we can use `JobManagerOptions.ARCHIVE_DIR.key()` directly.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);
+            Path refreshDir = refreshLocation.getPath();
+            // contents of /:refreshDir
+            FileStatus[] jobArchives;
+            jobArchives = listArchives(refreshLocation.getFs(), refreshDir);
+            Path jobArchivePath = Arrays.stream(jobArchives)
+                .map(FileStatus::getPath)
+                .filter(path -> path.getName().equals(jobId)).findFirst().orElse(null);
+
+            Objects.requireNonNull(jobArchivePath, "Archive Log Path Exception");
+            for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchivePath)) {
+                String path = archive.getPath();
+                System.out.println(path);
+                if (path.equals("/jobs/overview")) {
+                    String json = archive.getJson();
+                    ObjectMapper objectMapper = new ObjectMapper();
+                    Jobs jobs = objectMapper.readValue(json, Jobs.class);
+                    System.out.println(json);
+                    List<Overview> overviews = jobs.getJobs();
+                    return overviews.get(0).getState();

Review Comment:
   If the session mode is used, `jobs.getJobs();` have some jobs.
   
   In general, we should check `overview.getJid` here.



##########
streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ApplicationServiceImpl.java:
##########
@@ -1361,6 +1368,11 @@ public void start(Application appParam, boolean auto) throws Exception {
         } else {
             if (ExecutionMode.isKubernetesApplicationMode(application.getExecutionMode())) {
                 AssertUtils.state(buildResult != null);
+                String flinkConfPath = String.format("%s/conf/flink-conf.yaml", flinkEnv.getFlinkHome());

Review Comment:
   You can refactor `FlinkEnv.doSetFlinkConf()`. After refactor, you can call `flinkEnv.getFlinkConf()` directly.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);
+            Path refreshDir = refreshLocation.getPath();
+            // contents of /:refreshDir
+            FileStatus[] jobArchives;
+            jobArchives = listArchives(refreshLocation.getFs(), refreshDir);
+            Path jobArchivePath = Arrays.stream(jobArchives)
+                .map(FileStatus::getPath)
+                .filter(path -> path.getName().equals(jobId)).findFirst().orElse(null);
+
+            Objects.requireNonNull(jobArchivePath, "Archive Log Path Exception");
+            for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchivePath)) {
+                String path = archive.getPath();
+                System.out.println(path);
+                if (path.equals("/jobs/overview")) {
+                    String json = archive.getJson();
+                    ObjectMapper objectMapper = new ObjectMapper();
+                    Jobs jobs = objectMapper.readValue(json, Jobs.class);
+                    System.out.println(json);

Review Comment:
   This line can be deleted.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);
+            Path refreshDir = refreshLocation.getPath();
+            // contents of /:refreshDir
+            FileStatus[] jobArchives;
+            jobArchives = listArchives(refreshLocation.getFs(), refreshDir);
+            Path jobArchivePath = Arrays.stream(jobArchives)
+                .map(FileStatus::getPath)
+                .filter(path -> path.getName().equals(jobId)).findFirst().orElse(null);
+
+            Objects.requireNonNull(jobArchivePath, "Archive Log Path Exception");
+            for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchivePath)) {
+                String path = archive.getPath();
+                System.out.println(path);

Review Comment:
   This line can be deleted.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);
+            Path refreshDir = refreshLocation.getPath();
+            // contents of /:refreshDir
+            FileStatus[] jobArchives;
+            jobArchives = listArchives(refreshLocation.getFs(), refreshDir);
+            Path jobArchivePath = Arrays.stream(jobArchives)
+                .map(FileStatus::getPath)
+                .filter(path -> path.getName().equals(jobId)).findFirst().orElse(null);
+
+            Objects.requireNonNull(jobArchivePath, "Archive Log Path Exception");
+            for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchivePath)) {
+                String path = archive.getPath();
+                System.out.println(path);
+                if (path.equals("/jobs/overview")) {
+                    String json = archive.getJson();
+                    ObjectMapper objectMapper = new ObjectMapper();
+                    Jobs jobs = objectMapper.readValue(json, Jobs.class);
+                    System.out.println(json);
+                    List<Overview> overviews = jobs.getJobs();
+                    return overviews.get(0).getState();
+                }
+                return "FAILED";
+            }
+            return "FAILED";
+        } catch (Exception e) {
+            return "FAILED";

Review Comment:
   Too many `"FAILED"`, this can be extract a static field.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/Task.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.streampark.archives;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+public class Task {

Review Comment:
   We didn't use the task, so we can remove the task in this version?
   
   And we can add it if it's needed in the future.



##########
streampark-flink/streampark-flink-kubernetes/pom.xml:
##########
@@ -96,6 +96,24 @@
             <version>${snakeyaml.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <optional>true</optional>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <optional>true</optional>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <optional>true</optional>
+        </dependency>

Review Comment:
   Are all dependencies necessary?
   
   In your kubernetes env, you use hdfs as your archive storage, so you add these dependencies, right?
   
   If yes.  If some users use s3, it will miss some jar, right?



##########
streampark-console/streampark-console-service/src/main/resources/application.yml:
##########
@@ -91,6 +91,8 @@ streampark:
   workspace:
     local: /opt/streampark_workspace
     remote: hdfs://hdfscluster/streampark   # support hdfs:///streampark/ 、 /streampark 、hdfs://host:ip/streampark/
+  archive:
+    path: hdfs:///streampark/completed-jobs/

Review Comment:
   The archive path shouldn't be configured. It should under the `workspace.remote`. What do you think?



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);
+            Path refreshDir = refreshLocation.getPath();
+            // contents of /:refreshDir
+            FileStatus[] jobArchives;
+            jobArchives = listArchives(refreshLocation.getFs(), refreshDir);

Review Comment:
   These 2 lines can be merged to 1 line.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);
+            Path refreshDir = refreshLocation.getPath();
+            // contents of /:refreshDir
+            FileStatus[] jobArchives;
+            jobArchives = listArchives(refreshLocation.getFs(), refreshDir);
+            Path jobArchivePath = Arrays.stream(jobArchives)
+                .map(FileStatus::getPath)
+                .filter(path -> path.getName().equals(jobId)).findFirst().orElse(null);

Review Comment:
   How about list the `archivePath/jobId` dir directly?
   
   I don't think list the archivePath is necessary.



##########
streampark-flink/streampark-flink-kubernetes/src/main/java/org/apache/streampark/archives/FetchArchives.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.streampark.archives;
+
+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.webmonitor.history.ArchivedJson;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class FetchArchives {
+
+    public static String fetchArchives(String jobId, String archivePath) {
+        try {
+            Objects.requireNonNull(jobId, "JobId cannot be empty.");
+            Objects.requireNonNull(archivePath, "archivePath cannot be empty.");
+            Path refreshPath = new Path(archivePath);
+            FileSystem refreshFS = refreshPath.getFileSystem();
+            RefreshLocation refreshLocation = new RefreshLocation(refreshPath, refreshFS);

Review Comment:
   Why need the `RefreshLocation`?
   
   The `refreshPath.getFileSystem()` can be used directly.



-- 
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@streampark.apache.org

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