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/03/10 16:44:42 UTC

[incubator-inlong] branch master updated: [INLONG-3046][Manager] The status was incorrect after approving an inlong group (#3048)

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/incubator-inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 4d13d88  [INLONG-3046][Manager] The status was incorrect after approving an inlong group (#3048)
4d13d88 is described below

commit 4d13d882b0e393c05543e5df198a881feca70b2c
Author: healchow <he...@gmail.com>
AuthorDate: Fri Mar 11 00:44:37 2022 +0800

    [INLONG-3046][Manager] The status was incorrect after approving an inlong group (#3048)
---
 .../client/api/inner/InnerInlongManagerClient.java | 30 +++-----
 .../common/pojo/group/InlongGroupRequest.java      | 16 +----
 .../dao/mapper/InlongGroupEntityMapper.java        |  2 +
 .../manager/dao/mapper/StreamSinkEntityMapper.java | 10 +--
 .../resources/mappers/InlongGroupEntityMapper.xml  |  7 ++
 .../resources/mappers/StreamSinkEntityMapper.xml   | 35 ++++------
 .../core/impl/InlongGroupProcessOperation.java     | 79 +++++++---------------
 .../service/core/impl/InlongGroupServiceImpl.java  | 67 ++++++++----------
 .../service/sink/StreamSinkServiceImpl.java        | 30 ++++----
 .../listener/GroupCompleteProcessListener.java     | 14 ++--
 .../group/listener/GroupPassTaskListener.java      |  8 +--
 .../listener/UpdateGroupCompleteListener.java      | 11 +--
 .../core/impl/InlongGroupProcessOperationTest.java |  5 +-
 inlong-manager/manager-test/pom.xml                | 30 ++++----
 .../org/apache/inlong/manager/web/UtilsTest.java   | 40 ++++-------
 inlong-manager/pom.xml                             | 39 ++++++-----
 16 files changed, 170 insertions(+), 253 deletions(-)

diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/InnerInlongManagerClient.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/InnerInlongManagerClient.java
index eb674c9..8c8c4d7 100644
--- a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/InnerInlongManagerClient.java
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/InnerInlongManagerClient.java
@@ -62,15 +62,11 @@ public class InnerInlongManagerClient {
 
     private static final String HTTP_PATH = "api/inlong/manager";
 
-    private OkHttpClient httpClient;
-
-    private String host;
-
-    private int port;
-
-    private String uname;
-
-    private String passwd;
+    private final OkHttpClient httpClient;
+    private final String host;
+    private final int port;
+    private final String uname;
+    private final String passwd;
 
     public InnerInlongManagerClient(InlongClientImpl inlongClient) {
         ClientConfiguration configuration = inlongClient.getConfiguration();
@@ -207,16 +203,14 @@ public class InnerInlongManagerClient {
      *
      * @return groupId && errMsg
      */
-    public Pair<String, String> updateGroup(InlongGroupRequest groupInfo) {
-        groupInfo.setCreateTime(null);
-        groupInfo.setModifyTime(null);
+    public Pair<String, String> updateGroup(InlongGroupRequest groupRequest) {
         String path = HTTP_PATH + "/group/update";
-        final String biz = GsonUtil.toJson(groupInfo);
-        final RequestBody bizBody = RequestBody.create(MediaType.parse("application/json"), biz);
+        final String group = GsonUtil.toJson(groupRequest);
+        final RequestBody groupBody = RequestBody.create(MediaType.parse("application/json"), group);
         final String url = formatUrl(path);
         Request request = new Request.Builder()
                 .url(url)
-                .method("POST", bizBody)
+                .method("POST", groupBody)
                 .build();
 
         Call call = httpClient.newCall(request);
@@ -631,9 +625,6 @@ public class InnerInlongManagerClient {
 
     /**
      * get inlong group error messages
-     *
-     * @param inlongGroupId
-     * @return
      */
     public List<EventLogView> getInlongGroupError(String inlongGroupId) {
         final String path = HTTP_PATH + "/workflow/event/list";
@@ -662,9 +653,6 @@ public class InnerInlongManagerClient {
 
     /**
      * get inlong group error messages
-     *
-     * @param inlongGroupId
-     * @return
      */
     public List<InlongStreamConfigLogListResponse> getStreamLogs(String inlongGroupId, String inlongStreamId) {
         final String path = HTTP_PATH + "/stream/config/log/list";
diff --git a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/group/InlongGroupRequest.java b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/group/InlongGroupRequest.java
index 8909e36..3bc97f9 100644
--- a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/group/InlongGroupRequest.java
+++ b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/group/InlongGroupRequest.java
@@ -17,7 +17,6 @@
 
 package org.apache.inlong.manager.common.pojo.group;
 
-import com.fasterxml.jackson.annotation.JsonFormat;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 import lombok.AllArgsConstructor;
@@ -26,7 +25,6 @@ import lombok.Data;
 import lombok.NoArgsConstructor;
 
 import javax.validation.constraints.NotNull;
-import java.util.Date;
 import java.util.List;
 
 /**
@@ -99,21 +97,9 @@ public class InlongGroupRequest {
     @ApiModelProperty(value = "Name of followers, separated by commas")
     private String followers;
 
-    private Integer status;
-
-    @ApiModelProperty(value = "is deleted? 0: deleted, 1: not deleted")
-    private Integer isDeleted = 0;
-
+    @ApiModelProperty(value = "Name of creator")
     private String creator;
 
-    private String modifier;
-
-    @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss")
-    private Date createTime;
-
-    @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss")
-    private Date modifyTime;
-
     @ApiModelProperty(value = "Temporary view, string in JSON format")
     private String tempView;
 
diff --git a/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/InlongGroupEntityMapper.java b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/InlongGroupEntityMapper.java
index 9013ae1..c2b7e10 100644
--- a/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/InlongGroupEntityMapper.java
+++ b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/InlongGroupEntityMapper.java
@@ -41,6 +41,8 @@ public interface InlongGroupEntityMapper {
 
     InlongGroupEntity selectByGroupId(String groupId);
 
+    InlongGroupEntity selectByGroupIdForUpdate(String groupId);
+
     Integer selectIdentifierExist(String groupId);
 
     List<InlongGroupEntity> selectByCondition(InlongGroupPageRequest request);
diff --git a/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/StreamSinkEntityMapper.java b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/StreamSinkEntityMapper.java
index f8606b5..a09f5c3 100644
--- a/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/StreamSinkEntityMapper.java
+++ b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/StreamSinkEntityMapper.java
@@ -59,13 +59,13 @@ public interface StreamSinkEntityMapper {
             @Param("streamId") String streamId);
 
     /**
-     * According to the inlong group id and inlong stream id, query valid sink information
+     * Query valid sink list by the given group id and stream id.
      *
-     * @param groupId inlong group id
-     * @param streamId inlong stream id
-     * @return Sink entity list
+     * @param groupId Inlong group id.
+     * @param streamId Inlong stream id.
+     * @return Sink entity list.
      */
-    List<StreamSinkEntity> selectByIdentifier(@Param("groupId") String groupId, @Param("streamId") String streamId);
+    List<StreamSinkEntity> selectByRelatedId(@Param("groupId") String groupId, @Param("streamId") String streamId);
 
     /**
      * According to the group id, stream id and sink type, query valid sink entity list.
diff --git a/inlong-manager/manager-dao/src/main/resources/mappers/InlongGroupEntityMapper.xml b/inlong-manager/manager-dao/src/main/resources/mappers/InlongGroupEntityMapper.xml
index 4999d54..965dfc8 100644
--- a/inlong-manager/manager-dao/src/main/resources/mappers/InlongGroupEntityMapper.xml
+++ b/inlong-manager/manager-dao/src/main/resources/mappers/InlongGroupEntityMapper.xml
@@ -79,6 +79,13 @@
         where inlong_group_id = #{groupId, jdbcType=VARCHAR}
         and is_deleted = 0
     </select>
+    <select id="selectByGroupIdForUpdate" resultMap="BaseResultMap">
+        select
+        <include refid="Base_Column_List"/>
+        from inlong_group
+        where inlong_group_id = #{groupId, jdbcType=VARCHAR}
+        and is_deleted = 0 for update
+    </select>
     <select id="selectIdentifierExist" resultType="java.lang.Integer">
         select count(1)
         from inlong_group
diff --git a/inlong-manager/manager-dao/src/main/resources/mappers/StreamSinkEntityMapper.xml b/inlong-manager/manager-dao/src/main/resources/mappers/StreamSinkEntityMapper.xml
index ba625c4..a1ecfc5 100644
--- a/inlong-manager/manager-dao/src/main/resources/mappers/StreamSinkEntityMapper.xml
+++ b/inlong-manager/manager-dao/src/main/resources/mappers/StreamSinkEntityMapper.xml
@@ -106,8 +106,7 @@
           and s.inlong_group_id = #{groupId, jdbcType=VARCHAR}
           and s.inlong_stream_id = #{streamId, jdbcType=VARCHAR}
     </select>
-    <select id="selectByIdentifier"
-            resultType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
+    <select id="selectByRelatedId" resultType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
         select
         <include refid="Base_Column_List"/>
         from stream_sink
@@ -119,8 +118,7 @@
             </if>
         </where>
     </select>
-    <select id="selectByIdAndType"
-            resultType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
+    <select id="selectByIdAndType" resultType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
         select
         <include refid="Base_Column_List"/>
         from stream_sink
@@ -143,8 +141,7 @@
             and sink_type = #{sinkType, jdbcType=VARCHAR}
             and is_deleted = 0
             and inlong_stream_id in
-            <foreach collection="streamIdList" open="(" close=")" separator="," index="index"
-                    item="item">
+            <foreach collection="streamIdList" open="(" close=")" separator="," index="index" item="item">
                 #{item}
             </foreach>
         </where>
@@ -162,8 +159,7 @@
             </if>
         </where>
     </select>
-    <select id="selectAllConfig"
-            resultType="org.apache.inlong.manager.common.pojo.sink.SinkForSortDTO">
+    <select id="selectAllConfig" resultType="org.apache.inlong.manager.common.pojo.sink.SinkForSortDTO">
         select sink.id,
         sink.inlong_group_id,
         sink.inlong_stream_id,
@@ -204,19 +200,15 @@
         insert into stream_sink (id, inlong_group_id, inlong_stream_id,
                                  sink_type, sink_name, storage_period,
                                  enable_create_resource, ext_params,
-                                 operate_log, status,
-                                 previous_status, is_deleted, creator,
-                                 modifier, create_time, modify_time)
-        values (#{id,jdbcType=INTEGER}, #{inlongGroupId,jdbcType=VARCHAR},
-                #{inlongStreamId,jdbcType=VARCHAR},
-                #{sinkType,jdbcType=VARCHAR}, #{sinkName,jdbcType=VARCHAR},
-                #{storagePeriod,jdbcType=INTEGER},
+                                 operate_log, status, previous_status,
+                                 is_deleted, creator, modifier,
+                                 create_time, modify_time)
+        values (#{id,jdbcType=INTEGER}, #{inlongGroupId,jdbcType=VARCHAR}, #{inlongStreamId,jdbcType=VARCHAR},
+                #{sinkType,jdbcType=VARCHAR}, #{sinkName,jdbcType=VARCHAR}, #{storagePeriod,jdbcType=INTEGER},
                 #{enableCreateResource,jdbcType=TINYINT}, #{extParams,jdbcType=VARCHAR},
-                #{operateLog,jdbcType=VARCHAR}, #{status,jdbcType=INTEGER},
-                #{previousStatus,jdbcType=INTEGER}, #{isDeleted,jdbcType=INTEGER},
-                #{creator,jdbcType=VARCHAR},
-                #{modifier,jdbcType=VARCHAR}, #{createTime,jdbcType=TIMESTAMP},
-                #{modifyTime,jdbcType=TIMESTAMP})
+                #{operateLog,jdbcType=VARCHAR}, #{status,jdbcType=INTEGER}, #{previousStatus,jdbcType=INTEGER},
+                #{isDeleted,jdbcType=INTEGER}, #{creator,jdbcType=VARCHAR}, #{modifier,jdbcType=VARCHAR},
+                #{createTime,jdbcType=TIMESTAMP}, #{modifyTime,jdbcType=TIMESTAMP})
     </insert>
     <insert id="insertSelective" useGeneratedKeys="true" keyProperty="id"
             parameterType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
@@ -375,8 +367,7 @@
         </set>
         where id = #{id,jdbcType=INTEGER}
     </update>
-    <update id="updateByPrimaryKey"
-            parameterType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
+    <update id="updateByPrimaryKey" parameterType="org.apache.inlong.manager.dao.entity.StreamSinkEntity">
         update stream_sink
         set inlong_group_id        = #{inlongGroupId,jdbcType=VARCHAR},
             inlong_stream_id       = #{inlongStreamId,jdbcType=VARCHAR},
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperation.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperation.java
index d16149f..57cdd70 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperation.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperation.java
@@ -17,10 +17,7 @@
 
 package org.apache.inlong.manager.service.core.impl;
 
-import org.apache.inlong.manager.common.enums.Constant;
-import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
 import org.apache.inlong.manager.common.enums.GroupState;
-import org.apache.inlong.manager.common.exceptions.BusinessException;
 import org.apache.inlong.manager.common.pojo.group.InlongGroupInfo;
 import org.apache.inlong.manager.common.pojo.stream.InlongStreamInfo;
 import org.apache.inlong.manager.common.pojo.stream.StreamBriefResponse;
@@ -29,7 +26,6 @@ import org.apache.inlong.manager.common.pojo.workflow.form.NewGroupProcessForm;
 import org.apache.inlong.manager.common.pojo.workflow.form.UpdateGroupProcessForm;
 import org.apache.inlong.manager.common.pojo.workflow.form.UpdateGroupProcessForm.OperateType;
 import org.apache.inlong.manager.common.util.CommonBeanUtils;
-import org.apache.inlong.manager.common.util.Preconditions;
 import org.apache.inlong.manager.dao.entity.InlongStreamEntity;
 import org.apache.inlong.manager.dao.mapper.InlongStreamEntityMapper;
 import org.apache.inlong.manager.service.core.InlongGroupService;
@@ -65,51 +61,40 @@ public class InlongGroupProcessOperation {
      *
      * @param groupId Inlong group id
      * @param operator Operator name
-     * @return WorkflowProcess information
+     * @return Workflow result
      */
     public WorkflowResult startProcess(String groupId, String operator) {
         LOGGER.info("begin to start approve process, groupId = {}, operator = {}", groupId, operator);
-        final GroupState nextState = GroupState.GROUP_WAIT_APPROVAL;
-        InlongGroupInfo groupInfo = validateGroup(groupId, nextState);
-
-        // Modify inlong group status
-        groupInfo.setStatus(nextState.getCode());
-        groupService.update(groupInfo.genRequest(), operator);
-
+        groupService.updateStatus(groupId, GroupState.GROUP_WAIT_APPROVAL.getCode(), operator);
         // Initiate the approval process
-        NewGroupProcessForm form = genNewGroupProcessForm(groupInfo);
+        NewGroupProcessForm form = genNewGroupProcessForm(groupId);
         return workflowService.start(ProcessName.NEW_GROUP_PROCESS, operator, form);
     }
 
     /**
-     * Suspend resource application group which is started up successfully, stop dataSource collecting task
-     * and sort task related to application group asynchronously, persist the application status if necessary
+     * Suspend resource application group which is started up successfully,
+     * stop source and sort task related to application group asynchronously,
+     * persist the application status if necessary.
      *
-     * @return WorkflowProcess information
+     * @return Workflow result
      */
     public WorkflowResult suspendProcess(String groupId, String operator) {
         LOGGER.info("begin to suspend process, groupId = {}, operator = {}", groupId, operator);
-        final GroupState nextState = GroupState.GROUP_SUSPEND_ING;
-        InlongGroupInfo groupInfo = validateGroup(groupId, nextState);
-
-        groupInfo.setStatus(nextState.getCode());
-        groupService.update(groupInfo.genRequest(), operator);
-        UpdateGroupProcessForm form = genUpdateGroupProcessForm(groupInfo, OperateType.SUSPEND);
+        groupService.updateStatus(groupId, GroupState.GROUP_SUSPEND_ING.getCode(), operator);
+        UpdateGroupProcessForm form = genUpdateGroupProcessForm(groupId, OperateType.SUSPEND);
         return workflowService.start(ProcessName.SUSPEND_GROUP_PROCESS, operator, form);
     }
 
     /**
-     * Restart resource application group which is suspended successfully, starting from the last persist snapshot
+     * Restart resource application group which is suspended successfully,
+     * starting from the last persist snapshot.
      *
-     * @return WorkflowProcess information
+     * @return Workflow result
      */
     public WorkflowResult restartProcess(String groupId, String operator) {
         LOGGER.info("begin to restart process, groupId = {}, operator = {}", groupId, operator);
-        GroupState nextState = GroupState.GROUP_RESTART_ING;
-        InlongGroupInfo groupInfo = validateGroup(groupId, nextState);
-        groupInfo.setStatus(nextState.getCode());
-        groupService.update(groupInfo.genRequest(), operator);
-        UpdateGroupProcessForm form = genUpdateGroupProcessForm(groupInfo, OperateType.RESTART);
+        groupService.updateStatus(groupId, GroupState.GROUP_RESTART_ING.getCode(), operator);
+        UpdateGroupProcessForm form = genUpdateGroupProcessForm(groupId, OperateType.RESTART);
         return workflowService.start(ProcessName.RESTART_GROUP_PROCESS, operator, form);
     }
 
@@ -118,9 +103,8 @@ public class InlongGroupProcessOperation {
      */
     public boolean deleteProcess(String groupId, String operator) {
         LOGGER.info("begin to delete process, groupId = {}, operator = {}", groupId, operator);
-        InlongGroupInfo groupInfo = groupService.get(groupId);
-        UpdateGroupProcessForm form = genUpdateGroupProcessForm(groupInfo, OperateType.DELETE);
         try {
+            UpdateGroupProcessForm form = genUpdateGroupProcessForm(groupId, OperateType.DELETE);
             workflowService.start(ProcessName.DELETE_GROUP_PROCESS, operator, form);
         } catch (Exception ex) {
             LOGGER.error("exception while delete process, groupId = {}, operator = {}", groupId, operator, ex);
@@ -132,43 +116,28 @@ public class InlongGroupProcessOperation {
     /**
      * Generate the form of [New Group Workflow]
      */
-    public NewGroupProcessForm genNewGroupProcessForm(InlongGroupInfo groupInfo) {
+    public NewGroupProcessForm genNewGroupProcessForm(String groupId) {
         NewGroupProcessForm form = new NewGroupProcessForm();
+        InlongGroupInfo groupInfo = groupService.get(groupId);
         form.setGroupInfo(groupInfo);
-        // Query all inlong streams under the groupId and the sink information of each inlong stream
         List<StreamBriefResponse> infoList = streamService.getBriefList(groupInfo.getInlongGroupId());
         form.setStreamInfoList(infoList);
         return form;
     }
 
-    private UpdateGroupProcessForm genUpdateGroupProcessForm(InlongGroupInfo groupInfo,
-            OperateType operateType) {
-        UpdateGroupProcessForm updateForm = new UpdateGroupProcessForm();
+    private UpdateGroupProcessForm genUpdateGroupProcessForm(String groupId, OperateType operateType) {
+        InlongGroupInfo groupInfo = groupService.get(groupId);
+        UpdateGroupProcessForm form = new UpdateGroupProcessForm();
         if (OperateType.RESTART == operateType) {
             List<InlongStreamEntity> inlongStreamEntityList =
                     streamMapper.selectByGroupId(groupInfo.getInlongGroupId());
             List<InlongStreamInfo> inlongStreamInfoList = CommonBeanUtils.copyListProperties(inlongStreamEntityList,
                     InlongStreamInfo::new);
-            updateForm.setInlongStreamInfoList(inlongStreamInfoList);
+            form.setInlongStreamInfoList(inlongStreamInfoList);
         }
-        updateForm.setGroupInfo(groupInfo);
-        updateForm.setOperateType(operateType);
-        return updateForm;
-    }
-
-    private InlongGroupInfo validateGroup(String groupId, GroupState nextState) {
-        Preconditions.checkNotNull(groupId, Constant.GROUP_ID_IS_EMPTY);
-
-        // Check whether the current status of the inlong group allows the process to be re-initiated
-        InlongGroupInfo groupInfo = groupService.get(groupId);
-        if (groupInfo == null) {
-            LOGGER.error("inlong group not found by groupId={}", groupId);
-            throw new BusinessException(ErrorCodeEnum.GROUP_NOT_FOUND);
-        }
-        GroupState curState = GroupState.forCode(groupInfo.getStatus());
-        Preconditions.checkTrue(GroupState.isAllowedTransition(curState, nextState),
-                String.format("current status was not allowed to %s workflow", nextState.getDescription()));
-        return groupInfo;
+        form.setGroupInfo(groupInfo);
+        form.setOperateType(operateType);
+        return form;
     }
 
 }
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupServiceImpl.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupServiceImpl.java
index 5289333..9fde978 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupServiceImpl.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/InlongGroupServiceImpl.java
@@ -52,6 +52,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
+import org.springframework.transaction.annotation.Isolation;
 import org.springframework.transaction.annotation.Transactional;
 
 import java.util.Arrays;
@@ -213,10 +214,10 @@ public class InlongGroupServiceImpl implements InlongGroupService {
 
     @Transactional(rollbackFor = Throwable.class)
     @Override
-    public String update(InlongGroupRequest groupInfo, String operator) {
-        LOGGER.debug("begin to update inlong group={}", groupInfo);
-        Preconditions.checkNotNull(groupInfo, "inlong group is empty");
-        String groupId = groupInfo.getInlongGroupId();
+    public String update(InlongGroupRequest groupRequest, String operator) {
+        LOGGER.debug("begin to update inlong group={}", groupRequest);
+        Preconditions.checkNotNull(groupRequest, "inlong group is empty");
+        String groupId = groupRequest.getInlongGroupId();
         Preconditions.checkNotNull(groupId, Constant.GROUP_ID_IS_EMPTY);
 
         InlongGroupEntity entity = groupMapper.selectByGroupId(groupId);
@@ -226,20 +227,21 @@ public class InlongGroupServiceImpl implements InlongGroupService {
         }
 
         // Check whether the current status can be modified
-        this.checkGroupCanUpdate(entity, groupInfo, operator);
-
-        CommonBeanUtils.copyProperties(groupInfo, entity, true);
-        entity.setStatus(groupInfo.getStatus());
+        this.checkGroupCanUpdate(entity, groupRequest, operator);
+        CommonBeanUtils.copyProperties(groupRequest, entity, true);
+        if (GroupState.GROUP_CONFIG_FAILED.getCode().equals(entity.getStatus())) {
+            entity.setStatus(GroupState.GROUP_WAIT_SUBMIT.getCode());
+        }
 
         entity.setModifier(operator);
         groupMapper.updateByIdentifierSelective(entity);
 
         // Save extended information
-        this.saveOrUpdateExt(groupId, groupInfo.getExtList());
+        this.saveOrUpdateExt(groupId, groupRequest.getExtList());
 
         // Update the Pulsar info
-        if (Constant.MIDDLEWARE_PULSAR.equals(groupInfo.getMiddlewareType())) {
-            InlongGroupPulsarInfo pulsarInfo = (InlongGroupPulsarInfo) groupInfo.getMqExtInfo();
+        if (Constant.MIDDLEWARE_PULSAR.equals(groupRequest.getMiddlewareType())) {
+            InlongGroupPulsarInfo pulsarInfo = (InlongGroupPulsarInfo) groupRequest.getMqExtInfo();
             Preconditions.checkNotNull(pulsarInfo, "Pulsar info cannot be empty, as the middleware is Pulsar");
             Integer writeQuorum = pulsarInfo.getWriteQuorum();
             Integer ackQuorum = pulsarInfo.getAckQuorum();
@@ -281,46 +283,35 @@ public class InlongGroupServiceImpl implements InlongGroupService {
         }
         // Check whether the current state supports modification
         GroupState curState = GroupState.forCode(entity.getStatus());
-        if (groupInfo.getStatus() != null) {
-            GroupState nextState = GroupState.forCode(groupInfo.getStatus());
-            if (!GroupState.isAllowedTransition(curState, nextState)) {
-                String errMsg = String.format("Current state=%s is not allowed to transfer to state=%s",
-                        curState, nextState);
-                LOGGER.error(errMsg);
-                throw new BusinessException(ErrorCodeEnum.GROUP_UPDATE_NOT_ALLOWED, errMsg);
-            }
-        } else {
-            if (!GroupState.isAllowedUpdate(curState)) {
-                String errMsg = String.format("Current state=%s is not allowed to update",
-                        curState);
-                LOGGER.error(errMsg);
-                throw new BusinessException(ErrorCodeEnum.GROUP_UPDATE_NOT_ALLOWED, errMsg);
-            }
+        if (!GroupState.isAllowedUpdate(curState)) {
+            String errMsg = String.format("Current state=%s is not allowed to update", curState);
+            LOGGER.error(errMsg);
+            throw new BusinessException(ErrorCodeEnum.GROUP_UPDATE_NOT_ALLOWED, errMsg);
         }
     }
 
     @Override
+    @Transactional(rollbackFor = Throwable.class, isolation = Isolation.REPEATABLE_READ)
     public boolean updateStatus(String groupId, Integer status, String operator) {
-        LOGGER.debug("begin to update inlong group status, groupId={}, status={}, username={}", groupId, status,
-                operator);
+        LOGGER.info("begin to update group status to [{}] by groupId={}, username={}", status, groupId, operator);
         Preconditions.checkNotNull(groupId, Constant.GROUP_ID_IS_EMPTY);
-        InlongGroupEntity entity = groupMapper.selectByGroupId(groupId);
+        InlongGroupEntity entity = groupMapper.selectByGroupIdForUpdate(groupId);
         if (entity == null) {
             LOGGER.error("inlong group not found by groupId={}", groupId);
             throw new BusinessException(ErrorCodeEnum.GROUP_NOT_FOUND);
         }
         GroupState curState = GroupState.forCode(entity.getStatus());
         GroupState nextState = GroupState.forCode(status);
-        if (GroupState.isAllowedTransition(curState, nextState)) {
-            groupMapper.updateStatus(groupId, status, operator);
-            LOGGER.info("success to update inlong group status for groupId={}", groupId);
-            return true;
-        } else {
-            String warnMsg = String.format("Current state=%s is not allowed to transfer to state=%s",
+        if (!GroupState.isAllowedTransition(curState, nextState)) {
+            String errorMsg = String.format("Current state=%s is not allowed to transfer to state=%s",
                     curState, nextState);
-            LOGGER.warn(warnMsg);
-            return false;
+            LOGGER.error(errorMsg);
+            throw new BusinessException(errorMsg);
         }
+
+        groupMapper.updateStatus(groupId, status, operator);
+        LOGGER.info("success to update inlong group status to [{}] for groupId={}", status, groupId);
+        return true;
     }
 
     @Transactional(rollbackFor = Throwable.class)
@@ -452,8 +443,8 @@ public class InlongGroupServiceImpl implements InlongGroupService {
         return true;
     }
 
-    @Transactional(rollbackFor = Throwable.class)
     @Override
+    @Transactional(rollbackFor = Throwable.class)
     public void saveOrUpdateExt(String groupId, List<InlongGroupExtInfo> infoList) {
         LOGGER.debug("begin to save or update inlong group ext info, groupId={}, ext={}", groupId, infoList);
 
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/StreamSinkServiceImpl.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/StreamSinkServiceImpl.java
index 6646904..f126e6c 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/StreamSinkServiceImpl.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/StreamSinkServiceImpl.java
@@ -23,21 +23,12 @@ import com.github.pagehelper.PageInfo;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
-import java.util.concurrent.TimeUnit;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.inlong.manager.common.enums.Constant;
 import org.apache.inlong.manager.common.enums.EntityStatus;
 import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.enums.GroupState;
 import org.apache.inlong.manager.common.enums.SinkType;
 import org.apache.inlong.manager.common.pojo.group.InlongGroupInfo;
 import org.apache.inlong.manager.common.pojo.sink.SinkApproveDTO;
@@ -63,6 +54,17 @@ import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
 import org.springframework.transaction.annotation.Transactional;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
+import java.util.concurrent.TimeUnit;
+
 /**
  * Implementation of sink service interface
  */
@@ -114,7 +116,7 @@ public class StreamSinkServiceImpl implements StreamSinkService {
 
         // If the inlong group status is [Configuration Successful], then asynchronously initiate
         // the [Single inlong stream Resource Creation] workflow
-        if (EntityStatus.GROUP_CONFIG_SUCCESSFUL.getCode().equals(groupEntity.getStatus())) {
+        if (GroupState.GROUP_CONFIG_SUCCESSFUL.getCode().equals(groupEntity.getStatus())) {
             executorService.execute(new WorkflowStartRunnable(operator, groupEntity, streamId));
         }
 
@@ -143,7 +145,7 @@ public class StreamSinkServiceImpl implements StreamSinkService {
         LOGGER.debug("begin to list sink by groupId={}, streamId={}", groupId, streamId);
         Preconditions.checkNotNull(groupId, Constant.GROUP_ID_IS_EMPTY);
 
-        List<StreamSinkEntity> entityList = sinkMapper.selectByIdentifier(groupId, streamId);
+        List<StreamSinkEntity> entityList = sinkMapper.selectByRelatedId(groupId, streamId);
         if (CollectionUtils.isEmpty(entityList)) {
             return Collections.emptyList();
         }
@@ -265,7 +267,7 @@ public class StreamSinkServiceImpl implements StreamSinkService {
         commonOperateService.checkGroupStatus(groupId, operator);
 
         Date now = new Date();
-        List<StreamSinkEntity> entityList = sinkMapper.selectByIdentifier(groupId, streamId);
+        List<StreamSinkEntity> entityList = sinkMapper.selectByRelatedId(groupId, streamId);
         if (CollectionUtils.isNotEmpty(entityList)) {
             entityList.forEach(entity -> {
                 Integer id = entity.getId();
@@ -294,7 +296,7 @@ public class StreamSinkServiceImpl implements StreamSinkService {
         // Check if it can be deleted
         commonOperateService.checkGroupStatus(groupId, operator);
 
-        List<StreamSinkEntity> entityList = sinkMapper.selectByIdentifier(groupId, streamId);
+        List<StreamSinkEntity> entityList = sinkMapper.selectByRelatedId(groupId, streamId);
         if (CollectionUtils.isNotEmpty(entityList)) {
             entityList.forEach(entity -> {
                 sinkMapper.deleteByPrimaryKey(entity.getId());
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupCompleteProcessListener.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupCompleteProcessListener.java
index b08a76b..9aee197 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupCompleteProcessListener.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupCompleteProcessListener.java
@@ -22,7 +22,6 @@ import org.apache.inlong.manager.common.enums.EntityStatus;
 import org.apache.inlong.manager.common.enums.GroupState;
 import org.apache.inlong.manager.common.enums.SourceState;
 import org.apache.inlong.manager.common.exceptions.WorkflowListenerException;
-import org.apache.inlong.manager.common.pojo.group.InlongGroupInfo;
 import org.apache.inlong.manager.common.pojo.workflow.form.GroupResourceProcessForm;
 import org.apache.inlong.manager.dao.mapper.SourceFileDetailEntityMapper;
 import org.apache.inlong.manager.service.core.InlongGroupService;
@@ -64,20 +63,15 @@ public class GroupCompleteProcessListener implements ProcessEventListener {
     @Override
     public ListenerResult listen(WorkflowContext context) throws WorkflowListenerException {
         GroupResourceProcessForm form = (GroupResourceProcessForm) context.getProcessForm();
-
         String groupId = form.getInlongGroupId();
         String applicant = context.getApplicant();
-
-        InlongGroupInfo groupInfo = form.getGroupInfo();
-        groupInfo.setStatus(GroupState.GROUP_CONFIG_SUCCESSFUL.getCode());
-
-        // Update inlong group
-        groupService.update(groupInfo.genRequest(), applicant);
-        // Update inlong stream status
+        // Update status of inlong group, inlong stream...
+        groupService.updateStatus(groupId, GroupState.GROUP_CONFIG_SUCCESSFUL.getCode(), applicant);
         streamService.updateStatus(groupId, null, EntityStatus.STREAM_CONFIG_SUCCESSFUL.getCode(), applicant);
+
+        // TODO Remove update source file / db detail status
         // Update file data source status
         fileDetailMapper.updateStatusAfterApprove(groupId, null, EntityStatus.AGENT_ADD.getCode(), applicant);
-        // TODO Update source db detail status
         // dbDetailMapper.updateStatusAfterApprove(bid, null, EntityStatus.AGENT_WAIT_CREATE.getCode(), username);
 
         // Update stream source status
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupPassTaskListener.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupPassTaskListener.java
index c9c8960..b75ec1f 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupPassTaskListener.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/GroupPassTaskListener.java
@@ -17,8 +17,6 @@
 
 package org.apache.inlong.manager.service.workflow.group.listener;
 
-import java.util.List;
-import java.util.Objects;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.inlong.manager.common.enums.GroupState;
 import org.apache.inlong.manager.common.exceptions.WorkflowListenerException;
@@ -36,6 +34,9 @@ import org.apache.inlong.manager.workflow.event.task.TaskEventListener;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
+import java.util.List;
+import java.util.Objects;
+
 /**
  * Approve pass listener for new inlong group
  */
@@ -61,7 +62,6 @@ public class GroupPassTaskListener implements TaskEventListener {
         InlongGroupApproveForm form = (InlongGroupApproveForm) context.getActionContext().getForm();
 
         InlongGroupApproveRequest approveInfo = form.getGroupApproveInfo();
-
         // Only the [Wait approval] status allowed the passing operation
         String groupId = approveInfo.getInlongGroupId();
         InlongGroupEntity entity = groupMapper.selectByGroupId(groupId);
@@ -69,7 +69,7 @@ public class GroupPassTaskListener implements TaskEventListener {
             throw new WorkflowListenerException("inlong group not found with group id=" + groupId);
         }
         if (!Objects.equals(GroupState.GROUP_WAIT_APPROVAL.getCode(), entity.getStatus())) {
-            throw new WorkflowListenerException("current status was not allowed to approve inlong group");
+            throw new WorkflowListenerException("inlong group status is [wait_approval], not allowed to approve again");
         }
 
         // Save the inlong group information after approval
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/UpdateGroupCompleteListener.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/UpdateGroupCompleteListener.java
index c60c2b7..9dffedc 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/UpdateGroupCompleteListener.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/workflow/group/listener/UpdateGroupCompleteListener.java
@@ -51,20 +51,21 @@ public class UpdateGroupCompleteListener implements ProcessEventListener {
         String username = context.getApplicant();
         OperateType operateType = form.getOperateType();
         InlongGroupInfo groupInfo = form.getGroupInfo();
+        Integer nextStatus;
         switch (operateType) {
             case RESTART:
-                groupInfo.setStatus(GroupState.GROUP_RESTART.getCode());
+                nextStatus = GroupState.GROUP_RESTART.getCode();
                 break;
             case SUSPEND:
-                groupInfo.setStatus(GroupState.GROUP_SUSPEND.getCode());
+                nextStatus = GroupState.GROUP_SUSPEND.getCode();
                 break;
             case DELETE:
-                groupInfo.setStatus(GroupState.GROUP_DELETE.getCode());
+                nextStatus = GroupState.GROUP_DELETE.getCode();
                 break;
             default:
-                throw new RuntimeException(String.format("Unsupport operation=%s for Inlong group", operateType));
+                throw new RuntimeException(String.format("Unsupported operation=%s for Inlong group", operateType));
         }
-        groupService.update(groupInfo.genRequest(), username);
+        groupService.updateStatus(groupInfo.getInlongGroupId(), nextStatus, username);
         return ListenerResult.success();
     }
 
diff --git a/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperationTest.java b/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperationTest.java
index 19a7ade..8622abb 100644
--- a/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperationTest.java
+++ b/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/impl/InlongGroupProcessOperationTest.java
@@ -22,8 +22,8 @@ import org.apache.inlong.manager.common.enums.EntityStatus;
 import org.apache.inlong.manager.common.enums.GroupState;
 import org.apache.inlong.manager.common.enums.ProcessStatus;
 import org.apache.inlong.manager.common.pojo.group.InlongGroupInfo;
-import org.apache.inlong.manager.common.pojo.group.InlongGroupRequest;
 import org.apache.inlong.manager.common.pojo.group.InlongGroupPulsarInfo;
+import org.apache.inlong.manager.common.pojo.group.InlongGroupRequest;
 import org.apache.inlong.manager.common.pojo.workflow.ProcessResponse;
 import org.apache.inlong.manager.common.pojo.workflow.WorkflowResult;
 import org.apache.inlong.manager.service.ServiceBaseTest;
@@ -69,7 +69,6 @@ public class InlongGroupProcessOperationTest extends ServiceBaseTest {
         pulsarInfo.setInlongGroupId(GROUP_ID);
         groupInfo.setMqExtInfo(pulsarInfo);
         groupService.save(groupInfo, OPERATOR);
-        groupInfo.setStatus(status);
         groupService.update(groupInfo, OPERATOR);
     }
 
@@ -93,7 +92,7 @@ public class InlongGroupProcessOperationTest extends ServiceBaseTest {
         groupService.update(groupInfo.genRequest(), OPERATOR);
         groupInfo.setStatus(GroupState.GROUP_CONFIG_SUCCESSFUL.getCode());
         groupService.update(groupInfo.genRequest(), OPERATOR);
-        groupInfo = groupService.get(GROUP_ID);
+
         WorkflowResult result = groupProcessOperation.suspendProcess(GROUP_ID, OPERATOR);
         ProcessResponse response = result.getProcessInfo();
         Assert.assertSame(response.getStatus(), ProcessStatus.COMPLETED);
diff --git a/inlong-manager/manager-test/pom.xml b/inlong-manager/manager-test/pom.xml
index f763b71..012e99c 100644
--- a/inlong-manager/manager-test/pom.xml
+++ b/inlong-manager/manager-test/pom.xml
@@ -15,9 +15,9 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xmlns="http://maven.apache.org/POM/4.0.0"
+        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <artifactId>inlong-manager</artifactId>
         <groupId>org.apache.inlong</groupId>
@@ -30,33 +30,33 @@
 
     <dependencies>
         <dependency>
-            <groupId>org.powermock</groupId>
-            <artifactId>powermock-api-mockito2</artifactId>
-            <version>${powermock.version}</version>
-        </dependency>
-        <dependency>
             <groupId>org.springframework</groupId>
             <artifactId>spring-web</artifactId>
-            <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.springframework.boot</groupId>
             <artifactId>spring-boot-test</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-test</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito2</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.mvnsearch</groupId>
             <artifactId>h2-functions-4-mysql</artifactId>
-            <version>${h2_mysql.version}</version>
         </dependency>
-
         <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-test</artifactId>
-        </dependency>
     </dependencies>
 
 </project>
\ No newline at end of file
diff --git a/inlong-manager/manager-web/src/test/java/org/apache/inlong/manager/web/UtilsTest.java b/inlong-manager/manager-web/src/test/java/org/apache/inlong/manager/web/UtilsTest.java
index e56212d..ab0b2e9 100644
--- a/inlong-manager/manager-web/src/test/java/org/apache/inlong/manager/web/UtilsTest.java
+++ b/inlong-manager/manager-web/src/test/java/org/apache/inlong/manager/web/UtilsTest.java
@@ -20,18 +20,18 @@ package org.apache.inlong.manager.web;
 import org.apache.inlong.manager.common.pojo.group.InlongGroupRequest;
 import org.apache.inlong.manager.common.util.CommonBeanUtils;
 import org.apache.inlong.manager.dao.entity.InlongGroupEntity;
+import org.junit.Assert;
+import org.junit.Test;
+import org.springframework.beans.BeanUtils;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 import java.util.Optional;
-import org.junit.Assert;
-import org.junit.Test;
-import org.springframework.beans.BeanUtils;
 
 /**
  * Tool testing
- *
  */
 public class UtilsTest {
 
@@ -45,19 +45,13 @@ public class UtilsTest {
         entity.setCreateTime(new Date());
         entity.setModifyTime(null);
 
-        InlongGroupRequest info = new InlongGroupRequest();
-        info.setInlongGroupId("info");
-        info.setCreator("");
-        info.setModifier("user2");
-        info.setCreateTime(null);
-        info.setModifyTime(new Date());
+        InlongGroupRequest request = new InlongGroupRequest();
+        request.setInlongGroupId("info");
 
-        BeanUtils.copyProperties(info, entity);
-        Assert.assertNull(entity.getStatus());
-        Assert.assertEquals("", entity.getCreator());
-        Assert.assertEquals("user2", entity.getModifier());
-        Assert.assertNull(entity.getCreateTime());
-        Assert.assertNotNull(entity.getModifyTime());
+        BeanUtils.copyProperties(request, entity);
+        Assert.assertEquals(1, (int) entity.getStatus());
+        Assert.assertNotNull(entity.getCreateTime());
+        Assert.assertNull(entity.getModifyTime());
     }
 
     @Test
@@ -70,19 +64,13 @@ public class UtilsTest {
         entity.setCreateTime(new Date());
         entity.setModifyTime(null);
 
-        InlongGroupRequest info = new InlongGroupRequest();
-        info.setInlongGroupId("info");
-        info.setCreator("");
-        info.setModifier("user2");
-        info.setCreateTime(null);
-        info.setModifyTime(new Date());
+        InlongGroupRequest request = new InlongGroupRequest();
+        request.setInlongGroupId("info");
 
-        CommonBeanUtils.copyProperties(info, entity, true);
+        CommonBeanUtils.copyProperties(request, entity, true);
         Assert.assertEquals(1, (int) entity.getStatus());
-        Assert.assertEquals("", entity.getCreator());
-        Assert.assertEquals("user2", entity.getModifier());
         Assert.assertNotNull(entity.getCreateTime());
-        Assert.assertNotNull(entity.getModifyTime());
+        Assert.assertNull(entity.getModifyTime());
     }
 
     @Test
diff --git a/inlong-manager/pom.xml b/inlong-manager/pom.xml
index a661e72..a1edd87 100644
--- a/inlong-manager/pom.xml
+++ b/inlong-manager/pom.xml
@@ -52,7 +52,7 @@
         <pagehelper.springboot.version>1.3.0</pagehelper.springboot.version>
         <mysql.version>8.0.15</mysql.version>
         <h2.version>2.1.210</h2.version>
-        <h2_mysql.version>2.0.0</h2_mysql.version>
+        <h2-mysql.version>2.0.0</h2-mysql.version>
         <druid.version>1.2.5</druid.version>
         <shiro.version>1.8.0</shiro.version>
         <hive.version>3.1.2</hive.version>
@@ -124,17 +124,6 @@
             <artifactId>junit</artifactId>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>com.h2database</groupId>
-            <artifactId>h2</artifactId>
-            <version>${h2.version}</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mvnsearch</groupId>
-            <artifactId>h2-functions-4-mysql</artifactId>
-            <version>${h2_mysql.version}</version>
-        </dependency>
     </dependencies>
 
     <dependencyManagement>
@@ -151,11 +140,6 @@
             </dependency>
             <dependency>
                 <groupId>org.springframework.boot</groupId>
-                <artifactId>spring-boot-starter-security</artifactId>
-                <version>${spring-boot.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.springframework.boot</groupId>
                 <artifactId>spring-boot-starter-validation</artifactId>
                 <version>${spring-boot.version}</version>
             </dependency>
@@ -331,7 +315,6 @@
                 <artifactId>hive-jdbc</artifactId>
                 <version>${hive.version}</version>
             </dependency>
-
             <dependency>
                 <groupId>org.apache.hive</groupId>
                 <artifactId>hive-exec</artifactId>
@@ -414,7 +397,6 @@
                 <artifactId>httpcore</artifactId>
                 <version>${httpcore.version}</version>
             </dependency>
-
             <dependency>
                 <groupId>org.apache.httpcomponents</groupId>
                 <artifactId>httpclient</artifactId>
@@ -474,6 +456,23 @@
                 <version>${springframework.version}</version>
             </dependency>
 
+            <dependency>
+                <groupId>org.powermock</groupId>
+                <artifactId>powermock-api-mockito2</artifactId>
+                <version>${powermock.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.h2database</groupId>
+                <artifactId>h2</artifactId>
+                <version>${h2.version}</version>
+                <!--<scope>test</scope>-->
+            </dependency>
+            <dependency>
+                <groupId>org.mvnsearch</groupId>
+                <artifactId>h2-functions-4-mysql</artifactId>
+                <version>${h2-mysql.version}</version>
+                <!--<scope>test</scope>-->
+            </dependency>
         </dependencies>
     </dependencyManagement>
     <repositories>
@@ -493,6 +492,7 @@
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-compiler-plugin</artifactId>
+                <version>${plugin.compile.version}</version>
                 <configuration>
                     <encoding>${project.build.encoding}</encoding>
                     <source>${maven.compiler.source}</source>
@@ -508,7 +508,6 @@
                         </path>
                     </annotationProcessorPaths>
                 </configuration>
-                <version>${plugin.compile.version}</version>
             </plugin>
             <plugin>
                 <artifactId>maven-surefire-plugin</artifactId>