Skip to content
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 @@ -125,6 +125,65 @@ public void testDeadLetterTopicWithMessageKey() throws Exception {
consumer.close();
}

@Test
public void testDeadLetterTopicWithBinaryMessageKey() throws Exception {
final String topic = "persistent://my-property/my-ns/dead-letter-topic";

final int maxRedeliveryCount = 1;

final int sendMessages = 100;

Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic(topic)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.ackTimeout(1, TimeUnit.SECONDS)
.deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build())
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();

@Cleanup
PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0);// Creates new client connection
Consumer<byte[]> deadLetterConsumer = newPulsarClient.newConsumer(Schema.BYTES)
.topic("persistent://my-property/my-ns/dead-letter-topic-my-subscription-DLQ")
.subscriptionName("my-subscription")
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();

Producer<byte[]> producer = pulsarClient.newProducer(Schema.BYTES)
.topic(topic)
.create();

byte[] key = new byte[] {1, 2, 3, 4};
for (int i = 0; i < sendMessages; i++) {
producer.newMessage()
.keyBytes(key)
.value(String.format("Hello Pulsar [%d]", i).getBytes())
.send();
}

producer.close();

int totalReceived = 0;
do {
Message<byte[]> message = consumer.receive();
log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData()));
totalReceived++;
} while (totalReceived < sendMessages * (maxRedeliveryCount + 1));

int totalInDeadLetter = 0;
do {
Message message = deadLetterConsumer.receive();
assertEquals(message.getKeyBytes(), key);
log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData()));
deadLetterConsumer.acknowledge(message);
totalInDeadLetter++;
} while (totalInDeadLetter < sendMessages);

deadLetterConsumer.close();
consumer.close();
}

@Test(groups = "quarantine")
public void testDeadLetterTopic() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -657,6 +657,7 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
dlqProducer.newMessage(Schema.AUTO_PRODUCE_BYTES(retryMessage.getReaderSchema().get()))
.value(retryMessage.getData())
.properties(propertiesMap);
copyMessageKeyIfNeeded(message, typedMessageBuilderNew);
typedMessageBuilderNew.sendAsync().thenAccept(msgId -> {
doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null).thenAccept(v -> {
result.complete(null);
Expand All @@ -680,9 +681,7 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
if (delayTime > 0) {
typedMessageBuilderNew.deliverAfter(delayTime, unit);
}
if (message.hasKey()) {
typedMessageBuilderNew.key(message.getKey());
}
copyMessageKeyIfNeeded(message, typedMessageBuilderNew);
typedMessageBuilderNew.sendAsync()
.thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null))
.thenAccept(v -> result.complete(null))
Expand All @@ -707,6 +706,16 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
return result;
}

private static void copyMessageKeyIfNeeded(Message<?> message, TypedMessageBuilder<?> typedMessageBuilderNew) {
if (message.hasKey()) {
if (message.hasBase64EncodedKey()) {
typedMessageBuilderNew.keyBytes(message.getKeyBytes());
} else {
typedMessageBuilderNew.key(message.getKey());
}
}
}

private SortedMap<String, String> getPropertiesMap(Message<?> message,
String originMessageIdStr,
String originTopicNameStr) {
Expand Down Expand Up @@ -2033,9 +2042,7 @@ private CompletableFuture<Boolean> processPossibleToDLQ(MessageIdImpl messageId)
producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get()))
.value(message.getData())
.properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr));
if (message.hasKey()) {
typedMessageBuilderNew.key(message.getKey());
}
copyMessageKeyIfNeeded(message, typedMessageBuilderNew);
typedMessageBuilderNew.sendAsync()
.thenAccept(messageIdInDLQ -> {
possibleSendToDeadLetterTopicMessages.remove(finalMessageId);
Expand Down