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

Allows consumer retrieve the sequence id that the producer set. #4645

Merged
merged 2 commits into from
Jul 2, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -737,5 +737,69 @@ public void testOrderingOfKeyBasedBatchMessageContainer() throws PulsarClientExc
producer.close();
}

@Test(dataProvider = "containerBuilder")
private void testRetrieveSequenceIdGenerated(BatcherBuilder builder) throws Exception {

int numMsgs = 10;
final String topicName = "persistent://prop/ns-abc/testRetrieveSequenceIdGenerated-" + UUID.randomUUID();
final String subscriptionName = "sub-1";

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

Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
.batchingMaxPublishDelay(5, TimeUnit.SECONDS).batchingMaxMessages(numMsgs).enableBatching(true)
.batcherBuilder(builder)
.create();

List<CompletableFuture<MessageId>> sendFutureList = Lists.newArrayList();
for (int i = 0; i < numMsgs; i++) {
byte[] message = ("my-message-" + i).getBytes();
sendFutureList.add(producer.sendAsync(message));
}
FutureUtil.waitForAll(sendFutureList).get();

for (int i = 0; i < numMsgs; i++) {
Message<byte[]> received = consumer.receive();
Assert.assertEquals(received.getSequenceId(), i);
consumer.acknowledge(received);
}

producer.close();
consumer.close();
}

@Test(dataProvider = "containerBuilder")
private void testRetrieveSequenceIdSpecify(BatcherBuilder builder) throws Exception {

int numMsgs = 10;
final String topicName = "persistent://prop/ns-abc/testRetrieveSequenceIdSpecify-" + UUID.randomUUID();
final String subscriptionName = "sub-1";

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

Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
.batchingMaxPublishDelay(5, TimeUnit.SECONDS).batchingMaxMessages(numMsgs).enableBatching(true)
.batcherBuilder(builder)
.create();

List<CompletableFuture<MessageId>> sendFutureList = Lists.newArrayList();
for (int i = 0; i < numMsgs; i++) {
byte[] message = ("my-message-" + i).getBytes();
sendFutureList.add(producer.newMessage().sequenceId(i + 100).value(message).sendAsync());
}
FutureUtil.waitForAll(sendFutureList).get();

for (int i = 0; i < numMsgs; i++) {
Message<byte[]> received = consumer.receive();
Assert.assertEquals(received.getSequenceId(), i + 100);
consumer.acknowledge(received);
}

producer.close();
consumer.close();
}

private static final Logger LOG = LoggerFactory.getLogger(BatchMessageTest.class);
}
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,10 @@ static <T> MessageImpl<T> create(MessageMetadata.Builder msgMetadataBuilder, Byt
msgMetadataBuilder.setEventTime(singleMessageMetadata.getEventTime());
}

if (singleMessageMetadata.hasSequenceId()) {
msgMetadataBuilder.setSequenceId(singleMessageMetadata.getSequenceId());
}

this.schema = schema;
}

Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -1170,6 +1170,10 @@ public static ByteBuf serializeSingleMessageInBatchWithPayload(PulsarApi.Message
singleMessageMetadataBuilder.setEventTime(msgBuilder.getEventTime());
}

if (msgBuilder.hasSequenceId()) {
singleMessageMetadataBuilder.setSequenceId(msgBuilder.getSequenceId());
}

try {
return serializeSingleMessageInBatchWithPayload(singleMessageMetadataBuilder, payload, batchBuffer);
} finally {
Expand Down
2 changes: 2 additions & 0 deletions pulsar-common/src/main/proto/PulsarApi.proto
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,8 @@ message SingleMessageMetadata {
optional bool partition_key_b64_encoded = 6 [ default = false ];
// Specific a key to overwrite the message key which used for ordering dispatch in Key_Shared mode.
optional bytes ordering_key = 7;
// Allows consumer retrieve the sequence id that the producer set.
optional uint64 sequence_id = 8;
}

enum ServerError {
Expand Down