You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2022/09/22 06:41:44 UTC

[GitHub] [inlong] lucaspeng12138 opened a new pull request, #5989: [INLONG-5043][Manager] Add Apache Doris load node management

lucaspeng12138 opened a new pull request, #5989:
URL: https://github.com/apache/inlong/pull/5989

   ### Prepare a Pull Request
   - Fixes #5043 
   
   ### Motivation
   
   To surport the ability to Doris data integration, we need to add Apache Doris Load Node for management
   ### Design
   The design mainly follows the document [Manager Plugin](https://inlong.apache.org/zh-CN/docs/design_and_concept/how_to_extend_data_node_for_manager)
   
   1. Add corresponding SinkType enumeration in enumeration type org.apache.inlong.manager.common.Enums.
   2. In org.apache.inlong.manager.common.Pojo.Sink,create folder path,create the corresponding entity class.
   3. In the org.Apache.Inlong.Manager.Service.Sink path, created under the corresponding tools
   4. Support data source to LoadNode conversion function, reference code org.Apache. Inlong.Manager.Service.Sort.Util.LoadNodeUtils
   
   ### Implementation
   
   1. Add DORIS in enumeration type org.apache.inlong.manager.common.enums.SinkType
   2. Create folder "Doris" in org.apache.inlong.manager.common.pojo.sink, and create corresponding entity class:
   
   - DorisSink
   - DorisSinkDTO
   - DorisSinkRequest
   - DorisColumnInfo
   - DorisTableInfo
   
   3. Create folder "doris" in org.apache.inlong.manager.service.sink and implement the class:
   
   - DorisSinkOperator
   
   4. Add createLoadNode function in org.apache.inlong.manager.service.sort.util.LoadNodeUtils, it is like as follows:
   `public static DorisLoadNode createLoadNode(DorisSink dorisSink, List<FieldInfo> fieldInfos,
               List<FieldRelation> fieldRelations, Map<String, String> properties){
                    \\TODO 
         }`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] dockerzhang commented on pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on PR #5989:
URL: https://github.com/apache/inlong/pull/5989#issuecomment-1285060746

   duplicated with https://github.com/apache/inlong/pull/6233, close it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980674864


##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";

Review Comment:
   Just change them to `private final Xxx`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980672198


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/doris/DorisSinkDTO.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.pojo.sink.doris;
+
+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.List;
+import java.util.Map;
+
+/**
+ * Doris sink info.
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class DorisSinkDTO {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(DorisSinkDTO.class);
+
+    @ApiModelProperty("Doris 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 Doris")
+    private Map<String, Object> properties;
+
+    /**
+     * Get the dto instance from the request
+     */
+    public static DorisSinkDTO getFromRequest(DorisSinkRequest request) {
+        return DorisSinkDTO.builder()
+                .jdbcUrl(request.getJdbcUrl())
+                .tableName(request.getTableName())
+                .username(request.getUsername())
+                .password(request.getPassword())
+                .primaryKey(request.getPrimaryKey())
+                .properties(request.getProperties())
+                .build();
+    }
+
+    /**
+     * Get Doris sink info from JSON string
+     */
+    public static DorisSinkDTO getFromJson(@NotNull String extParams) {
+        try {
+            ObjectMapper objectMapper = new ObjectMapper().configure(

Review Comment:
   Suggest extracting the `objectMapper` to a static final field.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] lucaspeng12138 commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
lucaspeng12138 commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r979784562


##########
inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/consts/SinkType.java:
##########
@@ -37,4 +37,6 @@ public class SinkType {
     public static final String TDSQLPOSTGRESQL = "TDSQLPOSTGRESQL";
     public static final String DLCICEBERG = "DLCICEBERG";
 

Review Comment:
   resolved



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sort/util/LoadNodeUtils.java:
##########
@@ -493,6 +497,28 @@ public static DLCIcebergLoadNode createLoadNode(DLCIcebergSink dlcIcebergSink, L
         );
     }
 
+    /**
+     * Create load node of Doris.
+     */
+    public static DorisLoadNode createLoadNode(DorisSink dorisSink, List<FieldInfo> fieldInfos,
+                                               List<FieldRelation> fieldRelations, Map<String, String> properties) {

Review Comment:
   resolved



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] lucaspeng12138 commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
lucaspeng12138 commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980686815


##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";
+    private static final String globalStreamId = "stream1_doris";
+    private static final String globalOperator = "admin";
+    private static final String dorisJdbcUrl = "jdbc:mysql://127.0.0.1:6603/default";
+    private static final String dorisUsername = "doris_user";
+    private static final String dorisPassword = "doris_passwd";
+    private static final String dorisTableName = "doris_tbl";
+    @Autowired
+    private StreamSinkService sinkService;
+    @Autowired
+    private InlongStreamServiceTest streamServiceTest;
+
+    /**
+     * Save sink info.
+     */
+    public Integer saveSink(String sinkName) {
+        streamServiceTest.saveInlongStream(globalGroupId, globalStreamId, globalOperator);
+        DorisSinkRequest sinkInfo = new DorisSinkRequest();

Review Comment:
   resolved



##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";
+    private static final String globalStreamId = "stream1_doris";
+    private static final String globalOperator = "admin";
+    private static final String dorisJdbcUrl = "jdbc:mysql://127.0.0.1:6603/default";
+    private static final String dorisUsername = "doris_user";
+    private static final String dorisPassword = "doris_passwd";
+    private static final String dorisTableName = "doris_tbl";
+    @Autowired
+    private StreamSinkService sinkService;
+    @Autowired
+    private InlongStreamServiceTest streamServiceTest;
+
+    /**
+     * Save sink info.
+     */
+    public Integer saveSink(String sinkName) {
+        streamServiceTest.saveInlongStream(globalGroupId, globalStreamId, globalOperator);
+        DorisSinkRequest sinkInfo = new DorisSinkRequest();
+        sinkInfo.setInlongGroupId(globalGroupId);
+        sinkInfo.setInlongStreamId(globalStreamId);
+        sinkInfo.setSinkName(sinkName);
+        sinkInfo.setSinkType(SinkType.DORIS);
+        sinkInfo.setJdbcUrl(dorisJdbcUrl);
+        sinkInfo.setUsername(dorisUsername);
+        sinkInfo.setPassword(dorisPassword);
+        sinkInfo.setTableName(dorisTableName);
+        sinkInfo.setEnableCreateResource(InlongConstants.DISABLE_CREATE_RESOURCE);
+        sinkInfo.setId((int) (Math.random() * 100000 + 1));

Review Comment:
   resolved



##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";

Review Comment:
   resolved



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980677988


##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";
+    private static final String globalStreamId = "stream1_doris";
+    private static final String globalOperator = "admin";
+    private static final String dorisJdbcUrl = "jdbc:mysql://127.0.0.1:6603/default";
+    private static final String dorisUsername = "doris_user";
+    private static final String dorisPassword = "doris_passwd";
+    private static final String dorisTableName = "doris_tbl";
+    @Autowired
+    private StreamSinkService sinkService;
+    @Autowired
+    private InlongStreamServiceTest streamServiceTest;
+
+    /**
+     * Save sink info.
+     */
+    public Integer saveSink(String sinkName) {
+        streamServiceTest.saveInlongStream(globalGroupId, globalStreamId, globalOperator);
+        DorisSinkRequest sinkInfo = new DorisSinkRequest();
+        sinkInfo.setInlongGroupId(globalGroupId);
+        sinkInfo.setInlongStreamId(globalStreamId);
+        sinkInfo.setSinkName(sinkName);
+        sinkInfo.setSinkType(SinkType.DORIS);
+        sinkInfo.setJdbcUrl(dorisJdbcUrl);
+        sinkInfo.setUsername(dorisUsername);
+        sinkInfo.setPassword(dorisPassword);
+        sinkInfo.setTableName(dorisTableName);
+        sinkInfo.setEnableCreateResource(InlongConstants.DISABLE_CREATE_RESOURCE);
+        sinkInfo.setId((int) (Math.random() * 100000 + 1));

Review Comment:
   In the case of `save`, no need to set the id field.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] dockerzhang closed pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
dockerzhang closed pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management
URL: https://github.com/apache/inlong/pull/5989


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r977269139


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/doris/DorisSink.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.pojo.sink.doris;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import lombok.experimental.SuperBuilder;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+
+/**
+ * Doris sink info.
+ */
+@Data
+@SuperBuilder
+@AllArgsConstructor
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@ApiModel(value = "Doris sink info")
+@JsonTypeDefine(value = SinkType.DORIS)
+public class DorisSink extends StreamSink {
+
+    @ApiModelProperty("Doris JDBC URL, such as jdbc:mysql://host:port/database")

Review Comment:
   `mysql`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] lucaspeng12138 commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
lucaspeng12138 commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r979786144


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/doris/DorisSink.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.pojo.sink.doris;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import lombok.experimental.SuperBuilder;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+
+/**
+ * Doris sink info.
+ */
+@Data
+@SuperBuilder
+@AllArgsConstructor
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@ApiModel(value = "Doris sink info")
+@JsonTypeDefine(value = SinkType.DORIS)
+public class DorisSink extends StreamSink {
+
+    @ApiModelProperty("Doris JDBC URL, such as jdbc:mysql://host:port/database")

Review Comment:
   yes, according to https://doris.apache.org/zh-CN/docs/dev/install/install-deploy



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] EMsnap commented on pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
EMsnap commented on PR #5989:
URL: https://github.com/apache/inlong/pull/5989#issuecomment-1259423835

   there may be a few changes on sink parameter since we are optimizing the doris connector, so I suggest hold for now


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r977272185


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sort/util/LoadNodeUtils.java:
##########
@@ -493,6 +497,28 @@ public static DLCIcebergLoadNode createLoadNode(DLCIcebergSink dlcIcebergSink, L
         );
     }
 
+    /**
+     * Create load node of Doris.
+     */
+    public static DorisLoadNode createLoadNode(DorisSink dorisSink, List<FieldInfo> fieldInfos,
+                                               List<FieldRelation> fieldRelations, Map<String, String> properties) {

Review Comment:
   No need to indent to the above params.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r977369788


##########
inlong-manager/manager-common/src/main/java/org/apache/inlong/manager/common/consts/SinkType.java:
##########
@@ -37,4 +37,6 @@ public class SinkType {
     public static final String TDSQLPOSTGRESQL = "TDSQLPOSTGRESQL";
     public static final String DLCICEBERG = "DLCICEBERG";
 

Review Comment:
   No need a blank line here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980672617


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/doris/DorisTableInfo.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.pojo.sink.doris;
+
+import lombok.Data;
+
+import java.util.List;
+
+/**
+ * Doris table info.
+ */
+@Data
+public class DorisTableInfo {
+
+    private String tableName;
+
+    private String comment;
+
+    private String primaryKey;
+
+    private String userName;

Review Comment:
   Suggest changing to `username`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] lucaspeng12138 commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
lucaspeng12138 commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980686894


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/doris/DorisTableInfo.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.pojo.sink.doris;
+
+import lombok.Data;
+
+import java.util.List;
+
+/**
+ * Doris table info.
+ */
+@Data
+public class DorisTableInfo {
+
+    private String tableName;
+
+    private String comment;
+
+    private String primaryKey;
+
+    private String userName;

Review Comment:
   resolved



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/doris/DorisSinkDTO.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.pojo.sink.doris;
+
+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.List;
+import java.util.Map;
+
+/**
+ * Doris sink info.
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class DorisSinkDTO {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(DorisSinkDTO.class);
+
+    @ApiModelProperty("Doris 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 Doris")
+    private Map<String, Object> properties;
+
+    /**
+     * Get the dto instance from the request
+     */
+    public static DorisSinkDTO getFromRequest(DorisSinkRequest request) {
+        return DorisSinkDTO.builder()
+                .jdbcUrl(request.getJdbcUrl())
+                .tableName(request.getTableName())
+                .username(request.getUsername())
+                .password(request.getPassword())
+                .primaryKey(request.getPrimaryKey())
+                .properties(request.getProperties())
+                .build();
+    }
+
+    /**
+     * Get Doris sink info from JSON string
+     */
+    public static DorisSinkDTO getFromJson(@NotNull String extParams) {
+        try {
+            ObjectMapper objectMapper = new ObjectMapper().configure(

Review Comment:
   resolved



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980677988


##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";
+    private static final String globalStreamId = "stream1_doris";
+    private static final String globalOperator = "admin";
+    private static final String dorisJdbcUrl = "jdbc:mysql://127.0.0.1:6603/default";
+    private static final String dorisUsername = "doris_user";
+    private static final String dorisPassword = "doris_passwd";
+    private static final String dorisTableName = "doris_tbl";
+    @Autowired
+    private StreamSinkService sinkService;
+    @Autowired
+    private InlongStreamServiceTest streamServiceTest;
+
+    /**
+     * Save sink info.
+     */
+    public Integer saveSink(String sinkName) {
+        streamServiceTest.saveInlongStream(globalGroupId, globalStreamId, globalOperator);
+        DorisSinkRequest sinkInfo = new DorisSinkRequest();
+        sinkInfo.setInlongGroupId(globalGroupId);
+        sinkInfo.setInlongStreamId(globalStreamId);
+        sinkInfo.setSinkName(sinkName);
+        sinkInfo.setSinkType(SinkType.DORIS);
+        sinkInfo.setJdbcUrl(dorisJdbcUrl);
+        sinkInfo.setUsername(dorisUsername);
+        sinkInfo.setPassword(dorisPassword);
+        sinkInfo.setTableName(dorisTableName);
+        sinkInfo.setEnableCreateResource(InlongConstants.DISABLE_CREATE_RESOURCE);
+        sinkInfo.setId((int) (Math.random() * 100000 + 1));

Review Comment:
   Suggest using the `Random` class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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


[GitHub] [inlong] healchow commented on a diff in pull request #5989: [INLONG-5043][Manager] Add Apache Doris load node management

Posted by GitBox <gi...@apache.org>.
healchow commented on code in PR #5989:
URL: https://github.com/apache/inlong/pull/5989#discussion_r980678346


##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/sink/DorisSinkServiceTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.consts.SinkType;
+import org.apache.inlong.manager.pojo.sink.SinkRequest;
+import org.apache.inlong.manager.pojo.sink.StreamSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSink;
+import org.apache.inlong.manager.pojo.sink.doris.DorisSinkRequest;
+import org.apache.inlong.manager.service.ServiceBaseTest;
+import org.apache.inlong.manager.service.core.impl.InlongStreamServiceTest;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+/**
+ * Doris sink service test
+ */
+public class DorisSinkServiceTest extends ServiceBaseTest {
+
+    // Partial test data
+    private static final String globalGroupId = "b_group1";
+    private static final String globalStreamId = "stream1_doris";
+    private static final String globalOperator = "admin";
+    private static final String dorisJdbcUrl = "jdbc:mysql://127.0.0.1:6603/default";
+    private static final String dorisUsername = "doris_user";
+    private static final String dorisPassword = "doris_passwd";
+    private static final String dorisTableName = "doris_tbl";
+    @Autowired
+    private StreamSinkService sinkService;
+    @Autowired
+    private InlongStreamServiceTest streamServiceTest;
+
+    /**
+     * Save sink info.
+     */
+    public Integer saveSink(String sinkName) {
+        streamServiceTest.saveInlongStream(globalGroupId, globalStreamId, globalOperator);
+        DorisSinkRequest sinkInfo = new DorisSinkRequest();

Review Comment:
   Suggest changing to `sinkRequest`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

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