Closed BewareMyPower closed 3 years ago
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:
The design of PendingProduceQueue
has some problems, when I remove the queue and send synchronously, the offset will be continuous.
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
Persist[i]
be the time when message i
was persisted;Complete[i]
be the time when callback (MessagePublishContext#completed
) was invoked.Assuming there're two messages/records 0 and 1, each record has only 1 message. Then we have
Persist[0]
happens before Persist[1]
, which guarantees the messaging order.Persist[0]
happens before Complete[0]
.Persist[1]
happens before Complete[1]
.So there're two possible timelines (current offset is the offset of latest message, it's LEO - 1 in Kafka): |
Event | interceptor's index (current offset) | base offset |
---|---|---|---|
Persist[0] | 0 | ||
Persist[1] | 1 | ||
Complete[0] | 1 | 1 | |
Complete[1] | 1 | 1 |
or
Event | interceptor's index (current offset) | base offset |
---|---|---|
Persist[0] | 0 | |
Complete[0] | 0 | 0 |
Persist[1] | 1 | |
Complete[1] | 1 | 1 |
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.
Another fix from pulsar side is WIP, see https://github.com/apache/pulsar/pull/9257
The fix of Kafka client's send callback needs:
BrokerEntryMetadata
well.PublishContext
.PublishContext#setMetadataFromEntryData
API .The KafkaRequestHandlerTest#testProduceCallback
is flaky because this task is not completed but was closed accidentally before.
INFO io.streamnative.pulsar.handlers.kop.KafkaRequestHandlerTest - Actual offsets: [5, 1, 2, 3, 4, 5, 6, 7, 8, 9]
From the error log, we can see the first message's offset is 5. It's caused by the race condition I mentioned before.
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.