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/01 07:13:43 UTC

[GitHub] [inlong] healchow commented on a diff in pull request #5389: [INLONG-4976][Manager] The Manager module supports the use of Kafka

healchow commented on code in PR #5389:
URL: https://github.com/apache/inlong/pull/5389#discussion_r960294694


##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/kafka/KafkaOperator.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.resource.queue.kafka;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.inlong.manager.pojo.cluster.kafka.KafkaClusterInfo;
+import org.apache.inlong.manager.service.cluster.InlongClusterServiceImpl;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Service;
+
+/**
+ * kafka operator, supports creating topics and creating subscription.
+ */
+@Service
+public class KafkaOperator {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(InlongClusterServiceImpl.class);
+
+    /**
+     * Create Kafka topic
+     */
+    public void createTopic(KafkaClusterInfo kafkaClusterInfo, String topicName)
+            throws InterruptedException, ExecutionException {
+        AdminClient adminClient = KafkaUtils.getAdminClient(kafkaClusterInfo);
+        NewTopic topic = new NewTopic(topicName,
+                kafkaClusterInfo.getNumPartitions(),
+                kafkaClusterInfo.getReplicationFactor());
+        CreateTopicsResult result = adminClient.createTopics(Collections.singletonList(topic));
+        // To prevent the client from disconnecting too quickly and causing the Topic to not be created successfully
+        Thread.sleep(500);
+        LOGGER.info("success to create kafka topic={}, with={} numPartitions",
+                topicName,
+                result.numPartitions(topicName).get());
+    }
+
+    /**
+     * Force delete Kafka topic
+     */
+    public void forceDeleteTopic(KafkaClusterInfo kafkaClusterInfo, String topicName) {
+        AdminClient adminClient = KafkaUtils.getAdminClient(kafkaClusterInfo);
+        DeleteTopicsResult result = adminClient.deleteTopics(Collections.singletonList(topicName));
+        LOGGER.info("success to delete topic={}", topicName);
+    }
+
+    public boolean topicIsExists(KafkaClusterInfo kafkaClusterInfo, String topic)
+            throws ExecutionException, InterruptedException {
+        AdminClient adminClient = KafkaUtils.getAdminClient(kafkaClusterInfo);
+        Set<String> topicList = adminClient.listTopics().names().get();
+        return topicList.contains(topic);
+    }
+
+    public void createSubscription(KafkaClusterInfo kafkaClusterInfo, String subscription) {
+
+        KafkaConsumer kafkaConsumer = KafkaUtils.createKafkaConsumer(kafkaClusterInfo);
+        // subscription
+        kafkaConsumer.subscribe(Collections.singletonList(subscription));
+    }
+
+    public boolean subscriptionIsExists(KafkaClusterInfo kafkaClusterInfo, String topic) {
+        KafkaConsumer consumer = KafkaUtils.createKafkaConsumer(kafkaClusterInfo);
+        try {
+            Map<String, List<PartitionInfo>> topics = consumer.listTopics();
+            List<PartitionInfo> partitions = topics.get(topic);
+            if (partitions == null) {
+                LOGGER.info("subscription is not exist");
+                return false;
+            }
+            return true;
+        } finally {
+            consumer.close();
+        }
+

Review Comment:
   Remove the blank line here.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/kafka/KafkaOperator.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.resource.queue.kafka;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.inlong.manager.pojo.cluster.kafka.KafkaClusterInfo;
+import org.apache.inlong.manager.service.cluster.InlongClusterServiceImpl;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Service;
+
+/**
+ * kafka operator, supports creating topics and creating subscription.
+ */
+@Service
+public class KafkaOperator {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(InlongClusterServiceImpl.class);
+
+    /**
+     * Create Kafka topic
+     */
+    public void createTopic(KafkaClusterInfo kafkaClusterInfo, String topicName)
+            throws InterruptedException, ExecutionException {
+        AdminClient adminClient = KafkaUtils.getAdminClient(kafkaClusterInfo);
+        NewTopic topic = new NewTopic(topicName,
+                kafkaClusterInfo.getNumPartitions(),
+                kafkaClusterInfo.getReplicationFactor());
+        CreateTopicsResult result = adminClient.createTopics(Collections.singletonList(topic));
+        // To prevent the client from disconnecting too quickly and causing the Topic to not be created successfully
+        Thread.sleep(500);
+        LOGGER.info("success to create kafka topic={}, with={} numPartitions",
+                topicName,
+                result.numPartitions(topicName).get());
+    }
+
+    /**
+     * Force delete Kafka topic
+     */
+    public void forceDeleteTopic(KafkaClusterInfo kafkaClusterInfo, String topicName) {
+        AdminClient adminClient = KafkaUtils.getAdminClient(kafkaClusterInfo);
+        DeleteTopicsResult result = adminClient.deleteTopics(Collections.singletonList(topicName));
+        LOGGER.info("success to delete topic={}", topicName);
+    }
+
+    public boolean topicIsExists(KafkaClusterInfo kafkaClusterInfo, String topic)
+            throws ExecutionException, InterruptedException {
+        AdminClient adminClient = KafkaUtils.getAdminClient(kafkaClusterInfo);
+        Set<String> topicList = adminClient.listTopics().names().get();
+        return topicList.contains(topic);
+    }
+
+    public void createSubscription(KafkaClusterInfo kafkaClusterInfo, String subscription) {
+
+        KafkaConsumer kafkaConsumer = KafkaUtils.createKafkaConsumer(kafkaClusterInfo);
+        // subscription
+        kafkaConsumer.subscribe(Collections.singletonList(subscription));
+    }
+
+    public boolean subscriptionIsExists(KafkaClusterInfo kafkaClusterInfo, String topic) {
+        KafkaConsumer consumer = KafkaUtils.createKafkaConsumer(kafkaClusterInfo);

Review Comment:
   Is it possible to use a `try-with-resource` statement?



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/kafka/KafkaResourceOperators.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.resource.queue.kafka;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.NotNull;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.inlong.manager.common.consts.MQType;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.exceptions.WorkflowListenerException;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
+import org.apache.inlong.manager.pojo.cluster.kafka.KafkaClusterInfo;
+import org.apache.inlong.manager.pojo.group.InlongGroupInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamBriefInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
+import org.apache.inlong.manager.service.cluster.InlongClusterService;
+import org.apache.inlong.manager.service.core.ConsumptionService;
+import org.apache.inlong.manager.service.resource.queue.QueueResourceOperator;
+import org.apache.inlong.manager.service.stream.InlongStreamService;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import java.util.List;
+
+/**
+ * kafka for create kafka Topic and Subscription
+ */
+@Slf4j
+@Service
+public class KafkaResourceOperators implements QueueResourceOperator {
+
+    @Autowired
+    private InlongClusterService clusterService;
+    @Autowired
+    private InlongStreamService streamService;
+    @Autowired
+    private KafkaOperator kafkaOperator;
+    @Autowired
+    private ConsumptionService consumptionService;
+
+    @Override
+    public boolean accept(String mqType) {
+        return MQType.KAFKA.equals(mqType);
+    }
+
+    @Override
+    public void createQueueForGroup(@NotNull InlongGroupInfo groupInfo, @NotBlank String operator) {
+
+        String groupId = groupInfo.getInlongGroupId();
+        log.info("begin to create kafka resource for groupId={}", groupId);
+
+        // get kafka cluster via the inlong cluster tag from the inlong group
+        String clusterTag = groupInfo.getInlongClusterTag();
+        KafkaClusterInfo kafkaCluster = (KafkaClusterInfo) clusterService.getOne(clusterTag, null,
+                ClusterType.KAFKA);
+        try {
+            // 1. create kafka Topic - each Inlong Stream corresponds to a Kafka Topic
+            List<InlongStreamBriefInfo> streamInfoList = streamService.getTopicList(groupId);
+            if (streamInfoList == null || streamInfoList.isEmpty()) {
+                log.warn("skip to create kafka topic and subscription as no streams for groupId={}", groupId);
+                return;
+            }
+            for (InlongStreamBriefInfo streamInfo : streamInfoList) {
+                this.createKafkaTopic(groupInfo, kafkaCluster, streamInfo.getInlongStreamId());
+            }
+        } catch (Exception e) {
+            String msg = String.format("failed to create kafka resource for groupId=%s", groupId);
+            log.error(msg, e);
+            throw new WorkflowListenerException(msg + ": " + e.getMessage());
+        }
+
+        log.info("success to create kafka resource for groupId={}, cluster={}", groupId, kafkaCluster);
+    }
+
+    @Override
+    public void deleteQueueForGroup(InlongGroupInfo groupInfo, String operator) {
+        Preconditions.checkNotNull(groupInfo, "inlong group info cannot be null");
+
+        String groupId = groupInfo.getInlongGroupId();
+        log.info("begin to delete kafka resource for groupId={}", groupId);
+
+        ClusterInfo clusterInfo = clusterService.getOne(groupInfo.getInlongClusterTag(), null, ClusterType.KAFKA);
+        try {
+            List<InlongStreamBriefInfo> streamInfoList = streamService.getTopicList(groupId);
+            if (streamInfoList == null || streamInfoList.isEmpty()) {
+                log.warn("skip to create kafka topic and subscription as no streams for groupId={}", groupId);
+                return;
+            }
+            for (InlongStreamBriefInfo streamInfo : streamInfoList) {
+                this.deleteKafkaTopic(groupInfo, (KafkaClusterInfo) clusterInfo, streamInfo.getInlongStreamId());
+            }
+        } catch (Exception e) {
+            log.error("failed to delete kafka resource for groupId=" + groupId, e);
+            throw new WorkflowListenerException("failed to delete kafka resource: " + e.getMessage());
+        }
+
+        log.info("success to delete kafka resource for groupId={}, cluster={}", groupId, clusterInfo);
+

Review Comment:
   Remove the blank line here.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/group/InlongKafkaOperator.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.group;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.common.consts.MQType;
+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.InlongGroupEntity;
+import org.apache.inlong.manager.pojo.group.InlongGroupInfo;
+import org.apache.inlong.manager.pojo.group.InlongGroupRequest;
+import org.apache.inlong.manager.pojo.group.InlongGroupTopicInfo;
+import org.apache.inlong.manager.pojo.group.kafka.InlongKafkaDTO;
+import org.apache.inlong.manager.pojo.group.kafka.InlongKafkaInfo;
+import org.apache.inlong.manager.pojo.group.kafka.InlongKafkaRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+/**
+ * Inlong group operator for Kafka.
+ */
+@Service
+public class InlongKafkaOperator extends AbstractGroupOperator {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(InlongKafkaOperator.class);
+
+    @Autowired
+    private ObjectMapper objectMapper;
+
+    @Override
+    public Boolean accept(String mqType) {
+        return getMQType().equals(mqType);
+    }
+
+    @Override
+    public String getMQType() {
+        return MQType.KAFKA;
+    }
+
+    @Override
+    public InlongGroupInfo getFromEntity(InlongGroupEntity entity) {
+        if (entity == null) {
+            throw new BusinessException(ErrorCodeEnum.GROUP_NOT_FOUND);
+        }
+
+        InlongKafkaInfo groupInfo = new InlongKafkaInfo();
+        CommonBeanUtils.copyProperties(entity, groupInfo);
+
+        if (StringUtils.isNotBlank(entity.getExtParams())) {
+            InlongKafkaDTO dto = InlongKafkaDTO.getFromJson(entity.getExtParams());
+            CommonBeanUtils.copyProperties(dto, groupInfo);
+        }
+
+        return groupInfo;
+    }
+
+    @Override
+    protected void setTargetEntity(InlongGroupRequest request, InlongGroupEntity targetEntity) {
+        InlongKafkaRequest kafkaRequest = (InlongKafkaRequest) request;
+
+        //TODO check kafka params

Review Comment:
   Please add one blank space after all `//`.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/cluster/kafka/KafkaClusterInfo.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cluster.kafka;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import lombok.experimental.SuperBuilder;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
+
+/**
+ * Inlong cluster info for Kafka
+ */
+@Data
+@SuperBuilder
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeDefine(value = ClusterType.KAFKA)
+@ApiModel("Inlong cluster info for Kafka")
+public class KafkaClusterInfo extends ClusterInfo {
+
+    @ApiModelProperty(value = "Kafka admin bootStrapServers, such as: 127.0.0.1:9092",
+            notes = "Kafka service URL is the 'url' field of the cluster")
+    private String bootstrapServers;

Review Comment:
   `bootstrapServers` can be saved to the `ClusterInfo#url`.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/cluster/kafka/KafkaClusterRequest.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.cluster.kafka;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+import org.apache.inlong.manager.pojo.cluster.ClusterRequest;
+
+import javax.validation.constraints.NotBlank;
+
+/**
+ * Inlong cluster request for Kafka
+ */
+@Data
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeDefine(value = ClusterType.KAFKA)
+@ApiModel("Inlong cluster request for Kafka")
+public class KafkaClusterRequest extends ClusterRequest {
+
+    @NotBlank(message = "adminUrl cannot be blank")
+    @ApiModelProperty(value = "Kafka admin URL, such as: http://127.0.0.1:8080",
+            notes = "Kafka service URL is the 'url' field of the cluster")
+    private String adminUrl;

Review Comment:
   Not needed.



##########
inlong-manager/manager-service/pom.xml:
##########
@@ -126,7 +126,6 @@
                 </exclusion>
             </exclusions>
         </dependency>
-

Review Comment:
   Please do not remove this blank line.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/kafka/KafkaResourceOperators.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.resource.queue.kafka;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.NotNull;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.inlong.manager.common.consts.MQType;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.exceptions.WorkflowListenerException;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
+import org.apache.inlong.manager.pojo.cluster.kafka.KafkaClusterInfo;
+import org.apache.inlong.manager.pojo.group.InlongGroupInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamBriefInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
+import org.apache.inlong.manager.service.cluster.InlongClusterService;
+import org.apache.inlong.manager.service.core.ConsumptionService;
+import org.apache.inlong.manager.service.resource.queue.QueueResourceOperator;
+import org.apache.inlong.manager.service.stream.InlongStreamService;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import java.util.List;
+
+/**
+ * kafka for create kafka Topic and Subscription

Review Comment:
   Change to `Operator for creating Kafka Topic and Subscription`.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/kafka/KafkaResourceOperators.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.resource.queue.kafka;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.NotNull;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.inlong.manager.common.consts.MQType;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.exceptions.WorkflowListenerException;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
+import org.apache.inlong.manager.pojo.cluster.kafka.KafkaClusterInfo;
+import org.apache.inlong.manager.pojo.group.InlongGroupInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamBriefInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
+import org.apache.inlong.manager.service.cluster.InlongClusterService;
+import org.apache.inlong.manager.service.core.ConsumptionService;
+import org.apache.inlong.manager.service.resource.queue.QueueResourceOperator;
+import org.apache.inlong.manager.service.stream.InlongStreamService;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import java.util.List;
+
+/**
+ * kafka for create kafka Topic and Subscription
+ */
+@Slf4j
+@Service
+public class KafkaResourceOperators implements QueueResourceOperator {
+
+    @Autowired
+    private InlongClusterService clusterService;
+    @Autowired
+    private InlongStreamService streamService;
+    @Autowired
+    private KafkaOperator kafkaOperator;
+    @Autowired
+    private ConsumptionService consumptionService;
+
+    @Override
+    public boolean accept(String mqType) {
+        return MQType.KAFKA.equals(mqType);
+    }
+
+    @Override
+    public void createQueueForGroup(@NotNull InlongGroupInfo groupInfo, @NotBlank String operator) {
+

Review Comment:
   Not need the blank line here.



##########
inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/kafka/KafkaResourceOperators.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.resource.queue.kafka;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.NotNull;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.inlong.manager.common.consts.MQType;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.exceptions.WorkflowListenerException;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
+import org.apache.inlong.manager.pojo.cluster.kafka.KafkaClusterInfo;
+import org.apache.inlong.manager.pojo.group.InlongGroupInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamBriefInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
+import org.apache.inlong.manager.service.cluster.InlongClusterService;
+import org.apache.inlong.manager.service.core.ConsumptionService;
+import org.apache.inlong.manager.service.resource.queue.QueueResourceOperator;
+import org.apache.inlong.manager.service.stream.InlongStreamService;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import java.util.List;
+
+/**
+ * kafka for create kafka Topic and Subscription
+ */
+@Slf4j
+@Service
+public class KafkaResourceOperators implements QueueResourceOperator {
+
+    @Autowired
+    private InlongClusterService clusterService;
+    @Autowired
+    private InlongStreamService streamService;
+    @Autowired
+    private KafkaOperator kafkaOperator;
+    @Autowired
+    private ConsumptionService consumptionService;
+
+    @Override
+    public boolean accept(String mqType) {
+        return MQType.KAFKA.equals(mqType);
+    }
+
+    @Override
+    public void createQueueForGroup(@NotNull InlongGroupInfo groupInfo, @NotBlank String operator) {
+
+        String groupId = groupInfo.getInlongGroupId();
+        log.info("begin to create kafka resource for groupId={}", groupId);
+
+        // get kafka cluster via the inlong cluster tag from the inlong group
+        String clusterTag = groupInfo.getInlongClusterTag();
+        KafkaClusterInfo kafkaCluster = (KafkaClusterInfo) clusterService.getOne(clusterTag, null,
+                ClusterType.KAFKA);
+        try {
+            // 1. create kafka Topic - each Inlong Stream corresponds to a Kafka Topic
+            List<InlongStreamBriefInfo> streamInfoList = streamService.getTopicList(groupId);
+            if (streamInfoList == null || streamInfoList.isEmpty()) {
+                log.warn("skip to create kafka topic and subscription as no streams for groupId={}", groupId);
+                return;
+            }
+            for (InlongStreamBriefInfo streamInfo : streamInfoList) {
+                this.createKafkaTopic(groupInfo, kafkaCluster, streamInfo.getInlongStreamId());
+            }
+        } catch (Exception e) {
+            String msg = String.format("failed to create kafka resource for groupId=%s", groupId);
+            log.error(msg, e);
+            throw new WorkflowListenerException(msg + ": " + e.getMessage());
+        }
+
+        log.info("success to create kafka resource for groupId={}, cluster={}", groupId, kafkaCluster);
+    }
+
+    @Override
+    public void deleteQueueForGroup(InlongGroupInfo groupInfo, String operator) {
+        Preconditions.checkNotNull(groupInfo, "inlong group info cannot be null");
+
+        String groupId = groupInfo.getInlongGroupId();
+        log.info("begin to delete kafka resource for groupId={}", groupId);
+
+        ClusterInfo clusterInfo = clusterService.getOne(groupInfo.getInlongClusterTag(), null, ClusterType.KAFKA);
+        try {
+            List<InlongStreamBriefInfo> streamInfoList = streamService.getTopicList(groupId);
+            if (streamInfoList == null || streamInfoList.isEmpty()) {
+                log.warn("skip to create kafka topic and subscription as no streams for groupId={}", groupId);
+                return;
+            }
+            for (InlongStreamBriefInfo streamInfo : streamInfoList) {
+                this.deleteKafkaTopic(groupInfo, (KafkaClusterInfo) clusterInfo, streamInfo.getInlongStreamId());
+            }
+        } catch (Exception e) {
+            log.error("failed to delete kafka resource for groupId=" + groupId, e);
+            throw new WorkflowListenerException("failed to delete kafka resource: " + e.getMessage());
+        }
+
+        log.info("success to delete kafka resource for groupId={}, cluster={}", groupId, clusterInfo);
+
+    }
+
+    @Override
+    public void createQueueForStream(InlongGroupInfo groupInfo, InlongStreamInfo streamInfo,
+            String operator) {
+        Preconditions.checkNotNull(groupInfo, "inlong group info cannot be null");
+        Preconditions.checkNotNull(streamInfo, "inlong stream info cannot be null");
+        Preconditions.checkNotNull(operator, "operator cannot be null");
+
+        String groupId = streamInfo.getInlongGroupId();
+        String streamId = streamInfo.getInlongStreamId();
+        log.info("begin to create kafka resource for groupId={}, streamId={}", groupId, streamId);
+
+        try {
+            // get kafka cluster via the inlong cluster tag from the inlong group
+            String clusterTag = groupInfo.getInlongClusterTag();
+            ClusterInfo clusterInfo = clusterService.getOne(clusterTag, null, ClusterType.KAFKA);
+            // create kafka topic
+            this.createKafkaTopic(groupInfo, (KafkaClusterInfo) clusterInfo, streamInfo.getInlongStreamId());
+        } catch (Exception e) {
+            String msg = String.format("failed to create kafka topic for groupId=%s, streamId=%s", groupId, streamId);
+            log.error(msg, e);
+            throw new WorkflowListenerException(msg + ": " + e.getMessage());
+        }
+
+        log.info("success to create kafka resource for groupId={}, streamId={}", groupId, streamId);
+

Review Comment:
   Same as above.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/cluster/kafka/KafkaClusterInfo.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cluster.kafka;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+import lombok.experimental.SuperBuilder;
+import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.util.CommonBeanUtils;
+import org.apache.inlong.manager.common.util.JsonTypeDefine;
+import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
+
+/**
+ * Inlong cluster info for Kafka
+ */
+@Data
+@SuperBuilder
+@ToString(callSuper = true)
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeDefine(value = ClusterType.KAFKA)
+@ApiModel("Inlong cluster info for Kafka")
+public class KafkaClusterInfo extends ClusterInfo {
+
+    @ApiModelProperty(value = "Kafka admin bootStrapServers, such as: 127.0.0.1:9092",
+            notes = "Kafka service URL is the 'url' field of the cluster")
+    private String bootstrapServers;
+    // partition number
+    private int numPartitions;
+    // replicationFactor number
+    short replicationFactor = 1;
+    //consumer grouping
+    private String groupId;
+    // autocommit interval
+    private String autoCommit;
+
+    //TODO add new attribute

Review Comment:
   Not needing those parameters, they should belong to the InlongGroup layer, because those params may use other values for different topics.



##########
inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/cluster/kafka/KafkaClusterDTO.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.cluster.kafka;
+
+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.inlong.manager.common.enums.ErrorCodeEnum;
+import org.apache.inlong.manager.common.exceptions.BusinessException;
+
+import javax.validation.constraints.NotNull;
+
+/**
+ * Kafka cluster info
+ */
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+@ApiModel("Kafka cluster info")
+public class KafkaClusterDTO {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
+            .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); // thread safe
+
+    @ApiModelProperty(value = "Kafka admin URL, such as: http://127.0.0.1:8080",
+            notes = "Pulsar service URL is the 'url' field of the cluster")

Review Comment:
   Kafka only needs BOOTSTRAP_SERVERS_CONFIG, it can be saved to the URL in ClusterInfo 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