Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix][client] moving get sequenceId into the sync code segment #17836

Merged
merged 14 commits into from
Dec 13, 2022
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import lombok.extern.slf4j.Slf4j;
Expand Down Expand Up @@ -372,4 +375,51 @@ public void testKeyBasedBatchingOrder() throws Exception {
consumer.close();
producer.close();
}

@Test
public void testUpdateSequenceIdInSyncCodeSegment() throws Exception {
final String topic = "persistent://my-property/my-ns/testUpdateSequenceIdInSyncCodeSegment";
int totalMessage = 200;
int threadSize = 5;
String topicName = "subscription";
ExecutorService executorService = Executors.newFixedThreadPool(threadSize);
conf.setBrokerDeduplicationEnabled(true);

//build producer/consumer
Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topic)
.producerName("producer")
.sendTimeout(0, TimeUnit.SECONDS)
.create();

Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topic)
.subscriptionType(SubscriptionType.Exclusive)
.subscriptionName(topicName)
.subscribe();

CountDownLatch countDownLatch = new CountDownLatch(threadSize);
//Send messages in multiple-thread
for (int i = 0; i < threadSize; i++) {
executorService.submit(() -> {
try {
for (int j = 0; j < totalMessage; j++) {
//The message will be sent with out-of-order sequence ID.
producer.newMessage().sendAsync();
}
} catch (Exception e) {
log.error("Failed to send/ack messages with transaction.", e);
} finally {
countDownLatch.countDown();
}
});
}
//wait the all send op is executed and store its futures in the arraylist.
countDownLatch.await();

for (int i = 0; i < threadSize * totalMessage; i++) {
Message<byte[]> msg = consumer.receive(5, TimeUnit.SECONDS);
assertNotNull(msg);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -485,9 +485,6 @@ public void sendAsync(Message<?> message, SendCallback callback) {
return;
}

// Update the message metadata before computing the payload chunk size to avoid a large message cannot be split
// into chunks.
final long sequenceId = updateMessageMetadata(msgMetadata, uncompressedSize);

// send in chunks
int totalChunks;
Expand Down Expand Up @@ -527,7 +524,6 @@ public void sendAsync(Message<?> message, SendCallback callback) {

try {
int readStartIndex = 0;
String uuid = totalChunks > 1 ? String.format("%s-%d", producerName, sequenceId) : null;
ChunkedMessageCtx chunkedMessageCtx = totalChunks > 1 ? ChunkedMessageCtx.get(totalChunks) : null;
byte[] schemaVersion = totalChunks > 1 && msg.getMessageBuilder().hasSchemaVersion()
? msg.getMessageBuilder().getSchemaVersion() : null;
Expand All @@ -553,6 +549,11 @@ public void sendAsync(Message<?> message, SendCallback callback) {
return;
}
synchronized (this) {
// Update the message metadata before computing the payload chunk size
// to avoid a large message cannot be split into chunks.
final long sequenceId = updateMessageMetadata(msgMetadata, uncompressedSize);
String uuid = totalChunks > 1 ? String.format("%s-%d", producerName, sequenceId) : null;

serializeAndSendMessage(msg, payload, sequenceId, uuid, chunkId, totalChunks,
readStartIndex, payloadChunkSize, compressedPayload, compressed,
compressedPayload.readableBytes(), callback, chunkedMessageCtx);
Expand Down