You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/08/19 05:06:29 UTC

[inlong] branch master updated: [INLONG-5562][Manager] Add the workflow API in the manager client (#5563)

This is an automated email from the ASF dual-hosted git repository.

dockerzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 88e95e823 [INLONG-5562][Manager] Add the workflow API in the manager client (#5563)
88e95e823 is described below

commit 88e95e82375436d4c34626a3f935be6701819d7e
Author: haifxu <xh...@gmail.com>
AuthorDate: Fri Aug 19 13:06:25 2022 +0800

    [INLONG-5562][Manager] Add the workflow API in the manager client (#5563)
---
 .../apache/inlong/manager/client/api/Workflow.java |  99 ++++++++++++++
 .../manager/client/api/impl/WorkflowImpl.java      |  92 +++++++++++++
 .../client/api/inner/client/WorkflowClient.java    | 142 ++++++++++++++++++++-
 .../manager/client/api/service/WorkflowApi.java    |  32 ++++-
 .../{WorkflowApi.java => WorkflowEventApi.java}    |  14 +-
 .../client/api/inner/ClientFactoryTest.java        |   4 +
 .../client/api/inner/WorkflowClientTest.java       |  99 ++++++++++++++
 .../manager/pojo/workflow/WorkflowResult.java      |   6 +
 8 files changed, 470 insertions(+), 18 deletions(-)

diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/Workflow.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/Workflow.java
new file mode 100644
index 000000000..b6edbe647
--- /dev/null
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/Workflow.java
@@ -0,0 +1,99 @@
+/*
+ * 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.inlong.manager.client.api;
+
+import com.github.pagehelper.PageInfo;
+import org.apache.inlong.manager.pojo.workflow.ProcessDetailResponse;
+import org.apache.inlong.manager.pojo.workflow.ProcessRequest;
+import org.apache.inlong.manager.pojo.workflow.ProcessResponse;
+import org.apache.inlong.manager.pojo.workflow.TaskRequest;
+import org.apache.inlong.manager.pojo.workflow.TaskResponse;
+import org.apache.inlong.manager.pojo.workflow.WorkflowOperationRequest;
+import org.apache.inlong.manager.pojo.workflow.WorkflowResult;
+
+public interface Workflow {
+
+    /**
+     * Initiation process
+     *
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    WorkflowResult start(WorkflowOperationRequest request);
+
+    /**
+     * Initiation process
+     *
+     * @param processId process id
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    WorkflowResult cancel(Integer processId, WorkflowOperationRequest request);
+
+    /**
+     * Continue process when pending or failed
+     *
+     * @param processId process id
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    WorkflowResult continueProcess(Integer processId, WorkflowOperationRequest request);
+
+    /**
+     * Cancellation process application
+     *
+     * @param taskId taskId
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    WorkflowResult reject(Integer taskId, WorkflowOperationRequest request);
+
+    /**
+     * Complete task-true to automatic task
+     *
+     * @param taskId taskId
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    WorkflowResult complete(Integer taskId, WorkflowOperationRequest request);
+
+    /**
+     * Query process details according to the tracking number
+     *
+     * @param processId processId
+     * @param taskId taskId
+     * @return process detail response
+     */
+    ProcessDetailResponse detail(Integer processId, Integer taskId);
+
+    /**
+     * Get process list
+     *
+     * @param request workflow process request
+     * @return process response list
+     */
+    PageInfo<ProcessResponse> listProcess(ProcessRequest request);
+
+    /**
+     * Get task list
+     *
+     * @param request workflow task query request
+     * @return task response list
+     */
+    PageInfo<TaskResponse> listTask(TaskRequest request);
+}
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/impl/WorkflowImpl.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/impl/WorkflowImpl.java
new file mode 100644
index 000000000..aea0f9ee6
--- /dev/null
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/impl/WorkflowImpl.java
@@ -0,0 +1,92 @@
+/*
+ * 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.inlong.manager.client.api.impl;
+
+import com.github.pagehelper.PageInfo;
+import org.apache.inlong.manager.client.api.ClientConfiguration;
+import org.apache.inlong.manager.client.api.Workflow;
+import org.apache.inlong.manager.client.api.inner.client.ClientFactory;
+import org.apache.inlong.manager.client.api.inner.client.WorkflowClient;
+import org.apache.inlong.manager.client.api.util.ClientUtils;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.workflow.ProcessDetailResponse;
+import org.apache.inlong.manager.pojo.workflow.ProcessRequest;
+import org.apache.inlong.manager.pojo.workflow.ProcessResponse;
+import org.apache.inlong.manager.pojo.workflow.TaskRequest;
+import org.apache.inlong.manager.pojo.workflow.TaskResponse;
+import org.apache.inlong.manager.pojo.workflow.WorkflowOperationRequest;
+import org.apache.inlong.manager.pojo.workflow.WorkflowResult;
+
+public class WorkflowImpl implements Workflow {
+
+    private final WorkflowClient workflowClient;
+
+    public WorkflowImpl(ClientConfiguration configuration) {
+        ClientFactory clientFactory = ClientUtils.getClientFactory(configuration);
+        this.workflowClient = clientFactory.getWorkflowClient();
+    }
+
+    @Override
+    public WorkflowResult start(WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(request.getName(), "process name cannot be null");
+        Preconditions.checkNotNull(request.getForm(), "form cannot be null");
+        return workflowClient.start(request);
+    }
+
+    @Override
+    public WorkflowResult cancel(Integer processId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(processId, "process id cannot be null");
+        return workflowClient.cancel(processId, request);
+    }
+
+    @Override
+    public WorkflowResult continueProcess(Integer processId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(processId, "process id cannot be null");
+        return workflowClient.continueProcess(processId, request);
+    }
+
+    @Override
+    public WorkflowResult reject(Integer taskId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(taskId, "task id cannot be null");
+        return workflowClient.reject(taskId, request);
+    }
+
+    @Override
+    public WorkflowResult complete(Integer taskId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(taskId, "task id cannot be null");
+        return workflowClient.complete(taskId, request);
+    }
+
+    @Override
+    public ProcessDetailResponse detail(Integer processId, Integer taskId) {
+        Preconditions.checkNotNull(processId, "process id cannot be null");
+        return workflowClient.detail(processId, taskId);
+    }
+
+    @Override
+    public PageInfo<ProcessResponse> listProcess(ProcessRequest request) {
+        Preconditions.checkNotNull(request, "process request cannot be null");
+        return workflowClient.listProcess(request);
+    }
+
+    @Override
+    public PageInfo<TaskResponse> listTask(TaskRequest request) {
+        Preconditions.checkNotNull(request, "task request cannot be null");
+        return workflowClient.listTask(request);
+    }
+}
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/WorkflowClient.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/WorkflowClient.java
index dc4d8a81c..fed7e97a7 100644
--- a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/WorkflowClient.java
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/WorkflowClient.java
@@ -20,15 +20,23 @@ package org.apache.inlong.manager.client.api.inner.client;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.github.pagehelper.PageInfo;
 import com.google.common.collect.Lists;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.inlong.manager.client.api.ClientConfiguration;
 import org.apache.inlong.manager.client.api.service.WorkflowApi;
 import org.apache.inlong.manager.client.api.util.ClientUtils;
+import org.apache.inlong.manager.common.util.JsonUtils;
+import org.apache.inlong.manager.common.util.Preconditions;
 import org.apache.inlong.manager.pojo.common.Response;
+import org.apache.inlong.manager.pojo.workflow.ProcessDetailResponse;
+import org.apache.inlong.manager.pojo.workflow.ProcessRequest;
+import org.apache.inlong.manager.pojo.workflow.ProcessResponse;
+import org.apache.inlong.manager.pojo.workflow.TaskRequest;
+import org.apache.inlong.manager.pojo.workflow.TaskResponse;
+import org.apache.inlong.manager.pojo.workflow.WorkflowOperationRequest;
 import org.apache.inlong.manager.pojo.workflow.WorkflowResult;
 import org.apache.inlong.manager.pojo.workflow.form.process.ApplyGroupProcessForm;
-import org.apache.inlong.manager.common.util.JsonUtils;
 
 import java.util.Map;
 
@@ -45,6 +53,13 @@ public class WorkflowClient {
         workflowApi = ClientUtils.createRetrofit(configuration).create(WorkflowApi.class);
     }
 
+    /**
+     * Approval the process
+     *
+     * @param taskId taskId
+     * @param groupProcessForm inlong group process form
+     * @return workflow result info
+     */
     public WorkflowResult startInlongGroup(int taskId, ApplyGroupProcessForm groupProcessForm) {
         ObjectNode workflowTaskOperation = objectMapper.createObjectNode();
         workflowTaskOperation.putPOJO("transferTo", Lists.newArrayList());
@@ -68,4 +83,129 @@ public class WorkflowClient {
         return response.getData();
     }
 
+    /**
+     * Initiation process
+     *
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    public WorkflowResult start(WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(request.getName(), "process name cannot be null");
+        Preconditions.checkNotNull(request.getForm(), "form cannot be null");
+
+        Response<WorkflowResult> response = ClientUtils.executeHttpCall(workflowApi.start(request));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Initiation process
+     *
+     * @param processId process id
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    public WorkflowResult cancel(Integer processId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(processId, "process id cannot be null");
+
+        Response<WorkflowResult> response = ClientUtils.executeHttpCall(workflowApi.cancel(processId, request));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Continue process when pending or failed
+     *
+     * @param processId process id
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    public WorkflowResult continueProcess(Integer processId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(processId, "process id cannot be null");
+
+        Response<WorkflowResult> response = ClientUtils.executeHttpCall(
+                workflowApi.continueProcess(processId, request));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Cancellation process application
+     *
+     * @param taskId taskId
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    public WorkflowResult reject(Integer taskId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(taskId, "task id cannot be null");
+
+        Response<WorkflowResult> response = ClientUtils.executeHttpCall(workflowApi.reject(taskId, request));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Complete task-true to automatic task
+     *
+     * @param taskId taskId
+     * @param request workflow operation request
+     * @return workflow result info
+     */
+    public WorkflowResult complete(Integer taskId, WorkflowOperationRequest request) {
+        Preconditions.checkNotNull(taskId, "task id cannot be null");
+
+        Response<WorkflowResult> response = ClientUtils.executeHttpCall(workflowApi.complete(taskId, request));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Query process details according to the tracking number
+     *
+     * @param processId processId
+     * @param taskId taskId
+     * @return process detail response
+     */
+    public ProcessDetailResponse detail(Integer processId, Integer taskId) {
+        Preconditions.checkNotNull(processId, "process id cannot be null");
+
+        Response<ProcessDetailResponse> response = ClientUtils.executeHttpCall(workflowApi.detail(processId, taskId));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Get process list
+     *
+     * @param request workflow process request
+     * @return process response list
+     */
+    public PageInfo<ProcessResponse> listProcess(ProcessRequest request) {
+        Preconditions.checkNotNull(request, "process request cannot be null");
+
+        Map<String, Object> requestMap = JsonUtils.OBJECT_MAPPER.convertValue(request,
+                new TypeReference<Map<String, Object>>() {
+                });
+        Response<PageInfo<ProcessResponse>> response = ClientUtils.executeHttpCall(workflowApi.listProcess(requestMap));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Get task list
+     *
+     * @param request workflow task query request
+     * @return task response list
+     */
+    public PageInfo<TaskResponse> listTask(TaskRequest request) {
+        Preconditions.checkNotNull(request, "task request cannot be null");
+
+        Map<String, Object> requestMap = JsonUtils.OBJECT_MAPPER.convertValue(request,
+                new TypeReference<Map<String, Object>>() {
+                });
+        Response<PageInfo<TaskResponse>> response = ClientUtils.executeHttpCall(workflowApi.listTask(requestMap));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
 }
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java
index c9e70dbfa..d11e2144c 100644
--- a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java
@@ -19,7 +19,10 @@ package org.apache.inlong.manager.client.api.service;
 
 import com.github.pagehelper.PageInfo;
 import org.apache.inlong.manager.pojo.common.Response;
-import org.apache.inlong.manager.pojo.workflow.EventLogResponse;
+import org.apache.inlong.manager.pojo.workflow.ProcessDetailResponse;
+import org.apache.inlong.manager.pojo.workflow.ProcessResponse;
+import org.apache.inlong.manager.pojo.workflow.TaskResponse;
+import org.apache.inlong.manager.pojo.workflow.WorkflowOperationRequest;
 import org.apache.inlong.manager.pojo.workflow.WorkflowResult;
 import retrofit2.Call;
 import retrofit2.http.Body;
@@ -37,8 +40,29 @@ public interface WorkflowApi {
     @POST("workflow/approve/{taskId}")
     Call<Response<WorkflowResult>> startInlongGroup(@Path("taskId") Integer taskId, @Body Map<String, Object> request);
 
-    @GET("workflow/event/list")
-    Call<Response<PageInfo<EventLogResponse>>> getInlongGroupError(@Query("inlongGroupId") String groupId,
-            @Query("status") Integer status);
+    @POST("workflow/start")
+    Call<Response<WorkflowResult>> start(@Body WorkflowOperationRequest request);
+
+    @POST("workflow/cancel/{processId}")
+    Call<Response<WorkflowResult>> cancel(@Path("processId") Integer processId, @Body WorkflowOperationRequest request);
+
+    @POST("workflow/continue/{processId}")
+    Call<Response<WorkflowResult>> continueProcess(@Path("processId") Integer processId,
+            @Body WorkflowOperationRequest request);
+
+    @POST("workflow/reject/{taskId}")
+    Call<Response<WorkflowResult>> reject(@Path("taskId") Integer taskId, @Body WorkflowOperationRequest request);
+
+    @POST("workflow/complete/{taskId}")
+    Call<Response<WorkflowResult>> complete(@Path("taskId") Integer taskId, @Body WorkflowOperationRequest request);
+
+    @GET("workflow/detail/{processId}")
+    Call<Response<ProcessDetailResponse>> detail(@Path("processId") Integer processId, @Query("taskId") Integer taskId);
+
+    @GET("workflow/listProcess")
+    Call<Response<PageInfo<ProcessResponse>>> listProcess(@Query("query") Map<String, Object> query);
+
+    @GET("workflow/listTask")
+    Call<Response<PageInfo<TaskResponse>>> listTask(@Query("query") Map<String, Object> query);
 
 }
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowEventApi.java
similarity index 74%
copy from inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java
copy to inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowEventApi.java
index c9e70dbfa..abecf1ebf 100644
--- a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowApi.java
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/WorkflowEventApi.java
@@ -20,25 +20,13 @@ package org.apache.inlong.manager.client.api.service;
 import com.github.pagehelper.PageInfo;
 import org.apache.inlong.manager.pojo.common.Response;
 import org.apache.inlong.manager.pojo.workflow.EventLogResponse;
-import org.apache.inlong.manager.pojo.workflow.WorkflowResult;
 import retrofit2.Call;
-import retrofit2.http.Body;
 import retrofit2.http.GET;
-import retrofit2.http.Headers;
-import retrofit2.http.POST;
-import retrofit2.http.Path;
 import retrofit2.http.Query;
 
-import java.util.Map;
-
-public interface WorkflowApi {
-
-    @Headers("Content-Type: application/json")
-    @POST("workflow/approve/{taskId}")
-    Call<Response<WorkflowResult>> startInlongGroup(@Path("taskId") Integer taskId, @Body Map<String, Object> request);
+public interface WorkflowEventApi {
 
     @GET("workflow/event/list")
     Call<Response<PageInfo<EventLogResponse>>> getInlongGroupError(@Query("inlongGroupId") String groupId,
             @Query("status") Integer status);
-
 }
diff --git a/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java b/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java
index bcd28a77f..54f8158ae 100644
--- a/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java
+++ b/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java
@@ -34,6 +34,7 @@ import org.apache.inlong.manager.client.api.inner.client.InlongStreamClient;
 import org.apache.inlong.manager.client.api.inner.client.StreamSinkClient;
 import org.apache.inlong.manager.client.api.inner.client.StreamSourceClient;
 import org.apache.inlong.manager.client.api.inner.client.UserClient;
+import org.apache.inlong.manager.client.api.inner.client.WorkflowClient;
 import org.apache.inlong.manager.client.api.util.ClientUtils;
 import org.apache.inlong.manager.common.auth.DefaultAuthentication;
 import org.apache.inlong.manager.common.auth.TokenAuthentication;
@@ -121,6 +122,7 @@ class ClientFactoryTest {
     private static InlongClusterClient clusterClient;
     private static DataNodeClient dataNodeClient;
     private static UserClient userClient;
+    private static WorkflowClient workflowClient;
 
     @BeforeAll
     static void setup() {
@@ -142,6 +144,7 @@ class ClientFactoryTest {
         clusterClient = clientFactory.getClusterClient();
         dataNodeClient = clientFactory.getDataNodeClient();
         userClient = clientFactory.getUserClient();
+        workflowClient = clientFactory.getWorkflowClient();
     }
 
     @AfterAll
@@ -1117,4 +1120,5 @@ class ClientFactoryTest {
         Boolean isDelete = userClient.delete(1);
         Assertions.assertTrue(isDelete);
     }
+
 }
diff --git a/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/WorkflowClientTest.java b/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/WorkflowClientTest.java
new file mode 100644
index 000000000..ae4686a26
--- /dev/null
+++ b/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/WorkflowClientTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.inlong.manager.client.api.inner;
+
+import com.github.pagehelper.PageInfo;
+import com.google.common.collect.Lists;
+import org.apache.inlong.manager.client.api.inner.client.WorkflowClient;
+import org.apache.inlong.manager.common.enums.ProcessName;
+import org.apache.inlong.manager.common.util.JsonUtils;
+import org.apache.inlong.manager.pojo.common.Response;
+import org.apache.inlong.manager.pojo.group.pulsar.InlongPulsarInfo;
+import org.apache.inlong.manager.pojo.workflow.ProcessRequest;
+import org.apache.inlong.manager.pojo.workflow.ProcessResponse;
+import org.apache.inlong.manager.pojo.workflow.WorkflowOperationRequest;
+import org.apache.inlong.manager.pojo.workflow.WorkflowResult;
+import org.apache.inlong.manager.pojo.workflow.form.process.ApplyGroupProcessForm;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.okJson;
+import static com.github.tomakehurst.wiremock.client.WireMock.post;
+import static com.github.tomakehurst.wiremock.client.WireMock.stubFor;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlMatching;
+
+public class WorkflowClientTest extends ClientFactoryTest {
+
+    private static final WorkflowClient workflowClient = clientFactory.getWorkflowClient();
+
+    @Test
+    void testWorkflowStart() {
+        WorkflowResult workflowResult = WorkflowResult.builder()
+                .processInfo(ProcessResponse.builder()
+                        .id(1)
+                        .name(ProcessName.APPLY_GROUP_PROCESS.getDisplayName())
+                        .applicant("test_user").build())
+                .build();
+
+        stubFor(
+                post(urlMatching("/inlong/manager/api/workflow/start.*"))
+                        .willReturn(
+                                okJson(JsonUtils.toJsonString(
+                                        Response.success(workflowResult))
+                                )
+                        )
+        );
+        WorkflowOperationRequest request = new WorkflowOperationRequest();
+        request.setName(ProcessName.APPLY_GROUP_PROCESS);
+        request.setApplicant("test_user");
+        ApplyGroupProcessForm form = new ApplyGroupProcessForm();
+        form.setGroupInfo(new InlongPulsarInfo());
+        request.setForm(form);
+
+        WorkflowResult workflowInfo = workflowClient.start(request);
+        Assertions.assertEquals(request.getName().getDisplayName(), workflowInfo.getProcessInfo().getName());
+        Assertions.assertEquals(request.getApplicant(), workflowInfo.getProcessInfo().getApplicant());
+    }
+
+    @Test
+    void testListProcess() {
+        List<ProcessResponse> responses = Lists.newArrayList(
+                ProcessResponse.builder()
+                        .id(1)
+                        .name("test_process")
+                        .build()
+        );
+
+        stubFor(
+                get(urlMatching("/inlong/manager/api/workflow/listProcess.*"))
+                        .willReturn(
+                                okJson(JsonUtils.toJsonString(
+                                        Response.success(new PageInfo<>(responses)))
+                                )
+                        )
+        );
+
+        ProcessRequest request = new ProcessRequest();
+        request.setId(1);
+        PageInfo<ProcessResponse> pageInfo = workflowClient.listProcess(request);
+        Assertions.assertEquals(JsonUtils.toJsonString(pageInfo.getList()), JsonUtils.toJsonString(responses));
+    }
+}
diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/workflow/WorkflowResult.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/workflow/WorkflowResult.java
index 941d9af93..0cfdd8a32 100644
--- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/workflow/WorkflowResult.java
+++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/workflow/WorkflowResult.java
@@ -19,7 +19,10 @@ package org.apache.inlong.manager.pojo.workflow;
 
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
 import lombok.Data;
+import lombok.NoArgsConstructor;
 
 import java.util.List;
 
@@ -27,6 +30,9 @@ import java.util.List;
  * Workflow result info
  */
 @Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
 @ApiModel("Workflow result info")
 public class WorkflowResult {