You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by GitBox <gi...@apache.org> on 2022/10/26 09:23:42 UTC

[GitHub] [incubator-seatunnel] ic4y opened a new pull request, #3191: [feature][st-engine] Add jobHistory

ic4y opened a new pull request, #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191

   <!--
   
   Thank you for contributing to SeaTunnel! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [GITHUB issue](https://github.com/apache/incubator-seatunnel/issues).
   
     - Name the pull request in the form "[Feature] [component] Title of the pull request", where *Feature* can be replaced by `Hotfix`, `Bug`, etc.
   
     - Minor fixes should be named following this pattern: `[hotfix] [docs] Fix typo in README.md doc`.
   
   -->
   
   ## Purpose of this pull request
   
   <!-- Describe the purpose of this pull request. For example: This pull request adds checkstyle plugin.-->
   
   ## Check list
   
   Add jobHistory.
   
   * [ ] Code changed are covered with tests, or it does not need tests for reason:
   * [ ] If any new Jar binary package adding in your PR, please add License Notice according
     [New License Guide](https://github.com/apache/incubator-seatunnel/blob/dev/docs/en/contribution/new-license.md)
   * [ ] If necessary, please update the documentation to describe the new feature. https://github.com/apache/incubator-seatunnel/tree/dev/docs
   


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] Hisoka-X commented on a diff in pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
Hisoka-X commented on code in PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#discussion_r1013561374


##########
seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/AbstractCommandArgs.java:
##########
@@ -29,8 +29,7 @@
 public abstract class AbstractCommandArgs implements CommandArgs {
 
     @Parameter(names = {"-c", "--config"},
-        description = "Config file",
-        required = true)

Review Comment:
   If removed `required=true`, the other command without config also will pass the command check. It will cause some unpredictable errors. Maybe you should remove `--config` in `AbstractCommandArgs`.



##########
seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/args/ClientCommandArgs.java:
##########
@@ -41,6 +41,14 @@ public class ClientCommandArgs extends AbstractCommandArgs {
         description = "The name of cluster")
     private String clusterName = "seatunnel_default_cluster";
 
+    @Parameter(names = {"-j", "--job-id"},

Review Comment:
   I can't know what can do according the description of the `-j` .



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java:
##########
@@ -133,6 +136,17 @@ public CoordinatorService(@NonNull NodeEngineImpl nodeEngine, @NonNull SeaTunnel
         this.engineConfig = engineConfig;
         masterActiveListener = Executors.newSingleThreadScheduledExecutor();
         masterActiveListener.scheduleAtFixedRate(this::checkNewActiveMaster, 0, 100, TimeUnit.MILLISECONDS);
+
+        jobHistoryService = new JobHistoryService(

Review Comment:
   Why not put it into `init` method? If you create new `JobHistoryService` in here, every node will have this service.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.seatunnel.engine.server.master;
+
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.execution.ExecutionState;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.map.IMap;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+public class JobHistoryService {
+    /**
+     * IMap key is one of jobId {@link org.apache.seatunnel.engine.server.dag.physical.PipelineLocation} and
+     * {@link org.apache.seatunnel.engine.server.execution.TaskGroupLocation}
+     * <p>
+     * The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
+     * {@link org.apache.seatunnel.engine.server.execution.ExecutionState}
+     * <p>
+     * This IMap is used to recovery runningJobStateIMap in JobMaster when a new master node active
+     */
+    private final IMap<Object, Object> runningJobStateIMap;
+
+    private final ILogger logger;
+
+    /**
+     * key: job id;
+     * <br> value: job master;
+     */
+    private final Map<Long, JobMaster> runningJobMasterMap;
+
+    /**
+     * finishedJobStateImap key is jobId and value is jobState(json)
+     * JobStateMapper Indicates the status of the job, pipeline, and task
+     */
+    //TODO need to limit the amount of storage
+    private final IMap<Long, JobStateMapper> finishedJobStateImap;
+
+    public JobHistoryService(
+        IMap<Object, Object> runningJobStateIMap,
+        ILogger logger,
+        Map<Long, JobMaster> runningJobMasterMap,
+        IMap<Long, JobStateMapper> finishedJobStateImap
+    ) {
+        this.runningJobStateIMap = runningJobStateIMap;
+        this.logger = logger;
+        this.runningJobMasterMap = runningJobMasterMap;
+        this.finishedJobStateImap = finishedJobStateImap;
+    }
+
+    // Gets the status of a running and completed job
+    public String listAllJob() {
+        ObjectMapper objectMapper = new ObjectMapper();

Review Comment:
   `ObjectMapper` is thread-safe, you can put it in your fields.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.seatunnel.engine.server.master;
+
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.execution.ExecutionState;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.map.IMap;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+public class JobHistoryService {
+    /**
+     * IMap key is one of jobId {@link org.apache.seatunnel.engine.server.dag.physical.PipelineLocation} and
+     * {@link org.apache.seatunnel.engine.server.execution.TaskGroupLocation}
+     * <p>
+     * The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
+     * {@link org.apache.seatunnel.engine.server.execution.ExecutionState}
+     * <p>
+     * This IMap is used to recovery runningJobStateIMap in JobMaster when a new master node active
+     */
+    private final IMap<Object, Object> runningJobStateIMap;
+
+    private final ILogger logger;
+
+    /**
+     * key: job id;
+     * <br> value: job master;
+     */
+    private final Map<Long, JobMaster> runningJobMasterMap;
+
+    /**
+     * finishedJobStateImap key is jobId and value is jobState(json)
+     * JobStateMapper Indicates the status of the job, pipeline, and task
+     */
+    //TODO need to limit the amount of storage
+    private final IMap<Long, JobStateMapper> finishedJobStateImap;
+
+    public JobHistoryService(
+        IMap<Object, Object> runningJobStateIMap,
+        ILogger logger,
+        Map<Long, JobMaster> runningJobMasterMap,
+        IMap<Long, JobStateMapper> finishedJobStateImap
+    ) {
+        this.runningJobStateIMap = runningJobStateIMap;
+        this.logger = logger;
+        this.runningJobMasterMap = runningJobMasterMap;
+        this.finishedJobStateImap = finishedJobStateImap;
+    }
+
+    // Gets the status of a running and completed job
+    public String listAllJob() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+        ArrayNode jobs = objectNode.putArray("jobs");
+
+        Stream.concat(runningJobMasterMap.values().stream().map(this::toJobStateMapper),
+                finishedJobStateImap.values().stream())
+            .forEach(jobStateMapper -> {
+                JobStatusMapper jobStatusMapper = new JobStatusMapper(jobStateMapper.jobId, jobStateMapper.jobStatus);
+                JsonNode jsonNode = objectMapper.valueToTree(jobStatusMapper);
+                jobs.add(jsonNode);
+            });
+        return jobs.toString();
+    }
+
+    // Get detailed status of a single job
+    public JobStateMapper getJobStatus(Long jobId) {
+        return runningJobMasterMap.containsKey(jobId) ? toJobStateMapper(runningJobMasterMap.get(jobId)) :
+            finishedJobStateImap.getOrDefault(jobId, null);
+    }
+
+    // Get detailed status of a single job as json
+    public String getJobStatusAsString(Long jobId) {
+        ObjectMapper objectMapper = new ObjectMapper();
+        JobStateMapper jobStatus = getJobStatus(jobId);
+        if (null != jobStatus) {
+            try {
+                return objectMapper.writeValueAsString(jobStatus);
+            } catch (JsonProcessingException e) {
+                logger.severe("serialize jobStateMapper err", e);
+                ObjectNode objectNode = objectMapper.createObjectNode();
+                objectNode.put("err", "serialize jobStateMapper err");
+                return objectNode.toString();
+            }
+        }
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectNode.put("err", String.format("jobId : %s not found", jobId));
+        return objectNode.toString();
+    }
+
+    @SuppressWarnings("checkstyle:MagicNumber")
+    public void storeFinishedJobState(JobMaster jobMaster) {
+        JobStateMapper jobStateMapper = toJobStateMapper(jobMaster);
+        finishedJobStateImap.put(jobStateMapper.jobId, jobStateMapper, 14, TimeUnit.DAYS);
+    }
+
+    private JobStateMapper toJobStateMapper(JobMaster jobMaster) {
+
+        Long jobId = jobMaster.getJobImmutableInformation().getJobId();
+        Map<PipelineLocation, PipelineStateMapper> pipelineStateMapperMap = new HashMap<>();
+
+        jobMaster.getPhysicalPlan().getPipelineList().forEach(pipeline -> {
+            PipelineLocation pipelineLocation = pipeline.getPipelineLocation();
+            PipelineStatus pipelineState = (PipelineStatus) runningJobStateIMap.get(pipelineLocation);
+            Map<TaskGroupLocation, ExecutionState> taskStateMap = new HashMap<>();
+            pipeline.getCoordinatorVertexList().forEach(coordinator -> {
+                TaskGroupLocation taskGroupLocation = coordinator.getTaskGroupLocation();
+                taskStateMap.put(taskGroupLocation, (ExecutionState) runningJobStateIMap.get(taskGroupLocation));
+            });
+            pipeline.getPhysicalVertexList().forEach(task -> {
+                TaskGroupLocation taskGroupLocation = task.getTaskGroupLocation();
+                taskStateMap.put(taskGroupLocation, (ExecutionState) runningJobStateIMap.get(taskGroupLocation));
+            });
+
+            PipelineStateMapper pipelineStateMapper = new PipelineStateMapper(pipelineState, taskStateMap);
+            pipelineStateMapperMap.put(pipelineLocation, pipelineStateMapper);
+        });
+        JobStatus jobStatus = (JobStatus) runningJobStateIMap.get(jobId);
+
+        return new JobStateMapper(jobId, jobStatus, pipelineStateMapperMap);
+    }
+
+    @AllArgsConstructor
+    @Data
+    public static final class JobStatusMapper implements Serializable {

Review Comment:
   It is strange named `Mapper` cause there are not map anything



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.seatunnel.engine.server.master;
+
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.execution.ExecutionState;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.map.IMap;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+public class JobHistoryService {
+    /**
+     * IMap key is one of jobId {@link org.apache.seatunnel.engine.server.dag.physical.PipelineLocation} and
+     * {@link org.apache.seatunnel.engine.server.execution.TaskGroupLocation}
+     * <p>
+     * The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
+     * {@link org.apache.seatunnel.engine.server.execution.ExecutionState}
+     * <p>
+     * This IMap is used to recovery runningJobStateIMap in JobMaster when a new master node active
+     */
+    private final IMap<Object, Object> runningJobStateIMap;
+
+    private final ILogger logger;
+
+    /**
+     * key: job id;
+     * <br> value: job master;
+     */
+    private final Map<Long, JobMaster> runningJobMasterMap;
+
+    /**
+     * finishedJobStateImap key is jobId and value is jobState(json)
+     * JobStateMapper Indicates the status of the job, pipeline, and task
+     */
+    //TODO need to limit the amount of storage
+    private final IMap<Long, JobStateMapper> finishedJobStateImap;
+
+    public JobHistoryService(
+        IMap<Object, Object> runningJobStateIMap,
+        ILogger logger,
+        Map<Long, JobMaster> runningJobMasterMap,
+        IMap<Long, JobStateMapper> finishedJobStateImap
+    ) {
+        this.runningJobStateIMap = runningJobStateIMap;
+        this.logger = logger;
+        this.runningJobMasterMap = runningJobMasterMap;
+        this.finishedJobStateImap = finishedJobStateImap;
+    }
+
+    // Gets the status of a running and completed job
+    public String listAllJob() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+        ArrayNode jobs = objectNode.putArray("jobs");
+
+        Stream.concat(runningJobMasterMap.values().stream().map(this::toJobStateMapper),
+                finishedJobStateImap.values().stream())
+            .forEach(jobStateMapper -> {
+                JobStatusMapper jobStatusMapper = new JobStatusMapper(jobStateMapper.jobId, jobStateMapper.jobStatus);
+                JsonNode jsonNode = objectMapper.valueToTree(jobStatusMapper);
+                jobs.add(jsonNode);
+            });
+        return jobs.toString();
+    }
+
+    // Get detailed status of a single job
+    public JobStateMapper getJobStatus(Long jobId) {
+        return runningJobMasterMap.containsKey(jobId) ? toJobStateMapper(runningJobMasterMap.get(jobId)) :
+            finishedJobStateImap.getOrDefault(jobId, null);
+    }
+
+    // Get detailed status of a single job as json
+    public String getJobStatusAsString(Long jobId) {
+        ObjectMapper objectMapper = new ObjectMapper();
+        JobStateMapper jobStatus = getJobStatus(jobId);
+        if (null != jobStatus) {
+            try {
+                return objectMapper.writeValueAsString(jobStatus);
+            } catch (JsonProcessingException e) {
+                logger.severe("serialize jobStateMapper err", e);
+                ObjectNode objectNode = objectMapper.createObjectNode();
+                objectNode.put("err", "serialize jobStateMapper err");
+                return objectNode.toString();
+            }
+        }
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectNode.put("err", String.format("jobId : %s not found", jobId));
+        return objectNode.toString();
+    }
+
+    @SuppressWarnings("checkstyle:MagicNumber")
+    public void storeFinishedJobState(JobMaster jobMaster) {
+        JobStateMapper jobStateMapper = toJobStateMapper(jobMaster);
+        finishedJobStateImap.put(jobStateMapper.jobId, jobStateMapper, 14, TimeUnit.DAYS);
+    }
+
+    private JobStateMapper toJobStateMapper(JobMaster jobMaster) {
+
+        Long jobId = jobMaster.getJobImmutableInformation().getJobId();
+        Map<PipelineLocation, PipelineStateMapper> pipelineStateMapperMap = new HashMap<>();
+
+        jobMaster.getPhysicalPlan().getPipelineList().forEach(pipeline -> {
+            PipelineLocation pipelineLocation = pipeline.getPipelineLocation();
+            PipelineStatus pipelineState = (PipelineStatus) runningJobStateIMap.get(pipelineLocation);
+            Map<TaskGroupLocation, ExecutionState> taskStateMap = new HashMap<>();
+            pipeline.getCoordinatorVertexList().forEach(coordinator -> {
+                TaskGroupLocation taskGroupLocation = coordinator.getTaskGroupLocation();
+                taskStateMap.put(taskGroupLocation, (ExecutionState) runningJobStateIMap.get(taskGroupLocation));
+            });
+            pipeline.getPhysicalVertexList().forEach(task -> {
+                TaskGroupLocation taskGroupLocation = task.getTaskGroupLocation();
+                taskStateMap.put(taskGroupLocation, (ExecutionState) runningJobStateIMap.get(taskGroupLocation));
+            });
+
+            PipelineStateMapper pipelineStateMapper = new PipelineStateMapper(pipelineState, taskStateMap);
+            pipelineStateMapperMap.put(pipelineLocation, pipelineStateMapper);
+        });
+        JobStatus jobStatus = (JobStatus) runningJobStateIMap.get(jobId);
+
+        return new JobStateMapper(jobId, jobStatus, pipelineStateMapperMap);
+    }
+
+    @AllArgsConstructor
+    @Data
+    public static final class JobStatusMapper implements Serializable {
+        Long jobId;
+        JobStatus jobStatus;
+    }
+
+    @AllArgsConstructor
+    @Data
+    public static final class JobStateMapper implements Serializable{
+        Long jobId;
+        JobStatus jobStatus;
+        Map<PipelineLocation, PipelineStateMapper> pipelineStateMapperMap;
+    }
+
+    @AllArgsConstructor
+    @Data
+    public static final class PipelineStateMapper implements Serializable{

Review Comment:
   Same as above



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.seatunnel.engine.server.master;
+
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.execution.ExecutionState;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.map.IMap;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+public class JobHistoryService {
+    /**
+     * IMap key is one of jobId {@link org.apache.seatunnel.engine.server.dag.physical.PipelineLocation} and
+     * {@link org.apache.seatunnel.engine.server.execution.TaskGroupLocation}
+     * <p>
+     * The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
+     * {@link org.apache.seatunnel.engine.server.execution.ExecutionState}
+     * <p>
+     * This IMap is used to recovery runningJobStateIMap in JobMaster when a new master node active
+     */
+    private final IMap<Object, Object> runningJobStateIMap;
+
+    private final ILogger logger;
+
+    /**
+     * key: job id;
+     * <br> value: job master;
+     */
+    private final Map<Long, JobMaster> runningJobMasterMap;
+
+    /**
+     * finishedJobStateImap key is jobId and value is jobState(json)
+     * JobStateMapper Indicates the status of the job, pipeline, and task
+     */
+    //TODO need to limit the amount of storage
+    private final IMap<Long, JobStateMapper> finishedJobStateImap;
+
+    public JobHistoryService(
+        IMap<Object, Object> runningJobStateIMap,
+        ILogger logger,
+        Map<Long, JobMaster> runningJobMasterMap,
+        IMap<Long, JobStateMapper> finishedJobStateImap
+    ) {
+        this.runningJobStateIMap = runningJobStateIMap;
+        this.logger = logger;
+        this.runningJobMasterMap = runningJobMasterMap;
+        this.finishedJobStateImap = finishedJobStateImap;
+    }
+
+    // Gets the status of a running and completed job
+    public String listAllJob() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+        ArrayNode jobs = objectNode.putArray("jobs");
+
+        Stream.concat(runningJobMasterMap.values().stream().map(this::toJobStateMapper),
+                finishedJobStateImap.values().stream())
+            .forEach(jobStateMapper -> {
+                JobStatusMapper jobStatusMapper = new JobStatusMapper(jobStateMapper.jobId, jobStateMapper.jobStatus);
+                JsonNode jsonNode = objectMapper.valueToTree(jobStatusMapper);
+                jobs.add(jsonNode);
+            });
+        return jobs.toString();
+    }
+
+    // Get detailed status of a single job
+    public JobStateMapper getJobStatus(Long jobId) {
+        return runningJobMasterMap.containsKey(jobId) ? toJobStateMapper(runningJobMasterMap.get(jobId)) :
+            finishedJobStateImap.getOrDefault(jobId, null);
+    }
+
+    // Get detailed status of a single job as json
+    public String getJobStatusAsString(Long jobId) {
+        ObjectMapper objectMapper = new ObjectMapper();
+        JobStateMapper jobStatus = getJobStatus(jobId);
+        if (null != jobStatus) {
+            try {
+                return objectMapper.writeValueAsString(jobStatus);
+            } catch (JsonProcessingException e) {
+                logger.severe("serialize jobStateMapper err", e);
+                ObjectNode objectNode = objectMapper.createObjectNode();
+                objectNode.put("err", "serialize jobStateMapper err");
+                return objectNode.toString();
+            }
+        }
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectNode.put("err", String.format("jobId : %s not found", jobId));
+        return objectNode.toString();
+    }
+
+    @SuppressWarnings("checkstyle:MagicNumber")
+    public void storeFinishedJobState(JobMaster jobMaster) {
+        JobStateMapper jobStateMapper = toJobStateMapper(jobMaster);
+        finishedJobStateImap.put(jobStateMapper.jobId, jobStateMapper, 14, TimeUnit.DAYS);
+    }
+
+    private JobStateMapper toJobStateMapper(JobMaster jobMaster) {
+
+        Long jobId = jobMaster.getJobImmutableInformation().getJobId();
+        Map<PipelineLocation, PipelineStateMapper> pipelineStateMapperMap = new HashMap<>();
+
+        jobMaster.getPhysicalPlan().getPipelineList().forEach(pipeline -> {
+            PipelineLocation pipelineLocation = pipeline.getPipelineLocation();
+            PipelineStatus pipelineState = (PipelineStatus) runningJobStateIMap.get(pipelineLocation);
+            Map<TaskGroupLocation, ExecutionState> taskStateMap = new HashMap<>();
+            pipeline.getCoordinatorVertexList().forEach(coordinator -> {
+                TaskGroupLocation taskGroupLocation = coordinator.getTaskGroupLocation();
+                taskStateMap.put(taskGroupLocation, (ExecutionState) runningJobStateIMap.get(taskGroupLocation));
+            });
+            pipeline.getPhysicalVertexList().forEach(task -> {
+                TaskGroupLocation taskGroupLocation = task.getTaskGroupLocation();
+                taskStateMap.put(taskGroupLocation, (ExecutionState) runningJobStateIMap.get(taskGroupLocation));
+            });
+
+            PipelineStateMapper pipelineStateMapper = new PipelineStateMapper(pipelineState, taskStateMap);
+            pipelineStateMapperMap.put(pipelineLocation, pipelineStateMapper);
+        });
+        JobStatus jobStatus = (JobStatus) runningJobStateIMap.get(jobId);
+
+        return new JobStateMapper(jobId, jobStatus, pipelineStateMapperMap);
+    }
+
+    @AllArgsConstructor
+    @Data
+    public static final class JobStatusMapper implements Serializable {
+        Long jobId;
+        JobStatus jobStatus;
+    }
+
+    @AllArgsConstructor
+    @Data
+    public static final class JobStateMapper implements Serializable{

Review Comment:
   Same as above



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/GetJobStateOperation.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.seatunnel.engine.server.operation;
+
+import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException;
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.serializable.OperationDataSerializerHook;
+
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+import com.hazelcast.nio.serialization.IdentifiedDataSerializable;
+import com.hazelcast.spi.impl.AllowedDuringPassiveState;
+import com.hazelcast.spi.impl.operationservice.Operation;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class GetJobStateOperation extends Operation implements IdentifiedDataSerializable, AllowedDuringPassiveState {
+    private Long jobId;
+
+    private String response;
+
+    public GetJobStateOperation() {
+    }
+
+    public GetJobStateOperation(Long jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public final int getFactoryId() {
+        return OperationDataSerializerHook.FACTORY_ID;
+    }
+
+    @Override
+    public int getClassId() {
+        return OperationDataSerializerHook.PRINT_MESSAGE_OPERATOR;
+    }
+
+    @Override
+    protected void writeInternal(ObjectDataOutput out) throws IOException {
+        super.writeInternal(out);
+        out.writeLong(jobId);
+    }
+
+    @Override
+    protected void readInternal(ObjectDataInput in) throws IOException {
+        super.readInternal(in);
+        jobId = in.readLong();
+    }
+
+    @Override
+    public void run() {
+        SeaTunnelServer service = getService();
+        CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> {

Review Comment:
   Why use `CompletableFuture` cause you use get immediately. This is sync logic.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/ListJobStatusOperation.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.seatunnel.engine.server.operation;
+
+import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException;
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+
+import com.hazelcast.spi.impl.AllowedDuringPassiveState;
+import com.hazelcast.spi.impl.operationservice.Operation;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class ListJobStatusOperation extends Operation implements AllowedDuringPassiveState {
+
+    private String response;
+
+    public ListJobStatusOperation() {
+    }
+
+    @Override
+    public void run() {
+        SeaTunnelServer service = getService();
+        CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> {

Review Comment:
   Same as above



-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ic4y commented on pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
ic4y commented on PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#issuecomment-1302877678

   @EricJoy2048 @hailin0 @Hisoka-X PTLA Thanks


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] EricJoy2048 closed pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
EricJoy2048 closed pull request #3191: [feature][st-engine] Add jobHistory
URL: https://github.com/apache/incubator-seatunnel/pull/3191


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] hailin0 commented on a diff in pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
hailin0 commented on code in PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#discussion_r1010257511


##########
seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/args/ClientCommandArgs.java:
##########
@@ -41,6 +41,14 @@ public class ClientCommandArgs extends AbstractCommandArgs {
         description = "The name of cluster")
     private String clusterName = "seatunnel_default_cluster";
 
+    @Parameter(names = {"-j", "--job-state"},

Review Comment:
   rename `job-state` to `job-id`?



-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ic4y commented on a diff in pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
ic4y commented on code in PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#discussion_r1015007310


##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/GetJobStateOperation.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.seatunnel.engine.server.operation;
+
+import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException;
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.serializable.OperationDataSerializerHook;
+
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+import com.hazelcast.nio.serialization.IdentifiedDataSerializable;
+import com.hazelcast.spi.impl.AllowedDuringPassiveState;
+import com.hazelcast.spi.impl.operationservice.Operation;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class GetJobStateOperation extends Operation implements IdentifiedDataSerializable, AllowedDuringPassiveState {
+    private Long jobId;
+
+    private String response;
+
+    public GetJobStateOperation() {
+    }
+
+    public GetJobStateOperation(Long jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public final int getFactoryId() {
+        return OperationDataSerializerHook.FACTORY_ID;
+    }
+
+    @Override
+    public int getClassId() {
+        return OperationDataSerializerHook.PRINT_MESSAGE_OPERATOR;
+    }
+
+    @Override
+    protected void writeInternal(ObjectDataOutput out) throws IOException {
+        super.writeInternal(out);
+        out.writeLong(jobId);
+    }
+
+    @Override
+    protected void readInternal(ObjectDataInput in) throws IOException {
+        super.readInternal(in);
+        jobId = in.readLong();
+    }
+
+    @Override
+    public void run() {
+        SeaTunnelServer service = getService();
+        CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> {

Review Comment:
   Because `Imap` cannot be used directly in the `Operation` thread



-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] Hisoka-X merged pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
Hisoka-X merged PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ic4y closed pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
ic4y closed pull request #3191: [feature][st-engine] Add jobHistory
URL: https://github.com/apache/incubator-seatunnel/pull/3191


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ic4y commented on pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
ic4y commented on PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#issuecomment-1305084867

   @Hisoka-X PTLA thanks


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ic4y closed pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
ic4y closed pull request #3191: [feature][st-engine] Add jobHistory
URL: https://github.com/apache/incubator-seatunnel/pull/3191


-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] EricJoy2048 commented on a diff in pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
EricJoy2048 commented on code in PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#discussion_r1010115679


##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistorySevice.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.seatunnel.engine.server.master;
+
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.execution.ExecutionState;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.map.IMap;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+public class JobHistorySevice {
+    /**
+     * IMap key is one of jobId {@link org.apache.seatunnel.engine.server.dag.physical.PipelineLocation} and
+     * {@link org.apache.seatunnel.engine.server.execution.TaskGroupLocation}
+     * <p>
+     * The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
+     * {@link org.apache.seatunnel.engine.server.execution.ExecutionState}
+     * <p>
+     * This IMap is used to recovery runningJobStateIMap in JobMaster when a new master node active
+     */
+    private final IMap<Object, Object> runningJobStateIMap;
+
+    private final ILogger logger;
+
+    /**
+     * key: job id;
+     * <br> value: job master;
+     */
+    private final Map<Long, JobMaster> runningJobMasterMap;
+
+    /**
+     * finishedJobStateImap key is jobId and value is jobState(json)
+     * JobStateMapper Indicates the status of the job, pipeline, and task
+     */
+    //TODO need to limit the amount of storage
+    private final IMap<Long, JobStateMapper> finishedJobStateImap;
+
+    public JobHistorySevice(
+        IMap<Object, Object> runningJobStateIMap,
+        ILogger logger,
+        Map<Long, JobMaster> runningJobMasterMap,
+        IMap<Long, JobStateMapper> finishedJobStateImap
+    ) {
+        this.runningJobStateIMap = runningJobStateIMap;
+        this.logger = logger;
+        this.runningJobMasterMap = runningJobMasterMap;
+        this.finishedJobStateImap = finishedJobStateImap;
+    }
+
+    // Gets the status of a running and completed job
+    public String listAllJob() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+        ArrayNode jobs = objectNode.putArray("jobs");
+
+        Stream.concat(runningJobMasterMap.values().stream().map(this::toJobStateMapper),
+                finishedJobStateImap.values().stream())
+            .forEach(jobStateMapper -> {
+                JobStatusMapper jobStatusMapper = new JobStatusMapper(jobStateMapper.jobId, jobStateMapper.jobStatus);
+                JsonNode jsonNode = objectMapper.valueToTree(jobStatusMapper);
+                jobs.add(jsonNode);
+            });
+        return jobs.toString();
+    }
+
+    // Get detailed status of a single job
+    public JobStateMapper getJobStatus(Long jobId) {
+        return runningJobMasterMap.containsKey(jobId) ? toJobStateMapper(runningJobMasterMap.get(jobId)) :
+            finishedJobStateImap.getOrDefault(jobId, null);
+    }
+
+    // Get detailed status of a single job as json
+    public String getJobStatusAsString(Long jobId) {
+        ObjectMapper objectMapper = new ObjectMapper();
+        JobStateMapper jobStatus = getJobStatus(jobId);
+        if (null != jobStatus) {
+            try {
+                return objectMapper.writeValueAsString(jobStatus);
+            } catch (JsonProcessingException e) {
+                logger.severe("serialize jobStateMapper err", e);
+                ObjectNode objectNode = objectMapper.createObjectNode();
+                objectNode.put("err", "serialize jobStateMapper err");
+                return objectNode.toString();
+            }
+        } else {

Review Comment:
   `else` is not needed.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/ListJobStatusOperation.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.seatunnel.engine.server.operation;
+
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+
+import com.hazelcast.spi.impl.AllowedDuringPassiveState;
+import com.hazelcast.spi.impl.operationservice.Operation;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class ListJobStatusOperation extends Operation implements AllowedDuringPassiveState {
+
+    private String response;
+
+    public ListJobStatusOperation() {
+    }
+
+    @Override
+    public void run() {
+        SeaTunnelServer service = getService();
+        CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> {
+            return service.getCoordinatorService().jobHistorySevice.listAllJob();
+        });
+
+        try {
+            response = future.get();
+        } catch (InterruptedException | ExecutionException e) {
+            throw new RuntimeException(e);

Review Comment:
   same as above.



##########
seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java:
##########
@@ -66,10 +64,22 @@ public void execute() throws CommandExecuteException {
             ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig();
             clientConfig.setClusterName(clusterName);
             engineClient = new SeaTunnelClient(clientConfig);
-            JobExecutionEnvironment jobExecutionEnv = engineClient.createExecutionContext(configFile.toString(), jobConfig);
+            if (clientCommandArgs.isListJob()) {
+                String jobstatus = engineClient.listJobStatus();
+                log.info(jobstatus);

Review Comment:
   I think the command result use `System.out.print()` is better.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/GetJobStateOperation.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.seatunnel.engine.server.operation;
+
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.serializable.OperationDataSerializerHook;
+
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+import com.hazelcast.nio.serialization.IdentifiedDataSerializable;
+import com.hazelcast.spi.impl.AllowedDuringPassiveState;
+import com.hazelcast.spi.impl.operationservice.Operation;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class GetJobStateOperation extends Operation implements IdentifiedDataSerializable, AllowedDuringPassiveState {
+    private Long jobId;
+
+    private String response;
+
+    public GetJobStateOperation() {
+    }
+
+    public GetJobStateOperation(Long jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public final int getFactoryId() {
+        return OperationDataSerializerHook.FACTORY_ID;
+    }
+
+    @Override
+    public int getClassId() {
+        return OperationDataSerializerHook.PRINT_MESSAGE_OPERATOR;
+    }
+
+    @Override
+    protected void writeInternal(ObjectDataOutput out) throws IOException {
+        super.writeInternal(out);
+        out.writeLong(jobId);
+    }
+
+    @Override
+    protected void readInternal(ObjectDataInput in) throws IOException {
+        super.readInternal(in);
+        jobId = in.readLong();
+    }
+
+    @Override
+    public void run() {
+        SeaTunnelServer service = getService();
+        CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> {
+            return service.getCoordinatorService().jobHistorySevice.getJobStatusAsString(jobId);

Review Comment:
   `getJobHistoryService` is better.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/operation/GetJobStateOperation.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.seatunnel.engine.server.operation;
+
+import org.apache.seatunnel.engine.server.SeaTunnelServer;
+import org.apache.seatunnel.engine.server.serializable.OperationDataSerializerHook;
+
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+import com.hazelcast.nio.serialization.IdentifiedDataSerializable;
+import com.hazelcast.spi.impl.AllowedDuringPassiveState;
+import com.hazelcast.spi.impl.operationservice.Operation;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class GetJobStateOperation extends Operation implements IdentifiedDataSerializable, AllowedDuringPassiveState {
+    private Long jobId;
+
+    private String response;
+
+    public GetJobStateOperation() {
+    }
+
+    public GetJobStateOperation(Long jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public final int getFactoryId() {
+        return OperationDataSerializerHook.FACTORY_ID;
+    }
+
+    @Override
+    public int getClassId() {
+        return OperationDataSerializerHook.PRINT_MESSAGE_OPERATOR;
+    }
+
+    @Override
+    protected void writeInternal(ObjectDataOutput out) throws IOException {
+        super.writeInternal(out);
+        out.writeLong(jobId);
+    }
+
+    @Override
+    protected void readInternal(ObjectDataInput in) throws IOException {
+        super.readInternal(in);
+        jobId = in.readLong();
+    }
+
+    @Override
+    public void run() {
+        SeaTunnelServer service = getService();
+        CompletableFuture<String> future = CompletableFuture.supplyAsync(() -> {
+            return service.getCoordinatorService().jobHistorySevice.getJobStatusAsString(jobId);
+        });
+
+        try {
+            response = future.get();
+        } catch (InterruptedException | ExecutionException e) {
+            throw new RuntimeException(e);

Review Comment:
   Use `SeaTunnelEngineException` is better.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistorySevice.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.seatunnel.engine.server.master;
+
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.execution.ExecutionState;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.hazelcast.logging.ILogger;
+import com.hazelcast.map.IMap;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+public class JobHistorySevice {
+    /**
+     * IMap key is one of jobId {@link org.apache.seatunnel.engine.server.dag.physical.PipelineLocation} and
+     * {@link org.apache.seatunnel.engine.server.execution.TaskGroupLocation}
+     * <p>
+     * The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
+     * {@link org.apache.seatunnel.engine.server.execution.ExecutionState}
+     * <p>
+     * This IMap is used to recovery runningJobStateIMap in JobMaster when a new master node active
+     */
+    private final IMap<Object, Object> runningJobStateIMap;
+
+    private final ILogger logger;
+
+    /**
+     * key: job id;
+     * <br> value: job master;
+     */
+    private final Map<Long, JobMaster> runningJobMasterMap;
+
+    /**
+     * finishedJobStateImap key is jobId and value is jobState(json)
+     * JobStateMapper Indicates the status of the job, pipeline, and task
+     */
+    //TODO need to limit the amount of storage
+    private final IMap<Long, JobStateMapper> finishedJobStateImap;
+
+    public JobHistorySevice(
+        IMap<Object, Object> runningJobStateIMap,
+        ILogger logger,
+        Map<Long, JobMaster> runningJobMasterMap,
+        IMap<Long, JobStateMapper> finishedJobStateImap
+    ) {
+        this.runningJobStateIMap = runningJobStateIMap;
+        this.logger = logger;
+        this.runningJobMasterMap = runningJobMasterMap;
+        this.finishedJobStateImap = finishedJobStateImap;
+    }
+
+    // Gets the status of a running and completed job
+    public String listAllJob() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        ObjectNode objectNode = objectMapper.createObjectNode();
+        objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+        ArrayNode jobs = objectNode.putArray("jobs");
+
+        Stream.concat(runningJobMasterMap.values().stream().map(this::toJobStateMapper),
+                finishedJobStateImap.values().stream())
+            .forEach(jobStateMapper -> {
+                JobStatusMapper jobStatusMapper = new JobStatusMapper(jobStateMapper.jobId, jobStateMapper.jobStatus);
+                JsonNode jsonNode = objectMapper.valueToTree(jobStatusMapper);
+                jobs.add(jsonNode);
+            });
+        return jobs.toString();
+    }
+
+    // Get detailed status of a single job
+    public JobStateMapper getJobStatus(Long jobId) {
+        return runningJobMasterMap.containsKey(jobId) ? toJobStateMapper(runningJobMasterMap.get(jobId)) :
+            finishedJobStateImap.getOrDefault(jobId, null);
+    }
+
+    // Get detailed status of a single job as json
+    public String getJobStatusAsString(Long jobId) {
+        ObjectMapper objectMapper = new ObjectMapper();
+        JobStateMapper jobStatus = getJobStatus(jobId);
+        if (null != jobStatus) {
+            try {
+                return objectMapper.writeValueAsString(jobStatus);
+            } catch (JsonProcessingException e) {
+                logger.severe("serialize jobStateMapper err", e);
+                ObjectNode objectNode = objectMapper.createObjectNode();
+                objectNode.put("err", "serialize jobStateMapper err");
+                return objectNode.toString();
+            }
+        } else {
+            ObjectNode objectNode = objectMapper.createObjectNode();
+            objectNode.put("err", String.format("jobId : %s not found", jobId));
+            return objectNode.toString();
+        }
+    }
+
+    @SuppressWarnings("checkstyle:MagicNumber")
+    public void storeFinishedJobState(JobMaster jobMaster) {
+        JobStateMapper jobStateMapper = toJobStateMapper(jobMaster);
+        finishedJobStateImap.put(jobStateMapper.jobId, jobStateMapper, 90, TimeUnit.DAYS);

Review Comment:
   90 day is too long, Please make it configurable.



-- 
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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ic4y commented on a diff in pull request #3191: [feature][st-engine] Add jobHistory

Posted by GitBox <gi...@apache.org>.
ic4y commented on code in PR #3191:
URL: https://github.com/apache/incubator-seatunnel/pull/3191#discussion_r1011353040


##########
seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/args/ClientCommandArgs.java:
##########
@@ -41,6 +41,14 @@ public class ClientCommandArgs extends AbstractCommandArgs {
         description = "The name of cluster")
     private String clusterName = "seatunnel_default_cluster";
 
+    @Parameter(names = {"-j", "--job-state"},

Review Comment:
   Done



-- 
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: commits-unsubscribe@seatunnel.apache.org

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