-
Notifications
You must be signed in to change notification settings - Fork 128
[BUG] Offset in send callback is not continuous when batching is enabled #332
Comments
this is a little strange, how many entries(batched messages) are there in bk? it would be helpful to get the offset metadata from entry. |
It looks like there are two bugs:
diff --git a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaRequestHandler.java b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaRequestHandler.java
index 57ca22e..c7f63f8 100644
--- a/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaRequestHandler.java
+++ b/kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaRequestHandler.java
@@ -48,6 +48,7 @@ import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
@@ -602,10 +603,22 @@ public class KafkaRequestHandler extends KafkaCommandDecoder {
String fullPartitionName = KopTopic.toString(topicPartition);
PendingProduce pendingProduce = new PendingProduce(partitionResponse, topicManager, fullPartitionName,
entryFormatter, records, executor);
+ CountDownLatch latch = new CountDownLatch(1);
+ pendingProduce.whenComplete(() -> {
+ pendingProduce.publishMessages();
+ latch.countDown();
+ });
+ try {
+ latch.await();
+ } catch (InterruptedException ignored) {
+ // TODO:
+ }
+ /*
PendingProduceQueue queue =
pendingProduceQueueMap.computeIfAbsent(topicPartition, ignored -> new PendingProduceQueue());
queue.add(pendingProduce);
pendingProduce.whenComplete(queue::sendCompletedProduces);
+ */
}
CompletableFuture.allOf(responsesFutures.values().toArray(new CompletableFuture<?>[responsesSize]))
|
For what I mentioned before, the second bug is easy to fix, but the first bug needs some refactor because it's caused by race condition. Let
Assuming there're two messages/records 0 and 1, each record has only 1 message. Then we have
So there're two possible timelines (current offset is the offset of latest message, it's
or
We can see for the first case, the callback of message 0 retrieves the wrong current offset which belongs to message 1, because the interceptor has already updated twice. |
This PR is a partial fix of #332. Before this PR, the MessagePublishContext completed with the current offset, which is the latest offset. Then PartitionResponse will be filled with the offset. However, Kafka producer treat the PartitionResponse's offset as the base offset. For example, before this PR, when Kafka producer sends a batch with 3 single messages to a new topic, the offsets in send callback will be (2, 3, 4) before this PR. After this PR, the offsets in send callback will be (0, 1, 2).
Another fix from pulsar side is WIP, see apache/pulsar#9257 |
The fix of Kafka client's send callback needs:
|
The
From the error log, we can see the first message's offset is 5. It's caused by the race condition I mentioned before. |
Fixes #332 The offset in produce callback may be not accurate when the messages are sent in batch, see #332 (comment) for detail explanation. Since apache/pulsar#9257 introduced a new API that supports fetching some metadata from the entry data, we can use this API to get the accurate offset.
Describe the bug
#296 implemented continuous offset, but in producer's send callback, the offset is not continuous when batching is enabled.
To Reproduce
Modify
KafkaRequestHandlerTest#testProduceCallbck
to following code and rerun.The output:
We can see the offset is not continuous, even not monodic increasing. And the first offset is not zero.
However, if we use synchronous send, i.e.
producer.send(/* ... */).get()
, the first offset will be 0, and the offset will be continuous.Expected behavior
The offset in producer's send callback is continuous.
The text was updated successfully, but these errors were encountered: