You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/03/07 01:32:13 UTC

[incubator-inlong] branch master updated: [INLONG-2383][SDK] Support Kafka to consume PB compressed message protocol (#2797)

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

dockerzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 985b24a  [INLONG-2383][SDK] Support Kafka to consume PB compressed message protocol (#2797)
985b24a is described below

commit 985b24a6f4c5c08e651adfd1aff8e295cd3c13e7
Author: wardli <95...@users.noreply.github.com>
AuthorDate: Mon Mar 7 09:31:59 2022 +0800

    [INLONG-2383][SDK] Support Kafka to consume PB compressed message protocol (#2797)
---
 inlong-sdk/sort-sdk/pom.xml                        |   7 +
 .../inlong/sdk/sort/api/SortClientConfig.java      |  27 ++
 .../sdk/sort/impl/InLongTopicManagerImpl.java      |  27 +-
 .../sdk/sort/impl/kafka/AckOffsetOnRebalance.java  |  61 ++++
 .../sort/impl/kafka/InLongKafkaFetcherImpl.java    | 324 +++++++++++++++++++++
 .../impl/kafka/InLongKafkaFetcherImplTest.java     |  96 ++++++
 6 files changed, 539 insertions(+), 3 deletions(-)

diff --git a/inlong-sdk/sort-sdk/pom.xml b/inlong-sdk/sort-sdk/pom.xml
index ea5d8b2..8f7fd65 100644
--- a/inlong-sdk/sort-sdk/pom.xml
+++ b/inlong-sdk/sort-sdk/pom.xml
@@ -77,6 +77,7 @@
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <plugin.compile.version>3.8.1</plugin.compile.version>
         <pulsar.version>2.8.1</pulsar.version>
+        <kafka.version>2.4.1</kafka.version>
         <httpclient.version>4.5.13</httpclient.version>
         <snappy-java.version>1.1.7.3</snappy-java.version>
     </properties>
@@ -101,6 +102,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+            <version>${kafka.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.inlong</groupId>
             <artifactId>sdk-common</artifactId>
             <version>${project.version}</version>
diff --git a/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/api/SortClientConfig.java b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/api/SortClientConfig.java
index 116e7e5..7d96745 100644
--- a/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/api/SortClientConfig.java
+++ b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/api/SortClientConfig.java
@@ -32,6 +32,9 @@ public class SortClientConfig implements Serializable {
     private ReadCallback callback;
     private int callbackQueueSize = 100;
     private int pulsarReceiveQueueSize = 2000;
+    private int kafkaFetchWaitMs = 5000;
+    private int kafkaFetchSizeBytes = 3 * 1024 * 1024;
+    private int kafkaSocketRecvBufferSize = 5 * 1024 * 1024;
     private Semaphore globalInProgressRequest = new Semaphore(Short.MAX_VALUE, true);
     private String localIp;
     private String appName;
@@ -160,6 +163,30 @@ public class SortClientConfig implements Serializable {
         this.pulsarReceiveQueueSize = pulsarReceiveQueueSize;
     }
 
+    public int getKafkaFetchWaitMs() {
+        return kafkaFetchWaitMs;
+    }
+
+    public void setKafkaFetchWaitMs(int kafkaFetchWaitMs) {
+        this.kafkaFetchWaitMs = kafkaFetchWaitMs;
+    }
+
+    public int getKafkaFetchSizeBytes() {
+        return kafkaFetchSizeBytes;
+    }
+
+    public void setKafkaFetchSizeBytes(int kafkaFetchSizeBytes) {
+        this.kafkaFetchSizeBytes = kafkaFetchSizeBytes;
+    }
+
+    public int getKafkaSocketRecvBufferSize() {
+        return kafkaSocketRecvBufferSize;
+    }
+
+    public void setKafkaSocketRecvBufferSize(int kafkaSocketRecvBufferSize) {
+        this.kafkaSocketRecvBufferSize = kafkaSocketRecvBufferSize;
+    }
+
     public Semaphore getGlobalInProgressRequest() {
         return globalInProgressRequest;
     }
diff --git a/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/InLongTopicManagerImpl.java b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/InLongTopicManagerImpl.java
index 48885db..a4c66ff 100644
--- a/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/InLongTopicManagerImpl.java
+++ b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/InLongTopicManagerImpl.java
@@ -35,6 +35,7 @@ import org.apache.inlong.sdk.sort.api.InlongTopicTypeEnum;
 import org.apache.inlong.sdk.sort.api.QueryConsumeConfig;
 import org.apache.inlong.sdk.sort.entity.ConsumeConfig;
 import org.apache.inlong.sdk.sort.entity.InLongTopic;
+import org.apache.inlong.sdk.sort.impl.kafka.InLongKafkaFetcherImpl;
 import org.apache.inlong.sdk.sort.impl.pulsar.InLongPulsarFetcherImpl;
 import org.apache.inlong.sdk.sort.impl.tube.InLongTubeFetcherImpl;
 import org.apache.inlong.sdk.sort.impl.tube.TubeConsumerCreater;
@@ -74,6 +75,22 @@ public class InLongTopicManagerImpl extends InLongTopicManager {
         toBeSelectFetchers = new ArrayList<>(c);
     }
 
+    private boolean initFetcher(InLongTopicFetcher fetcher, InLongTopic inLongTopic) {
+        if (InlongTopicTypeEnum.PULSAR.getName().equals(inLongTopic.getTopicType())) {
+            logger.info("create fetcher topic is pulsar {}", inLongTopic);
+            return fetcher.init(pulsarClients.get(inLongTopic.getInLongCluster().getClusterId()));
+        } else if (InlongTopicTypeEnum.KAFKA.getName().equals(inLongTopic.getTopicType())) {
+            logger.info("create fetcher topic is kafka {}", inLongTopic);
+            return fetcher.init(inLongTopic.getInLongCluster().getBootstraps());
+        } else if (InlongTopicTypeEnum.TUBE.getName().equals(inLongTopic.getTopicType())) {
+            logger.info("create fetcher topic is tube {}", inLongTopic);
+            return fetcher.init(tubeFactories.get(inLongTopic.getInLongCluster().getClusterId()));
+        } else {
+            logger.error("create fetcher topic type not support " + inLongTopic.getTopicType());
+            return false;
+        }
+    }
+
     @Override
     public InLongTopicFetcher addFetcher(InLongTopic inLongTopic) {
 
@@ -93,7 +110,7 @@ public class InLongTopicManagerImpl extends InLongTopicManager {
                 } else {
                     result = inLongTopicFetcher;
                     if (result != null
-                            && !result.init(pulsarClients.get(inLongTopic.getInLongCluster().getClusterId()))) {
+                            && !initFetcher(result, inLongTopic)) {
                         logger.info("addFetcher init fail {}", inLongTopic.getTopicKey());
                         result.close();
                         result = null;
@@ -116,6 +133,9 @@ public class InLongTopicManagerImpl extends InLongTopicManager {
         if (InlongTopicTypeEnum.PULSAR.getName().equals(inLongTopic.getTopicType())) {
             logger.info("the topic is pulsar {}", inLongTopic);
             return new InLongPulsarFetcherImpl(inLongTopic, context);
+        } else if (InlongTopicTypeEnum.KAFKA.getName().equals(inLongTopic.getTopicType())) {
+            logger.info("the topic is kafka {}", inLongTopic);
+            return new InLongKafkaFetcherImpl(inLongTopic, context);
         } else if (InlongTopicTypeEnum.TUBE.getName().equals(inLongTopic.getTopicType())) {
             logger.info("the topic is tube {}", inLongTopic);
             return new InLongTubeFetcherImpl(inLongTopic, context);
@@ -211,7 +231,7 @@ public class InLongTopicManagerImpl extends InLongTopicManager {
                     logger.error(e.getMessage(), e);
                 }
             }
-            logger.info(" close {} {}", fetchKey, succ);
+            logger.info(" close fetcher{} {}", fetchKey, succ);
         }
     }
 
@@ -357,7 +377,7 @@ public class InLongTopicManagerImpl extends InLongTopicManager {
                             .authentication(AuthenticationFactory.token(inLongTopic.getInLongCluster().getToken()))
                             .build();
                     pulsarClients.put(inLongTopic.getInLongCluster().getClusterId(), pulsarClient);
-                    logger.info("create pulsar client succ {} {} {}",
+                    logger.info("create pulsar client succ {}",
                             new String[]{inLongTopic.getInLongCluster().getClusterId(),
                                     inLongTopic.getInLongCluster().getBootstraps(),
                                     inLongTopic.getInLongCluster().getToken()});
@@ -404,6 +424,7 @@ public class InLongTopicManagerImpl extends InLongTopicManager {
     }
 
     private void onlineKafkaTopic(InLongTopic inLongTopic) {
+        createNewFetcher(inLongTopic);
     }
 
     private void onlineTubeTopic(InLongTopic inLongTopic) {
diff --git a/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/kafka/AckOffsetOnRebalance.java b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/kafka/AckOffsetOnRebalance.java
new file mode 100644
index 0000000..c09385b
--- /dev/null
+++ b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/kafka/AckOffsetOnRebalance.java
@@ -0,0 +1,61 @@
+/*
+ * 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.sdk.sort.impl.kafka;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AckOffsetOnRebalance implements ConsumerRebalanceListener {
+
+    private final Logger logger = LoggerFactory.getLogger(AckOffsetOnRebalance.class);
+    private final KafkaConsumer<byte[], byte[]> consumer;
+    private final ConcurrentHashMap<TopicPartition, OffsetAndMetadata> commitOffsetMap;
+
+    public AckOffsetOnRebalance(KafkaConsumer<byte[], byte[]> consumer,
+            ConcurrentHashMap<TopicPartition, OffsetAndMetadata> commitOffsetMap) {
+        this.consumer = consumer;
+        this.commitOffsetMap = commitOffsetMap;
+    }
+
+    @Override
+    public void onPartitionsRevoked(Collection<TopicPartition> collection) {
+        logger.info("*- in ralance:onPartitionsRevoked");
+        while (!commitOffsetMap.isEmpty()) {
+            consumer.commitSync(commitOffsetMap);
+        }
+    }
+
+    @Override
+    public void onPartitionsAssigned(Collection<TopicPartition> collection) {
+        logger.info("*- in ralance:onPartitionsAssigned  ");
+        Map<TopicPartition, OffsetAndMetadata> committed = consumer.committed(new HashSet<>(collection));
+        for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : committed.entrySet()) {
+            consumer.seek(entry.getKey(), entry.getValue().offset());
+        }
+    }
+}
diff --git a/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/kafka/InLongKafkaFetcherImpl.java b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/kafka/InLongKafkaFetcherImpl.java
new file mode 100644
index 0000000..531e78f
--- /dev/null
+++ b/inlong-sdk/sort-sdk/src/main/java/org/apache/inlong/sdk/sort/impl/kafka/InLongKafkaFetcherImpl.java
@@ -0,0 +1,324 @@
+/*
+ * 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.sdk.sort.impl.kafka;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.inlong.sdk.sort.api.ClientContext;
+import org.apache.inlong.sdk.sort.api.InLongTopicFetcher;
+import org.apache.inlong.sdk.sort.api.SortClientConfig.ConsumeStrategy;
+import org.apache.inlong.sdk.sort.entity.InLongMessage;
+import org.apache.inlong.sdk.sort.entity.InLongTopic;
+import org.apache.inlong.sdk.sort.entity.MessageRecord;
+import org.apache.inlong.sdk.sort.util.StringUtil;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class InLongKafkaFetcherImpl extends InLongTopicFetcher {
+
+    private final Logger logger = LoggerFactory.getLogger(InLongKafkaFetcherImpl.class);
+    private final ConcurrentHashMap<TopicPartition, OffsetAndMetadata> commitOffsetMap = new ConcurrentHashMap<>();
+    private final AtomicLong ackOffsets = new AtomicLong(0);
+    private volatile boolean stopConsume = false;
+    private String bootstrapServers;
+    private KafkaConsumer<byte[], byte[]> consumer;
+
+    public InLongKafkaFetcherImpl(InLongTopic inLongTopic, ClientContext context) {
+        super(inLongTopic, context);
+    }
+
+    @Override
+    public boolean init(Object object) {
+        String bootstrapServers = (String) object;
+        try {
+            createKafkaConsumer(bootstrapServers);
+            if (consumer != null) {
+                consumer.subscribe(Collections.singletonList(inLongTopic.getTopic()),
+                        new AckOffsetOnRebalance(consumer, commitOffsetMap));
+            } else {
+                return false;
+            }
+            this.bootstrapServers = bootstrapServers;
+            String threadName = "sort_sdk_fetch_thread_" + StringUtil.formatDate(new Date(), "yyyy-MM-dd HH:mm:ss");
+            this.fetchThread = new Thread(new Fetcher(), threadName);
+            this.fetchThread.start();
+        } catch (Exception e) {
+            logger.error(e.getMessage(), e);
+            return false;
+        }
+        return true;
+    }
+
+    @Override
+    public void ack(String msgOffset) throws Exception {
+        String[] offset = msgOffset.split(":");
+        if (offset.length == 2) {
+            TopicPartition topicPartition = new TopicPartition(inLongTopic.getTopic(), Integer.parseInt(offset[0]));
+            OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(Long.parseLong(offset[1]));
+            commitOffsetMap.put(topicPartition, offsetAndMetadata);
+        }
+    }
+
+    @Override
+    public void pause() {
+        this.stopConsume = true;
+    }
+
+    @Override
+    public void resume() {
+        this.stopConsume = false;
+    }
+
+    @Override
+    public boolean close() {
+        this.closed = true;
+        try {
+            if (fetchThread != null) {
+                fetchThread.interrupt();
+            }
+            if (consumer != null) {
+                consumer.close();
+            }
+        } catch (Throwable throwable) {
+            throwable.printStackTrace();
+        }
+        logger.info("closed {}", inLongTopic);
+        return true;
+    }
+
+    @Override
+    public boolean isClosed() {
+        return false;
+    }
+
+    @Override
+    public void stopConsume(boolean stopConsume) {
+        this.stopConsume = stopConsume;
+    }
+
+    @Override
+    public boolean isConsumeStop() {
+        return this.stopConsume;
+    }
+
+    @Override
+    public InLongTopic getInLongTopic() {
+        return inLongTopic;
+    }
+
+    @Override
+    public long getConsumedDataSize() {
+        return 0;
+    }
+
+    @Override
+    public long getAckedOffset() {
+        return 0;
+    }
+
+    private void createKafkaConsumer(String bootstrapServers) {
+        Properties properties = new Properties();
+        properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+        properties.put(ConsumerConfig.CLIENT_ID_CONFIG, context.getConfig().getSortTaskId());
+        properties.put(ConsumerConfig.GROUP_ID_CONFIG, context.getConfig().getSortTaskId());
+        properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
+                ByteArrayDeserializer.class.getName());
+        properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
+                ByteArrayDeserializer.class.getName());
+        properties.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG,
+                context.getConfig().getKafkaSocketRecvBufferSize());
+        properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+        ConsumeStrategy offsetResetStrategy = context.getConfig().getOffsetResetStrategy();
+        if (offsetResetStrategy == ConsumeStrategy.lastest
+                || offsetResetStrategy == ConsumeStrategy.lastest_absolutely) {
+            properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest");
+        } else if (offsetResetStrategy == ConsumeStrategy.earliest
+                || offsetResetStrategy == ConsumeStrategy.earliest_absolutely) {
+            properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        } else {
+            properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
+        }
+        properties.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG,
+                context.getConfig().getKafkaFetchSizeBytes());
+        properties.put(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG,
+                context.getConfig().getKafkaFetchWaitMs());
+        properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+        properties.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
+                "org.apache.kafka.clients.consumer.StickyAssignor");
+        properties.put(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG, 120000L);
+        this.bootstrapServers = bootstrapServers;
+        this.consumer = new KafkaConsumer<>(properties);
+    }
+
+    public class Fetcher implements Runnable {
+
+        private void commitKafkaOffset() {
+            if (consumer != null && commitOffsetMap.size() > 0) {
+                try {
+                    consumer.commitSync(commitOffsetMap);
+                    commitOffsetMap.clear();
+                    //TODO monitor commit succ
+
+                } catch (Exception e) {
+                    //TODO monitor commit fail
+                }
+            }
+        }
+
+        /**
+         * put the received msg to onFinished method
+         *
+         * @param messageRecords {@link List < MessageRecord >}
+         */
+        private void handleAndCallbackMsg(List<MessageRecord> messageRecords) {
+            long start = System.currentTimeMillis();
+            try {
+                context.getStatManager()
+                        .getStatistics(context.getConfig().getSortTaskId(),
+                                inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                        .addCallbackTimes(1);
+                context.getConfig().getCallback().onFinishedBatch(messageRecords);
+                context.getStatManager()
+                        .getStatistics(context.getConfig().getSortTaskId(),
+                                inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                        .addCallbackTimeCost(System.currentTimeMillis() - start).addCallbackDoneTimes(1);
+            } catch (Exception e) {
+                context.getStatManager()
+                        .getStatistics(context.getConfig().getSortTaskId(),
+                                inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                        .addCallbackErrorTimes(1);
+                logger.error(e.getMessage(), e);
+            }
+        }
+
+        private String getOffset(int partitionId, long offset) {
+            return partitionId + ":" + offset;
+        }
+
+        private Map<String, String> getMsgHeaders(Headers headers) {
+            Map<String, String> headerMap = new HashMap<>();
+            for (Header header : headers) {
+                headerMap.put(header.key(), new String(header.value()));
+            }
+            return headerMap;
+        }
+
+        @Override
+        public void run() {
+            boolean hasPermit;
+            while (true) {
+                hasPermit = false;
+                try {
+                    if (context.getConfig().isStopConsume() || stopConsume) {
+                        TimeUnit.MILLISECONDS.sleep(50);
+                        continue;
+                    }
+
+                    if (sleepTime > 0) {
+                        TimeUnit.MILLISECONDS.sleep(sleepTime);
+                    }
+
+                    context.acquireRequestPermit();
+                    hasPermit = true;
+                    // fetch from kafka
+                    fetchFromKafka();
+                    // commit
+                    commitKafkaOffset();
+                } catch (Exception e) {
+                    context.getStatManager()
+                            .getStatistics(context.getConfig().getSortTaskId(),
+                                    inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                            .addFetchErrorTimes(1);
+                    logger.error(e.getMessage(), e);
+                } finally {
+                    if (hasPermit) {
+                        context.releaseRequestPermit();
+                    }
+                }
+            }
+        }
+
+        private void fetchFromKafka() throws Exception {
+            context.getStatManager()
+                    .getStatistics(context.getConfig().getSortTaskId(),
+                            inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                    .addMsgCount(1).addFetchTimes(1);
+
+            long startFetchTime = System.currentTimeMillis();
+            ConsumerRecords<byte[], byte[]> records = consumer
+                    .poll(Duration.ofMillis(context.getConfig().getKafkaFetchWaitMs()));
+            context.getStatManager()
+                    .getStatistics(context.getConfig().getSortTaskId(),
+                            inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                    .addFetchTimeCost(System.currentTimeMillis() - startFetchTime);
+            if (null != records && !records.isEmpty()) {
+
+                List<MessageRecord> msgs = new ArrayList<>();
+                for (ConsumerRecord<byte[], byte[]> msg : records) {
+                    String offsetKey = getOffset(msg.partition(), msg.offset());
+                    List<InLongMessage> inLongMessages = deserializer
+                            .deserialize(context, inLongTopic, getMsgHeaders(msg.headers()), msg.value());
+                    msgs.add(new MessageRecord(inLongTopic.getTopicKey(),
+                            inLongMessages,
+                            offsetKey, System.currentTimeMillis()));
+                    context.getStatManager()
+                            .getStatistics(context.getConfig().getSortTaskId(),
+                                    inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                            .addConsumeSize(msg.value().length);
+                }
+                context.getStatManager()
+                        .getStatistics(context.getConfig().getSortTaskId(),
+                                inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                        .addMsgCount(msgs.size());
+                handleAndCallbackMsg(msgs);
+                sleepTime = 0L;
+            } else {
+                context.getStatManager()
+                        .getStatistics(context.getConfig().getSortTaskId(),
+                                inLongTopic.getInLongCluster().getClusterId(), inLongTopic.getTopic())
+                        .addEmptyFetchTimes(1);
+                emptyFetchTimes++;
+                if (emptyFetchTimes >= context.getConfig().getEmptyPollTimes()) {
+                    sleepTime = Math.min((sleepTime += context.getConfig().getEmptyPollSleepStepMs()),
+                            context.getConfig().getMaxEmptyPollSleepMs());
+                    emptyFetchTimes = 0;
+                }
+            }
+        }
+    }
+}
diff --git a/inlong-sdk/sort-sdk/src/test/java/org/apache/inlong/sdk/sort/impl/kafka/InLongKafkaFetcherImplTest.java b/inlong-sdk/sort-sdk/src/test/java/org/apache/inlong/sdk/sort/impl/kafka/InLongKafkaFetcherImplTest.java
new file mode 100644
index 0000000..a2f3c43
--- /dev/null
+++ b/inlong-sdk/sort-sdk/src/test/java/org/apache/inlong/sdk/sort/impl/kafka/InLongKafkaFetcherImplTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.sdk.sort.impl.kafka;
+
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+import org.apache.inlong.sdk.sort.api.ClientContext;
+import org.apache.inlong.sdk.sort.api.SortClientConfig;
+import org.apache.inlong.sdk.sort.entity.CacheZoneCluster;
+import org.apache.inlong.sdk.sort.entity.InLongTopic;
+import org.apache.inlong.sdk.sort.impl.ClientContextImpl;
+import org.apache.inlong.sdk.sort.stat.SortClientStateCounter;
+import org.apache.inlong.sdk.sort.stat.StatManager;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+@PowerMockIgnore("javax.management.*")
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ClientContext.class})
+public class InLongKafkaFetcherImplTest {
+
+    private ClientContext clientContext;
+    private InLongTopic inLongTopic;
+    private SortClientConfig sortClientConfig;
+    private StatManager statManager;
+
+    /**
+     * setUp
+     */
+    @Before
+    public void setUp() throws Exception {
+        System.setProperty("log4j2.disable.jmx", Boolean.TRUE.toString());
+
+        inLongTopic = new InLongTopic();
+        inLongTopic.setTopic("testTopic");
+        inLongTopic.setPartitionId(0);
+        inLongTopic.setTopicType("pulsar");
+
+        CacheZoneCluster cacheZoneCluster = new CacheZoneCluster("clusterId", "bootstraps", "token");
+        inLongTopic.setInLongCluster(cacheZoneCluster);
+        clientContext = PowerMockito.mock(ClientContextImpl.class);
+
+        sortClientConfig = PowerMockito.mock(SortClientConfig.class);
+        statManager = PowerMockito.mock(StatManager.class);
+
+        when(clientContext.getConfig()).thenReturn(sortClientConfig);
+        when(clientContext.getStatManager()).thenReturn(statManager);
+        SortClientStateCounter sortClientStateCounter = new SortClientStateCounter("sortTaskId",
+                cacheZoneCluster.getClusterId(),
+                inLongTopic.getTopic(), 0);
+        when(statManager.getStatistics(anyString(), anyString(), anyString())).thenReturn(sortClientStateCounter);
+        when(sortClientConfig.getSortTaskId()).thenReturn("sortTaskId");
+
+    }
+
+    @Test
+    public void pause() {
+        InLongKafkaFetcherImpl inLongTopicFetcher = new InLongKafkaFetcherImpl(inLongTopic, clientContext);
+        inLongTopicFetcher.pause();
+    }
+
+    @Test
+    public void resume() {
+        InLongKafkaFetcherImpl inLongTopicFetcher = new InLongKafkaFetcherImpl(inLongTopic, clientContext);
+        inLongTopicFetcher.resume();
+    }
+
+    @Test
+    public void close() {
+        InLongKafkaFetcherImpl inLongTopicFetcher = new InLongKafkaFetcherImpl(inLongTopic, clientContext);
+        boolean close = inLongTopicFetcher.close();
+        Assert.assertTrue(close);
+    }
+}
\ No newline at end of file