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