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/08/05 13:27:38 UTC

[GitHub] [inlong] fuweng11 opened a new pull request, #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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

   ### Prepare a Pull Request
   - Fixes #5380 
   
   ### Motivation
   
   Modify the save function of datanode. datanode can save node data according to different node types
   
   
   ### Modifications
   
   1、Modify the save function of datanode. datanode can save node data according to different node types
   
   2、Add save modification function of hive node
   
   ### Verifying this change
   
   *(Please pick either of the following options)*
   
   - [ ] This change is a trivial rework/code cleanup without any test coverage.
   
   - [X] This change is already covered by existing tests, such as:
   org/apache/inlong/manager/service/core/impl/DataNodeServiceTest.java
   
   - [ ] This change added tests and can be verified as follows:
   
   ### Documentation
   
     - Does this pull request introduce a new feature? (no)
   
   


-- 
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 #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-web/src/test/java/org/apache/inlong/manager/web/controller/DataNodeControllerTest.java:
##########
@@ -36,23 +37,26 @@ class DataNodeControllerTest extends WebBaseTest {
     @Resource
     DataNodeEntityMapper dataNodeEntityMapper;
 
-    DataNodeRequest getDataNodeRequest() {
-        return DataNodeRequest.builder()
-                .name("hiveNode1")
-                .type("HIVE")
-                .url("127.0.0.1:8080")
-                .username("admin")
-                .token("123")
-                .inCharges("admin")
-                .build();
+    HiveDataNodeRequest getHiveDataNodeRequest() {
+        HiveDataNodeRequest hiveDataNodeRequest = new HiveDataNodeRequest();
+        hiveDataNodeRequest.setName("hiveNode1");
+        hiveDataNodeRequest.setType("HIVE");

Review Comment:
   Please use a String constant from DataNodeType.



-- 
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] fuweng11 commented on a diff in pull request #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/DataNodeServiceImpl.java:
##########
@@ -65,34 +70,39 @@ public Integer save(DataNodeRequest request, String operator) {
             LOGGER.error(errMsg);
             throw new BusinessException(errMsg);
         }
-        DataNodeEntity entity = CommonBeanUtils.copyProperties(request, DataNodeEntity::new);
-        entity.setCreator(operator);
-        entity.setModifier(operator);
-        dataNodeMapper.insert(entity);
-
+        // According to the data type, save sink information
+        DataNodeOperator operation = operatorFactory.getInstance(request.getType());
+        int id = operation.saveOpt(request, operator);
         LOGGER.debug("success to save data node={}", request);
-        return entity.getId();
+        return id;
     }
 
     @Override
-    public DataNodeResponse get(Integer id) {
+    public DataNodeInfo get(Integer id) {
         DataNodeEntity entity = dataNodeMapper.selectById(id);
         if (entity == null) {
             LOGGER.error("data node not found by id={}", id);
             throw new BusinessException("data node not found");
         }
-        DataNodeResponse response = CommonBeanUtils.copyProperties(entity, DataNodeResponse::new);
+        String dataNodeType = entity.getType();
+        DataNodeOperator operation = operatorFactory.getInstance(dataNodeType);
+        DataNodeInfo dataNodeInfo = operation.getFromEntity(entity);
+
         LOGGER.debug("success to get data node info by id={}", id);
-        return response;
+        return dataNodeInfo;
     }
 
     @Override
-    public PageInfo<DataNodeResponse> list(DataNodePageRequest request) {
+    public PageInfo<DataNodeInfo> list(DataNodePageRequest request) {
         PageHelper.startPage(request.getPageNum(), request.getPageSize());
         Page<DataNodeEntity> entityPage = (Page<DataNodeEntity>) dataNodeMapper.selectByCondition(request);
-        List<DataNodeResponse> responseList = CommonBeanUtils.copyListProperties(entityPage, DataNodeResponse::new);
-        PageInfo<DataNodeResponse> page = new PageInfo<>(responseList);
-        page.setTotal(entityPage.getTotal());
+        List<DataNodeInfo> list = entityPage.stream()
+                .map(entity -> {
+                    DataNodeOperator instance = operatorFactory.getInstance(entity.getType());
+                    return instance.getFromEntity(entity);
+                }).collect(Collectors.toList());
+        PageInfo<DataNodeInfo> page = new PageInfo<>(list);
+        page.setTotal(list.size());

Review Comment:
   The size of the` list` is equal to the size of the `entitypage`. The `list` is converted from the `entitypage`.



-- 
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 #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/DataNodeInfo.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.node;
+
+import com.fasterxml.jackson.annotation.JsonFormat;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+/**
+ * Data node info
+ */
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Data node info")
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, visible = true, property = "type")
+public abstract class DataNodeInfo {
+
+    @ApiModelProperty(value = "Primary key")
+    private Integer id;
+
+    @ApiModelProperty(value = "Cluster name")
+    private String name;
+
+    @ApiModelProperty(value = "Cluster type, including TUBE, PULSAR, DATA_PROXY, etc.")

Review Comment:
   Please change to `TUBEMQ` and `DATAPROXY`, thanks.



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

To unsubscribe, e-mail: commits-unsubscribe@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 #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/hive/HiveDataNodeDTO.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.node.hive;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.enums.DataSeparator;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.enums.FileFormat;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.AESUtils;
+import org.apache.inlong.manager.pojo.sink.hive.HivePartitionField;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.NotNull;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * hive data node info
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Hive data node info")
+public class HiveDataNodeDTO {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); // thread safe
+
+    @ApiModelProperty("Hive JDBC URL, such as jdbc:hive2://${ip}:${port}")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username of the Hive server")
+    private String username;
+
+    @ApiModelProperty("User password of the Hive server")
+    private String password;
+
+    @NotBlank(message = "dbName cannot be blank")
+    @ApiModelProperty("Target database name")
+    private String dbName;
+
+    @NotBlank(message = "tableName cannot be blank")
+    @ApiModelProperty("Target table name")
+    private String tableName;
+
+    @NotBlank(message = "dataPath cannot be blank")
+    @ApiModelProperty("Data path, such as: hdfs://ip:port/user/hive/warehouse/test.db")
+    private String dataPath;
+
+    @ApiModelProperty("Partition interval, support: 1 H, 1 D, 30 I, 10 I")
+    private Integer partitionInterval;
+
+    @ApiModelProperty("Partition field list")
+    private List<HivePartitionField> partitionFieldList;

Review Comment:
   Hive data node does not need partitionInterval, partitionFieldList, etc.
   
   It just saves the common fields for all Hive sinks. 



-- 
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 #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/node/AbstractDataNodeOperator.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.node;
+
+import org.apache.inlong.manager.common.consts.InlongConstants;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.dao.entity.DataNodeEntity;
+import org.apache.inlong.manager.dao.mapper.DataNodeEntityMapper;
+import org.apache.inlong.manager.pojo.node.DataNodeRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.transaction.annotation.Isolation;
+import org.springframework.transaction.annotation.Transactional;
+
+public abstract class AbstractDataNodeOperator implements DataNodeOperator {

Review Comment:
   Please add Java docs for new classes.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/hive/HiveDataNodeDTO.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.node.hive;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.AESUtils;
+
+import javax.validation.constraints.NotNull;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * hive data node info
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Hive data node info")
+public class HiveDataNodeDTO {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); // thread safe
+
+    @ApiModelProperty("Hive JDBC URL, such as jdbc:hive2://${ip}:${port}")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username of the Hive server")
+    private String username;
+
+    @ApiModelProperty("User password of the Hive server")
+    private String password;
+
+    @ApiModelProperty("Version for Hive, such as: 3.2.1")
+    private String hiveVersion;
+
+    @ApiModelProperty("Config directory of Hive on HDFS, needed by sort in light mode, must include hive-site.xml")
+    private String hiveConfDir;
+
+    @ApiModelProperty("Password encrypt version")
+    private Integer encryptVersion;
+
+    /**
+     * Get the dto instance from the request
+     */
+    public static HiveDataNodeDTO getFromRequest(HiveDataNodeRequest request) throws Exception {
+
+        Integer encryptVersion = AESUtils.getCurrentVersion(null);
+        String passwd = null;
+        if (StringUtils.isNotEmpty(request.getToken())) {
+            passwd = AESUtils.encryptToString(request.getToken().getBytes(StandardCharsets.UTF_8),
+                    encryptVersion);
+        }
+        return HiveDataNodeDTO.builder()
+                .jdbcUrl(request.getJdbcUrl())
+                .username(request.getUsername())
+                .password(passwd)
+                .hiveVersion(request.getHiveVersion())
+                .hiveConfDir(request.getHiveConfDir())
+                .encryptVersion(encryptVersion)
+                .build();
+    }
+
+    /**
+     * Get the dto instance from the JSON string.
+     */
+    public static HiveDataNodeDTO getFromJson(@NotNull String extParams) {
+        try {
+            OBJECT_MAPPER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+            return OBJECT_MAPPER.readValue(extParams, HiveDataNodeDTO.class).decryptPassword();
+        } catch (Exception e) {
+            throw new BusinessException(ErrorCodeEnum.GROUP_INFO_INCORRECT.getMessage());

Review Comment:
   Suggest adding an error log here.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/DataNodeInfo.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.node;
+
+import com.fasterxml.jackson.annotation.JsonFormat;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+/**
+ * Data node info
+ */
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Data node info")
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, visible = true, property = "type")
+public abstract class DataNodeInfo {
+
+    @ApiModelProperty(value = "Primary key")
+    private Integer id;
+
+    @ApiModelProperty(value = "Cluster name")
+    private String name;
+
+    @ApiModelProperty(value = "Cluster type, including TUBEMQ, PULSAR, DATAPROXY, etc.")
+    private String type;
+
+    @ApiModelProperty(value = "Cluster url")
+    private String url;
+
+    @ApiModelProperty(value = "Cluster tags, separated by commas")
+    private String clusterTags;
+
+    @ApiModelProperty(value = "Extension tag")
+    private String extTag;
+
+    @ApiModelProperty(value = "Cluster token")

Review Comment:
   Why `Cluster token`?



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/node/hive/HiveDataNodeOperator.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.manager.service.node.hive;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.consts.DataNodeType;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.dao.entity.DataNodeEntity;
+import org.apache.inlong.manager.pojo.node.DataNodeInfo;
+import org.apache.inlong.manager.pojo.node.DataNodeRequest;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeDTO;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeInfo;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeRequest;
+import org.apache.inlong.manager.service.node.AbstractDataNodeOperator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+@Service
+public class HiveDataNodeOperator extends AbstractDataNodeOperator {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveDataNodeOperator.class);
+
+    @Autowired
+    private ObjectMapper objectMapper;
+
+    @Override
+    public Boolean accept(String dataNodeType) {
+        return getDataNodeType().equals(dataNodeType);
+    }
+
+    @Override
+    public String getDataNodeType() {
+        return DataNodeType.HIVE;
+    }
+
+    @Override
+    public DataNodeInfo getFromEntity(DataNodeEntity entity) {
+        if (entity == null) {
+            throw new BusinessException(ErrorCodeEnum.DATA_NODE_NOT_FOUND);
+        }
+
+        HiveDataNodeInfo hiveDataNodeInfo = new HiveDataNodeInfo();
+        CommonBeanUtils.copyProperties(entity, hiveDataNodeInfo);
+        if (StringUtils.isNotBlank(entity.getExtParams())) {
+            HiveDataNodeDTO dto = HiveDataNodeDTO.getFromJson(entity.getExtParams());
+            CommonBeanUtils.copyProperties(dto, hiveDataNodeInfo);
+        }
+
+        LOGGER.info("success to get data node info from entity");
+        return hiveDataNodeInfo;
+    }
+
+    @Override
+    protected void setTargetEntity(DataNodeRequest request, DataNodeEntity targetEntity) {
+        HiveDataNodeRequest hiveDataNodeRequest = (HiveDataNodeRequest) request;
+        CommonBeanUtils.copyProperties(hiveDataNodeRequest, targetEntity, true);
+        try {
+            HiveDataNodeDTO dto = HiveDataNodeDTO.getFromRequest(hiveDataNodeRequest);
+            targetEntity.setExtParams(objectMapper.writeValueAsString(dto));
+            LOGGER.info("success to set entity for hive data node");
+        } catch (Exception e) {
+            throw new BusinessException(ErrorCodeEnum.SOURCE_INFO_INCORRECT.getMessage());

Review Comment:
   Add an error log for this exception.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/hive/HiveDataNodeInfo.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.node.hive;
+
+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.consts.DataNodeType;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+import org.apache.inlong.manager.pojo.node.DataNodeInfo;
+
+/**
+ * Data node info for hive
+ */
+@Data
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeDefine(value = DataNodeType.HIVE)
+@ApiModel("Data node info for hive")
+public class HiveDataNodeInfo extends DataNodeInfo {
+
+    @ApiModelProperty("Hive JDBC URL, such as jdbc:hive2://${ip}:${port}")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username of the Hive server")
+    private String username;

Review Comment:
   Suggest extracting the `username` and `password` into the parent class.
   
   By the way, the `token` field can replace the `password` field.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/node/hive/HiveDataNodeOperator.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.manager.service.node.hive;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.consts.DataNodeType;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.dao.entity.DataNodeEntity;
+import org.apache.inlong.manager.pojo.node.DataNodeInfo;
+import org.apache.inlong.manager.pojo.node.DataNodeRequest;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeDTO;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeInfo;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeRequest;
+import org.apache.inlong.manager.service.node.AbstractDataNodeOperator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+@Service
+public class HiveDataNodeOperator extends AbstractDataNodeOperator {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveDataNodeOperator.class);
+
+    @Autowired
+    private ObjectMapper objectMapper;
+
+    @Override
+    public Boolean accept(String dataNodeType) {
+        return getDataNodeType().equals(dataNodeType);
+    }
+
+    @Override
+    public String getDataNodeType() {
+        return DataNodeType.HIVE;
+    }
+
+    @Override
+    public DataNodeInfo getFromEntity(DataNodeEntity entity) {
+        if (entity == null) {
+            throw new BusinessException(ErrorCodeEnum.DATA_NODE_NOT_FOUND);
+        }
+
+        HiveDataNodeInfo hiveDataNodeInfo = new HiveDataNodeInfo();
+        CommonBeanUtils.copyProperties(entity, hiveDataNodeInfo);
+        if (StringUtils.isNotBlank(entity.getExtParams())) {
+            HiveDataNodeDTO dto = HiveDataNodeDTO.getFromJson(entity.getExtParams());
+            CommonBeanUtils.copyProperties(dto, hiveDataNodeInfo);
+        }
+
+        LOGGER.info("success to get data node info from entity");

Review Comment:
   Change the log level to debug.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/DataNodeInfo.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.node;
+
+import com.fasterxml.jackson.annotation.JsonFormat;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+/**
+ * Data node info
+ */
+@Data
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Data node info")
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, visible = true, property = "type")
+public abstract class DataNodeInfo {
+
+    @ApiModelProperty(value = "Primary key")
+    private Integer id;
+
+    @ApiModelProperty(value = "Cluster name")
+    private String name;
+
+    @ApiModelProperty(value = "Cluster type, including TUBEMQ, PULSAR, DATAPROXY, etc.")
+    private String type;
+
+    @ApiModelProperty(value = "Cluster url")
+    private String url;
+
+    @ApiModelProperty(value = "Cluster tags, separated by commas")
+    private String clusterTags;
+
+    @ApiModelProperty(value = "Extension tag")
+    private String extTag;
+
+    @ApiModelProperty(value = "Cluster token")
+    private String token;

Review Comment:
   Why no `username` for this token?



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/node/DataNodeOperator.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.node;
+
+import org.apache.inlong.manager.dao.entity.DataNodeEntity;
+import org.apache.inlong.manager.pojo.node.DataNodeInfo;
+import org.apache.inlong.manager.pojo.node.DataNodeRequest;
+
+public interface DataNodeOperator {

Review Comment:
   Add Java doc, please.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/node/hive/HiveDataNodeOperator.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.manager.service.node.hive;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.consts.DataNodeType;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.dao.entity.DataNodeEntity;
+import org.apache.inlong.manager.pojo.node.DataNodeInfo;
+import org.apache.inlong.manager.pojo.node.DataNodeRequest;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeDTO;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeInfo;
+import org.apache.inlong.manager.pojo.node.hive.HiveDataNodeRequest;
+import org.apache.inlong.manager.service.node.AbstractDataNodeOperator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+@Service
+public class HiveDataNodeOperator extends AbstractDataNodeOperator {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(HiveDataNodeOperator.class);
+
+    @Autowired
+    private ObjectMapper objectMapper;
+
+    @Override
+    public Boolean accept(String dataNodeType) {
+        return getDataNodeType().equals(dataNodeType);
+    }
+
+    @Override
+    public String getDataNodeType() {
+        return DataNodeType.HIVE;
+    }
+
+    @Override
+    public DataNodeInfo getFromEntity(DataNodeEntity entity) {
+        if (entity == null) {
+            throw new BusinessException(ErrorCodeEnum.DATA_NODE_NOT_FOUND);
+        }
+
+        HiveDataNodeInfo hiveDataNodeInfo = new HiveDataNodeInfo();
+        CommonBeanUtils.copyProperties(entity, hiveDataNodeInfo);
+        if (StringUtils.isNotBlank(entity.getExtParams())) {
+            HiveDataNodeDTO dto = HiveDataNodeDTO.getFromJson(entity.getExtParams());
+            CommonBeanUtils.copyProperties(dto, hiveDataNodeInfo);
+        }
+
+        LOGGER.info("success to get data node info from entity");
+        return hiveDataNodeInfo;
+    }
+
+    @Override
+    protected void setTargetEntity(DataNodeRequest request, DataNodeEntity targetEntity) {
+        HiveDataNodeRequest hiveDataNodeRequest = (HiveDataNodeRequest) request;
+        CommonBeanUtils.copyProperties(hiveDataNodeRequest, targetEntity, true);
+        try {
+            HiveDataNodeDTO dto = HiveDataNodeDTO.getFromRequest(hiveDataNodeRequest);
+            targetEntity.setExtParams(objectMapper.writeValueAsString(dto));
+            LOGGER.info("success to set entity for hive data node");

Review Comment:
   Change the log level to debug.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/DataNodeServiceImpl.java:
##########
@@ -65,33 +70,38 @@ public Integer save(DataNodeRequest request, String operator) {
             LOGGER.error(errMsg);
             throw new BusinessException(errMsg);
         }
-        DataNodeEntity entity = CommonBeanUtils.copyProperties(request, DataNodeEntity::new);
-        entity.setCreator(operator);
-        entity.setModifier(operator);
-        dataNodeMapper.insert(entity);
-
+        // According to the data type, save sink information
+        DataNodeOperator operation = operatorFactory.getInstance(request.getType());

Review Comment:
   Change the variable to `operator`.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sink/StreamSinkServiceImpl.java:
##########
@@ -266,20 +265,8 @@ public Boolean delete(Integer id, String operator) {
         StreamSinkEntity entity = sinkMapper.selectByPrimaryKey(id);
         Preconditions.checkNotNull(entity, ErrorCodeEnum.SINK_INFO_NOT_FOUND.getMessage());
         groupCheckService.checkGroupStatus(entity.getInlongGroupId(), operator);
-
-        entity.setPreviousStatus(entity.getStatus());
-        entity.setStatus(InlongConstants.DELETED_STATUS);
-        entity.setIsDeleted(id);
-        entity.setModifier(operator);
-        entity.setModifyTime(new Date());
-        int rowCount = sinkMapper.updateByPrimaryKeySelective(entity);
-        if (rowCount != InlongConstants.AFFECTED_ONE_ROW) {
-            LOGGER.error("sink has already updated with groupId={}, streamId={}, name={}, curVersion={}",
-                    entity.getInlongGroupId(), entity.getInlongStreamId(), entity.getSinkName(), entity.getVersion());
-            throw new BusinessException(ErrorCodeEnum.CONFIG_EXPIRED);
-        }
-        sinkFieldMapper.logicDeleteAll(id);
-
+        StreamSinkOperator operation = operatorFactory.getInstance(entity.getSinkType());

Review Comment:
   Suggest changing the variable to `operator`.



-- 
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 #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-service/src/test/java/org/apache/inlong/manager/service/core/impl/DataNodeServiceTest.java:
##########
@@ -96,17 +98,17 @@ public void testDataService() {
         Assertions.assertNotNull(id);
 
         // test get data node
-        DataNodeResponse nodeResponse = dataNodeService.get(id);
-        Assertions.assertNotNull(nodeResponse);
-        Assertions.assertEquals(type, nodeResponse.getType());
+        DataNodeInfo dataNodeInfo = dataNodeService.get(id);
+        Assertions.assertNotNull(dataNodeInfo);
+        Assertions.assertEquals(type, dataNodeInfo.getType());
 
         // test get data node list
-        PageInfo<DataNodeResponse> listDataNode = this.listOpt(type, nodeName);
+        PageInfo<DataNodeInfo> listDataNode = this.listOpt(type, nodeName);
         Assertions.assertEquals(listDataNode.getTotal(), 1);
 
         // test update data node
         String newNodeName = "kafkaNode1";
-        String newType = "KAFKA";
+        String newType = "HIVE";

Review Comment:
   Suggest using a String constant instead of a String variable.



-- 
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 merged pull request #5381: [INLONG-5380][Manager] Modify the saving function of the data node

Posted by GitBox <gi...@apache.org>.
EMsnap merged PR #5381:
URL: https://github.com/apache/inlong/pull/5381


-- 
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] leosanqing commented on a diff in pull request #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/DataNodeServiceImpl.java:
##########
@@ -65,34 +70,39 @@ public Integer save(DataNodeRequest request, String operator) {
             LOGGER.error(errMsg);
             throw new BusinessException(errMsg);
         }
-        DataNodeEntity entity = CommonBeanUtils.copyProperties(request, DataNodeEntity::new);
-        entity.setCreator(operator);
-        entity.setModifier(operator);
-        dataNodeMapper.insert(entity);
-
+        // According to the data type, save sink information
+        DataNodeOperator operation = operatorFactory.getInstance(request.getType());
+        int id = operation.saveOpt(request, operator);
         LOGGER.debug("success to save data node={}", request);
-        return entity.getId();
+        return id;
     }
 
     @Override
-    public DataNodeResponse get(Integer id) {
+    public DataNodeInfo get(Integer id) {
         DataNodeEntity entity = dataNodeMapper.selectById(id);
         if (entity == null) {
             LOGGER.error("data node not found by id={}", id);
             throw new BusinessException("data node not found");
         }
-        DataNodeResponse response = CommonBeanUtils.copyProperties(entity, DataNodeResponse::new);
+        String dataNodeType = entity.getType();
+        DataNodeOperator operation = operatorFactory.getInstance(dataNodeType);
+        DataNodeInfo dataNodeInfo = operation.getFromEntity(entity);
+
         LOGGER.debug("success to get data node info by id={}", id);
-        return response;
+        return dataNodeInfo;
     }
 
     @Override
-    public PageInfo<DataNodeResponse> list(DataNodePageRequest request) {
+    public PageInfo<DataNodeInfo> list(DataNodePageRequest request) {
         PageHelper.startPage(request.getPageNum(), request.getPageSize());
         Page<DataNodeEntity> entityPage = (Page<DataNodeEntity>) dataNodeMapper.selectByCondition(request);
-        List<DataNodeResponse> responseList = CommonBeanUtils.copyListProperties(entityPage, DataNodeResponse::new);
-        PageInfo<DataNodeResponse> page = new PageInfo<>(responseList);
-        page.setTotal(entityPage.getTotal());
+        List<DataNodeInfo> list = entityPage.stream()
+                .map(entity -> {
+                    DataNodeOperator instance = operatorFactory.getInstance(entity.getType());
+                    return instance.getFromEntity(entity);
+                }).collect(Collectors.toList());
+        PageInfo<DataNodeInfo> page = new PageInfo<>(list);
+        page.setTotal(list.size());

Review Comment:
   Not all in the list, at most a single page size



-- 
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] leosanqing commented on a diff in pull request #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/core/impl/DataNodeServiceImpl.java:
##########
@@ -65,34 +70,39 @@ public Integer save(DataNodeRequest request, String operator) {
             LOGGER.error(errMsg);
             throw new BusinessException(errMsg);
         }
-        DataNodeEntity entity = CommonBeanUtils.copyProperties(request, DataNodeEntity::new);
-        entity.setCreator(operator);
-        entity.setModifier(operator);
-        dataNodeMapper.insert(entity);
-
+        // According to the data type, save sink information
+        DataNodeOperator operation = operatorFactory.getInstance(request.getType());
+        int id = operation.saveOpt(request, operator);
         LOGGER.debug("success to save data node={}", request);
-        return entity.getId();
+        return id;
     }
 
     @Override
-    public DataNodeResponse get(Integer id) {
+    public DataNodeInfo get(Integer id) {
         DataNodeEntity entity = dataNodeMapper.selectById(id);
         if (entity == null) {
             LOGGER.error("data node not found by id={}", id);
             throw new BusinessException("data node not found");
         }
-        DataNodeResponse response = CommonBeanUtils.copyProperties(entity, DataNodeResponse::new);
+        String dataNodeType = entity.getType();
+        DataNodeOperator operation = operatorFactory.getInstance(dataNodeType);
+        DataNodeInfo dataNodeInfo = operation.getFromEntity(entity);
+
         LOGGER.debug("success to get data node info by id={}", id);
-        return response;
+        return dataNodeInfo;
     }
 
     @Override
-    public PageInfo<DataNodeResponse> list(DataNodePageRequest request) {
+    public PageInfo<DataNodeInfo> list(DataNodePageRequest request) {
         PageHelper.startPage(request.getPageNum(), request.getPageSize());
         Page<DataNodeEntity> entityPage = (Page<DataNodeEntity>) dataNodeMapper.selectByCondition(request);
-        List<DataNodeResponse> responseList = CommonBeanUtils.copyListProperties(entityPage, DataNodeResponse::new);
-        PageInfo<DataNodeResponse> page = new PageInfo<>(responseList);
-        page.setTotal(entityPage.getTotal());
+        List<DataNodeInfo> list = entityPage.stream()
+                .map(entity -> {
+                    DataNodeOperator instance = operatorFactory.getInstance(entity.getType());
+                    return instance.getFromEntity(entity);
+                }).collect(Collectors.toList());
+        PageInfo<DataNodeInfo> page = new PageInfo<>(list);
+        page.setTotal(list.size());

Review Comment:
   This should be `page.setTotal(entityPage.getTotal());`
   



-- 
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 #5381: [INLONG-5380][Manager] Modify the saving function of datanode

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


##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/node/hive/HiveDataNodeDTO.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.node.hive;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+import org.apache.inlong.manager.common.util.AESUtils;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.NotNull;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * hive data node info
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Hive data node info")
+public class HiveDataNodeDTO {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); // thread safe
+
+    @ApiModelProperty("Hive JDBC URL, such as jdbc:hive2://${ip}:${port}")
+    private String jdbcUrl;
+
+    @ApiModelProperty("Username of the Hive server")
+    private String username;
+
+    @ApiModelProperty("User password of the Hive server")
+    private String password;
+
+    @NotBlank(message = "dbName cannot be blank")
+    @ApiModelProperty("Target database name")
+    private String dbName;

Review Comment:
   Not need dbName, tableName, dataPath, and encryptVersion, as in the different Hive sinks, the db and table may not same.



-- 
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