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

[incubator-inlong] branch master updated: [INLONG-4414][Manager] Support MySQL sink (#4415)

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

healchow 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 eed23dbb7 [INLONG-4414][Manager] Support MySQL sink (#4415)
eed23dbb7 is described below

commit eed23dbb72bb83961f2e1b597bfaacfcaff41922
Author: jiancheng Lv <63...@qq.com>
AuthorDate: Wed Jun 8 13:00:13 2022 +0800

    [INLONG-4414][Manager] Support MySQL sink (#4415)
    
    * Support MySQL sink
    
    * Change the log info
    
    Co-authored-by: healchow <he...@gmail.com>
---
 .../inlong/manager/common/enums/SinkType.java      |   3 +
 .../manager/common/pojo/sink/mysql/MySQLSink.java  |  64 ++++++
 .../common/pojo/sink/mysql/MySQLSinkDTO.java       |  92 +++++++++
 .../pojo/sink/mysql/MySQLSinkListResponse.java     |  57 +++++
 .../common/pojo/sink/mysql/MySQLSinkRequest.java   |  53 +++++
 .../service/sink/mysql/MySQLSinkOperation.java     | 229 +++++++++++++++++++++
 .../manager/service/sort/util/LoadNodeUtils.java   |  34 +++
 .../core/sink/MysqlStreamSinkServiceTest.java      | 113 ++++++++++
 8 files changed, 645 insertions(+)

diff --git a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/enums/SinkType.java b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/enums/SinkType.java
index a798abf11..c387bf8e4 100644
--- a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/enums/SinkType.java
+++ b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/enums/SinkType.java
@@ -31,6 +31,8 @@ public enum SinkType {
     SQLSERVER,
     HDFS,
     GREENPLUM,
+    MYSQL,
+
 
     ;
 
@@ -44,6 +46,7 @@ public enum SinkType {
     public static final String SINK_SQLSERVER = "SQLSERVER";
     public static final String SINK_HDFS = "HDFS";
     public static final String SINK_GREENPLUM = "GREENPLUM";
+    public static final String SINK_MYSQL = "MYSQL";
 
     /**
      * Get the SinkType enum via the given sinkType string
diff --git a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSink.java b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSink.java
new file mode 100644
index 000000000..2b0181251
--- /dev/null
+++ b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSink.java
@@ -0,0 +1,64 @@
+/*
+ * 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.common.pojo.sink.mysql;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import org.apache.inlong.manager.common.enums.SinkType;
+import org.apache.inlong.manager.common.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.common.pojo.sink.StreamSink;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+
+/**
+ * MySQL sink info
+ */
+@Data
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@ApiModel(value = "MySQL sink info")
+@JsonTypeDefine(value = SinkType.SINK_MYSQL)
+public class MySQLSink extends StreamSink {
+
+    @ApiModelProperty("MySQL JDBC URL, such as jdbc:mysql://host:port/database")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username for JDBC URL")
+    private String username;
+
+    @ApiModelProperty("User password")
+    private String password;
+
+    @ApiModelProperty("Target table name")
+    private String tableName;
+
+    @ApiModelProperty("Primary key")
+    private String primaryKey;
+
+    public MySQLSink() {
+        this.setSinkType(SinkType.SINK_MYSQL);
+    }
+
+    @Override
+    public SinkRequest genSinkRequest() {
+        return CommonBeanUtils.copyProperties(this, MySQLSinkRequest::new);
+    }
+}
diff --git a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkDTO.java b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkDTO.java
new file mode 100644
index 000000000..c21ce464c
--- /dev/null
+++ b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkDTO.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.common.pojo.sink.mysql;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.validation.constraints.NotNull;
+import java.util.Map;
+
+/**
+ * MySQL sink info
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class MySQLSinkDTO {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final Logger LOGGER = LoggerFactory.getLogger(MySQLSinkDTO.class);
+
+    @ApiModelProperty("MySQL JDBC URL, such as jdbc:mysql://host:port/database")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username for JDBC URL")
+    private String username;
+
+    @ApiModelProperty("User password")
+    private String password;
+
+    @ApiModelProperty("Target table name")
+    private String tableName;
+
+    @ApiModelProperty("Primary key")
+    private String primaryKey;
+
+    @ApiModelProperty("Properties for MySQL")
+    private Map<String, Object> properties;
+
+    /**
+     * Get the dto instance from the request
+     */
+    public static MySQLSinkDTO getFromRequest(MySQLSinkRequest request) {
+        return MySQLSinkDTO.builder()
+                .jdbcUrl(request.getJdbcUrl())
+                .username(request.getUsername())
+                .password(request.getPassword())
+                .primaryKey(request.getPrimaryKey())
+                .tableName(request.getTableName())
+                .properties(request.getProperties())
+                .build();
+    }
+
+    /**
+     * Get MySQL sink info from JSON string
+     */
+    public static MySQLSinkDTO getFromJson(@NotNull String extParams) {
+        try {
+            OBJECT_MAPPER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+            return OBJECT_MAPPER.readValue(extParams, MySQLSinkDTO.class);
+        } catch (Exception e) {
+            LOGGER.error("fetch mysql sink info failed from json params: " + extParams, e);
+            throw new BusinessException(ErrorCodeEnum.SINK_INFO_INCORRECT.getMessage());
+        }
+    }
+
+}
diff --git a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkListResponse.java b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkListResponse.java
new file mode 100644
index 000000000..b5b0a4e9e
--- /dev/null
+++ b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkListResponse.java
@@ -0,0 +1,57 @@
+/*
+ * 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.common.pojo.sink.mysql;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+import lombok.experimental.SuperBuilder;
+import org.apache.inlong.manager.common.enums.SinkType;
+import org.apache.inlong.manager.common.pojo.sink.SinkListResponse;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+
+/**
+ * Response of MySQL sink list
+ */
+@Data
+@SuperBuilder
+@NoArgsConstructor
+@AllArgsConstructor
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeDefine(SinkType.SINK_MYSQL)
+@ApiModel("Response of MySQL sink paging list")
+public class MySQLSinkListResponse extends SinkListResponse {
+
+    @ApiModelProperty("MySQL JDBC URL, such as jdbc:mysql://host:port/database")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username for JDBC URL")
+    private String username;
+
+    @ApiModelProperty("User password")
+    private String password;
+
+    @ApiModelProperty("Target table name")
+    private String tableName;
+
+    @ApiModelProperty("Primary key")
+    private String primaryKey;
+}
diff --git a/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkRequest.java b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkRequest.java
new file mode 100644
index 000000000..2f8b967e4
--- /dev/null
+++ b/inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/pojo/sink/mysql/MySQLSinkRequest.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.inlong.manager.common.pojo.sink.mysql;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import org.apache.inlong.manager.common.enums.SinkType;
+import org.apache.inlong.manager.common.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+
+/**
+ * Request of the MySQL sink info
+ */
+@Data
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@ApiModel(value = "Request of the MySQL sink info")
+@JsonTypeDefine(value = SinkType.SINK_MYSQL)
+public class MySQLSinkRequest extends SinkRequest {
+
+    @ApiModelProperty("MySQL JDBC URL, such as jdbc:mysql://host:port/database")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username for JDBC URL")
+    private String username;
+
+    @ApiModelProperty("User password")
+    private String password;
+
+    @ApiModelProperty("Target table name")
+    private String tableName;
+
+    @ApiModelProperty("Primary key")
+    private String primaryKey;
+}
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/mysql/MySQLSinkOperation.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/mysql/MySQLSinkOperation.java
new file mode 100644
index 000000000..2ba28953f
--- /dev/null
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/mysql/MySQLSinkOperation.java
@@ -0,0 +1,229 @@
+/*
+ * 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.service.sink.mysql;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.github.pagehelper.Page;
+import com.github.pagehelper.PageInfo;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.enums.GlobalConstants;
+import org.apache.inlong.manager.common.enums.SinkStatus;
+import org.apache.inlong.manager.common.enums.SinkType;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.pojo.sink.SinkField;
+import org.apache.inlong.manager.common.pojo.sink.SinkListResponse;
+import org.apache.inlong.manager.common.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.common.pojo.sink.StreamSink;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSink;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSinkDTO;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSinkListResponse;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSinkRequest;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.dao.entity.StreamSinkEntity;
+import org.apache.inlong.manager.dao.entity.StreamSinkFieldEntity;
+import org.apache.inlong.manager.dao.mapper.StreamSinkEntityMapper;
+import org.apache.inlong.manager.dao.mapper.StreamSinkFieldEntityMapper;
+import org.apache.inlong.manager.service.sink.StreamSinkOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import javax.validation.constraints.NotNull;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Supplier;
+
+/**
+ * MySQL sink operation
+ */
+@Service
+public class MySQLSinkOperation implements StreamSinkOperation {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(MySQLSinkOperation.class);
+
+    @Autowired
+    private ObjectMapper objectMapper;
+    @Autowired
+    private StreamSinkEntityMapper sinkMapper;
+    @Autowired
+    private StreamSinkFieldEntityMapper sinkFieldMapper;
+
+    @Override
+    public Boolean accept(SinkType sinkType) {
+        return SinkType.MYSQL.equals(sinkType);
+    }
+
+    @Override
+    public Integer saveOpt(SinkRequest request, String operator) {
+        String sinkType = request.getSinkType();
+        Preconditions.checkTrue(SinkType.SINK_MYSQL.equals(sinkType),
+                ErrorCodeEnum.SINK_TYPE_NOT_SUPPORT.getMessage() + ": " + sinkType);
+
+        MySQLSinkRequest mySQLSinkRequest = (MySQLSinkRequest) request;
+        StreamSinkEntity entity = CommonBeanUtils.copyProperties(mySQLSinkRequest, StreamSinkEntity::new);
+        entity.setStatus(SinkStatus.NEW.getCode());
+        entity.setIsDeleted(GlobalConstants.UN_DELETED);
+        entity.setCreator(operator);
+        entity.setModifier(operator);
+        Date now = new Date();
+        entity.setCreateTime(now);
+        entity.setModifyTime(now);
+
+        // get the ext params
+        MySQLSinkDTO dto = MySQLSinkDTO.getFromRequest(mySQLSinkRequest);
+        try {
+            entity.setExtParams(objectMapper.writeValueAsString(dto));
+        } catch (Exception e) {
+            throw new BusinessException(ErrorCodeEnum.SINK_SAVE_FAILED);
+        }
+        sinkMapper.insert(entity);
+        Integer sinkId = entity.getId();
+        request.setId(sinkId);
+        this.saveFieldOpt(request);
+        return sinkId;
+    }
+
+    @Override
+    public void saveFieldOpt(SinkRequest request) {
+        List<SinkField> fieldList = request.getFieldList();
+        LOGGER.info("begin to save field={}", fieldList);
+        if (CollectionUtils.isEmpty(fieldList)) {
+            return;
+        }
+
+        int size = fieldList.size();
+        List<StreamSinkFieldEntity> entityList = new ArrayList<>(size);
+        String groupId = request.getInlongGroupId();
+        String streamId = request.getInlongStreamId();
+        String sinkType = request.getSinkType();
+        Integer sinkId = request.getId();
+        for (SinkField fieldInfo : fieldList) {
+            StreamSinkFieldEntity fieldEntity = CommonBeanUtils.copyProperties(fieldInfo, StreamSinkFieldEntity::new);
+            if (StringUtils.isEmpty(fieldEntity.getFieldComment())) {
+                fieldEntity.setFieldComment(fieldEntity.getFieldName());
+            }
+            fieldEntity.setInlongGroupId(groupId);
+            fieldEntity.setInlongStreamId(streamId);
+            fieldEntity.setSinkType(sinkType);
+            fieldEntity.setSinkId(sinkId);
+            fieldEntity.setIsDeleted(GlobalConstants.UN_DELETED);
+            entityList.add(fieldEntity);
+        }
+
+        sinkFieldMapper.insertAll(entityList);
+        LOGGER.info("success to save field");
+    }
+
+    @Override
+    public StreamSink getByEntity(@NotNull StreamSinkEntity entity) {
+        Preconditions.checkNotNull(entity, ErrorCodeEnum.SINK_INFO_NOT_FOUND.getMessage());
+        String existType = entity.getSinkType();
+        Preconditions.checkTrue(SinkType.SINK_MYSQL.equals(existType),
+                String.format(ErrorCodeEnum.SINK_TYPE_NOT_SAME.getMessage(), SinkType.SINK_MYSQL, existType));
+        StreamSink response = this.getFromEntity(entity, MySQLSink::new);
+        List<StreamSinkFieldEntity> entities = sinkFieldMapper.selectBySinkId(entity.getId());
+        List<SinkField> infos = CommonBeanUtils.copyListProperties(entities, SinkField::new);
+        response.setFieldList(infos);
+        return response;
+    }
+
+    @Override
+    public <T> T getFromEntity(StreamSinkEntity entity, Supplier<T> target) {
+        T result = target.get();
+        if (entity == null) {
+            return result;
+        }
+        String existType = entity.getSinkType();
+        Preconditions.checkTrue(SinkType.SINK_MYSQL.equals(existType),
+                String.format(ErrorCodeEnum.SINK_TYPE_NOT_SAME.getMessage(), SinkType.SINK_MYSQL, existType));
+
+        MySQLSinkDTO dto = MySQLSinkDTO.getFromJson(entity.getExtParams());
+        CommonBeanUtils.copyProperties(entity, result, true);
+        CommonBeanUtils.copyProperties(dto, result, true);
+
+        return result;
+    }
+
+    @Override
+    public PageInfo<? extends SinkListResponse> getPageInfo(Page<StreamSinkEntity> entityPage) {
+        if (CollectionUtils.isEmpty(entityPage)) {
+            return new PageInfo<>();
+        }
+        return entityPage.toPageInfo(entity -> this.getFromEntity(entity, MySQLSinkListResponse::new));
+    }
+
+    @Override
+    public void updateOpt(SinkRequest request, String operator) {
+        String sinkType = request.getSinkType();
+        Preconditions.checkTrue(SinkType.SINK_MYSQL.equals(sinkType),
+                String.format(ErrorCodeEnum.SINK_TYPE_NOT_SAME.getMessage(), SinkType.SINK_MYSQL, sinkType));
+
+        StreamSinkEntity entity = sinkMapper.selectByPrimaryKey(request.getId());
+        Preconditions.checkNotNull(entity, ErrorCodeEnum.SINK_INFO_NOT_FOUND.getMessage());
+        MySQLSinkRequest mySQLSinkRequest = (MySQLSinkRequest) request;
+        CommonBeanUtils.copyProperties(mySQLSinkRequest, entity, true);
+        try {
+            MySQLSinkDTO dto = MySQLSinkDTO.getFromRequest(mySQLSinkRequest);
+            entity.setExtParams(objectMapper.writeValueAsString(dto));
+        } catch (Exception e) {
+            throw new BusinessException(ErrorCodeEnum.SINK_INFO_INCORRECT.getMessage());
+        }
+
+        entity.setPreviousStatus(entity.getStatus());
+        entity.setStatus(SinkStatus.CONFIG_ING.getCode());
+        entity.setModifier(operator);
+        entity.setModifyTime(new Date());
+        sinkMapper.updateByPrimaryKeySelective(entity);
+
+        boolean onlyAdd = SinkStatus.CONFIG_SUCCESSFUL.getCode().equals(entity.getPreviousStatus());
+        this.updateFieldOpt(onlyAdd, mySQLSinkRequest);
+
+        LOGGER.info("success to update sink of type={}", sinkType);
+    }
+
+    @Override
+    public void updateFieldOpt(Boolean onlyAdd, SinkRequest request) {
+        Integer sinkId = request.getId();
+        List<SinkField> fieldRequestList = request.getFieldList();
+        if (CollectionUtils.isEmpty(fieldRequestList)) {
+            return;
+        }
+        if (onlyAdd) {
+            List<StreamSinkFieldEntity> existsFieldList = sinkFieldMapper.selectBySinkId(sinkId);
+            if (existsFieldList.size() > fieldRequestList.size()) {
+                throw new BusinessException(ErrorCodeEnum.SINK_FIELD_UPDATE_NOT_ALLOWED);
+            }
+            for (int i = 0; i < existsFieldList.size(); i++) {
+                if (!existsFieldList.get(i).getFieldName().equals(fieldRequestList.get(i).getFieldName())) {
+                    throw new BusinessException(ErrorCodeEnum.SINK_FIELD_UPDATE_NOT_ALLOWED);
+                }
+            }
+        }
+        // First physically delete the existing fields
+        sinkFieldMapper.deleteAll(sinkId);
+        // Then batch save the sink fields
+        this.saveFieldOpt(request);
+        LOGGER.info("success to update field");
+    }
+
+}
diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/LoadNodeUtils.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/LoadNodeUtils.java
index 19bfb3b76..ddc972983 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/LoadNodeUtils.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/LoadNodeUtils.java
@@ -35,6 +35,7 @@ import org.apache.inlong.manager.common.pojo.sink.hive.HivePartitionField;
 import org.apache.inlong.manager.common.pojo.sink.hive.HiveSink;
 import org.apache.inlong.manager.common.pojo.sink.iceberg.IcebergSink;
 import org.apache.inlong.manager.common.pojo.sink.kafka.KafkaSink;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSink;
 import org.apache.inlong.manager.common.pojo.sink.postgres.PostgresSink;
 import org.apache.inlong.manager.common.pojo.sink.sqlserver.SqlServerSink;
 import org.apache.inlong.sort.protocol.FieldInfo;
@@ -54,6 +55,7 @@ import org.apache.inlong.sort.protocol.node.load.HbaseLoadNode;
 import org.apache.inlong.sort.protocol.node.load.HiveLoadNode;
 import org.apache.inlong.sort.protocol.node.load.IcebergLoadNode;
 import org.apache.inlong.sort.protocol.node.load.KafkaLoadNode;
+import org.apache.inlong.sort.protocol.node.load.MySqlLoadNode;
 import org.apache.inlong.sort.protocol.node.load.PostgresLoadNode;
 import org.apache.inlong.sort.protocol.node.load.SqlServerLoadNode;
 import org.apache.inlong.sort.protocol.transformation.FieldRelation;
@@ -104,6 +106,8 @@ public class LoadNodeUtils {
                 return createLoadNode((HdfsSink) streamSink);
             case GREENPLUM:
                 return createLoadNode((GreenplumSink) streamSink);
+            case MYSQL:
+                return createLoadNode((MySQLSink) streamSink);
             default:
                 throw new BusinessException(String.format("Unsupported sinkType=%s to create load node", sinkType));
         }
@@ -456,6 +460,36 @@ public class LoadNodeUtils {
                 greenplumSink.getPrimaryKey());
     }
 
+    /**
+     * Create load node of MySQL.
+     */
+    public static MySqlLoadNode createLoadNode(MySQLSink mysqlSink) {
+        String id = mysqlSink.getSinkName();
+        String name = mysqlSink.getSinkName();
+        List<SinkField> fieldList = mysqlSink.getFieldList();
+        List<FieldInfo> fields = fieldList.stream()
+                .map(sinkField -> FieldInfoUtils.parseSinkFieldInfo(sinkField, name))
+                .collect(Collectors.toList());
+        List<FieldRelation> fieldRelations = parseSinkFields(fieldList, name);
+        Map<String, String> properties = mysqlSink.getProperties().entrySet().stream()
+                .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()));
+
+        return new MySqlLoadNode(
+                id,
+                name,
+                fields,
+                fieldRelations,
+                Lists.newArrayList(),
+                null,
+                null,
+                properties,
+                mysqlSink.getJdbcUrl(),
+                mysqlSink.getUsername(),
+                mysqlSink.getPassword(),
+                mysqlSink.getTableName(),
+                mysqlSink.getPrimaryKey());
+    }
+
     /**
      * Parse information field of data sink.
      */
diff --git a/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/sink/MysqlStreamSinkServiceTest.java b/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/sink/MysqlStreamSinkServiceTest.java
new file mode 100644
index 000000000..cee696395
--- /dev/null
+++ b/inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/sink/MysqlStreamSinkServiceTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.service.core.sink;
+
+import org.apache.inlong.manager.common.enums.GlobalConstants;
+import org.apache.inlong.manager.common.enums.SinkType;
+import org.apache.inlong.manager.common.pojo.sink.SinkField;
+import org.apache.inlong.manager.common.pojo.sink.StreamSink;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSink;
+import org.apache.inlong.manager.common.pojo.sink.mysql.MySQLSinkRequest;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.apache.inlong.manager.service.sink.StreamSinkService;
+import org.junit.Assert;
+import org.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Stream sink service test
+ */
+public class MysqlStreamSinkServiceTest extends ServiceBaseTest {
+
+    private static final String globalGroupId = "b_group1_mysql";
+    private static final String globalStreamId = "stream1_mysql";
+    private static final String globalOperator = "admin";
+    private static final String fieldName = "mysql_field";
+    private static final String fieldType = "mysql_type";
+    private static final Integer fieldId = 1;
+
+    @Autowired
+    private StreamSinkService sinkService;
+    @Autowired
+    private InlongStreamServiceTest streamServiceTest;
+
+    /**
+     * Save sink info.
+     */
+    public Integer saveSink(String sinkName) {
+        streamServiceTest.saveInlongStream(globalGroupId, globalStreamId, globalOperator);
+        MySQLSinkRequest sinkInfo = new MySQLSinkRequest();
+        sinkInfo.setInlongGroupId(globalGroupId);
+        sinkInfo.setInlongStreamId(globalStreamId);
+        sinkInfo.setSinkType(SinkType.SINK_MYSQL);
+
+        sinkInfo.setJdbcUrl("jdbc:mysql://localhost:5432/database");
+        sinkInfo.setUsername("mysql");
+        sinkInfo.setPassword("inlong");
+        sinkInfo.setTableName("user");
+        sinkInfo.setPrimaryKey("name,age");
+
+        sinkInfo.setSinkName(sinkName);
+        sinkInfo.setEnableCreateResource(GlobalConstants.DISABLE_CREATE_RESOURCE);
+        SinkField sinkField = new SinkField();
+        sinkField.setFieldName(fieldName);
+        sinkField.setFieldType(fieldType);
+        sinkField.setId(fieldId);
+        List<SinkField> sinkFieldList = new ArrayList<>();
+        sinkFieldList.add(sinkField);
+        sinkInfo.setFieldList(sinkFieldList);
+        return sinkService.save(sinkInfo, globalOperator);
+    }
+
+    /**
+     * Delete MySQL sink info by sink id.
+     */
+    public void deleteMySQLSink(Integer mysqlSinkId) {
+        boolean result = sinkService.delete(mysqlSinkId, globalOperator);
+        Assert.assertTrue(result);
+    }
+
+    @Test
+    public void testListByIdentifier() {
+        Integer mysqlSinkId = this.saveSink("mysql_default1");
+        StreamSink sink = sinkService.get(mysqlSinkId);
+        Assert.assertEquals(globalGroupId, sink.getInlongGroupId());
+        deleteMySQLSink(mysqlSinkId);
+    }
+
+    @Test
+    public void testGetAndUpdate() {
+        Integer mysqlSinkId = this.saveSink("mysql_default2");
+        StreamSink response = sinkService.get(mysqlSinkId);
+        Assert.assertEquals(globalGroupId, response.getInlongGroupId());
+
+        MySQLSink mysqlSink = (MySQLSink) response;
+        mysqlSink.setEnableCreateResource(GlobalConstants.ENABLE_CREATE_RESOURCE);
+        MySQLSinkRequest request = CommonBeanUtils.copyProperties(mysqlSink,
+                MySQLSinkRequest::new);
+        boolean result = sinkService.update(request, globalOperator);
+        Assert.assertTrue(result);
+        deleteMySQLSink(mysqlSinkId);
+    }
+
+}