Skip to content

Commit

Permalink
[fix][client] Copy eventTime to retry letter topic and DLQ messages (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
SantanuKar43 authored Mar 5, 2025
1 parent 0cc266d commit 8303b96
Show file tree
Hide file tree
Showing 4 changed files with 75 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
Expand Down Expand Up @@ -261,6 +262,67 @@ public void testDeadLetterTopicMessagesWithOrderingKey() throws Exception {
consumer.close();
}

@Test
public void testDeadLetterTopicMessagesWithEventTime() 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();

long testEventTime = Instant.now().toEpochMilli();
for (int i = 0; i < sendMessages; i++) {
producer.newMessage()
.eventTime(testEventTime)
.value(String.format("Hello Pulsar, eventTime: [%d]", testEventTime).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<byte[]> message = deadLetterConsumer.receive();
assertEquals(message.getEventTime(), testEventTime);
log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData()));
deadLetterConsumer.acknowledge(message);
totalInDeadLetter++;
} while (totalInDeadLetter < sendMessages);

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

public void testDeadLetterTopicWithProducerName() throws Exception {
final String topic = "persistent://my-property/my-ns/dead-letter-topic";
final String subscription = "my-subscription";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import java.time.Instant;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
Expand Down Expand Up @@ -300,6 +301,7 @@ public void testRetryTopicProperties() throws Exception {

byte[] key = "key".getBytes();
byte[] orderingKey = "orderingKey".getBytes();
long eventTime = Instant.now().toEpochMilli();

final int maxRedeliveryCount = 3;

Expand Down Expand Up @@ -333,6 +335,7 @@ public void testRetryTopicProperties() throws Exception {
.value(String.format("Hello Pulsar [%d]", i).getBytes())
.keyBytes(key)
.orderingKey(orderingKey)
.eventTime(eventTime)
.send();
originMessageIds.add(msgId.toString());
}
Expand All @@ -350,6 +353,7 @@ public void testRetryTopicProperties() throws Exception {
assertEquals(message.getKeyBytes(), key);
assertTrue(message.hasOrderingKey());
assertEquals(message.getOrderingKey(), orderingKey);
assertEquals(message.getEventTime(), eventTime);
retryMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID));
}
consumer.reconsumeLater(message, 1, TimeUnit.SECONDS);
Expand All @@ -373,6 +377,7 @@ public void testRetryTopicProperties() throws Exception {
assertEquals(message.getKeyBytes(), key);
assertTrue(message.hasOrderingKey());
assertEquals(message.getOrderingKey(), orderingKey);
assertEquals(message.getEventTime(), eventTime);
deadLetterMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID));
}
deadLetterConsumer.acknowledge(message);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -717,6 +717,7 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
.value(retryMessage.getData())
.properties(propertiesMap);
copyMessageKeysIfNeeded(message, typedMessageBuilderNew);
copyMessageEventTime(message, typedMessageBuilderNew);
typedMessageBuilderNew.sendAsync().thenAccept(msgId -> {
consumerDlqMessagesCounter.increment();

Expand Down Expand Up @@ -749,6 +750,7 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
typedMessageBuilderNew.deliverAfter(delayTime, unit);
}
copyMessageKeysIfNeeded(message, typedMessageBuilderNew);
copyMessageEventTime(message, typedMessageBuilderNew);
typedMessageBuilderNew.sendAsync()
.thenCompose(
__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null))
Expand Down Expand Up @@ -824,6 +826,11 @@ private MessageImpl<?> getMessageImpl(Message<?> message) {
return null;
}

private static void copyMessageEventTime(Message<?> message,
TypedMessageBuilder<byte[]> typedMessageBuilderNew) {
typedMessageBuilderNew.eventTime(message.getEventTime());
}

@Override
public void negativeAcknowledge(MessageId messageId) {
consumerNacksCounter.increment();
Expand Down Expand Up @@ -2242,6 +2249,7 @@ private CompletableFuture<Boolean> processPossibleToDLQ(MessageIdAdv messageId)
.value(message.getData())
.properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr));
copyMessageKeysIfNeeded(message, typedMessageBuilderNew);
copyMessageEventTime(message, typedMessageBuilderNew);
typedMessageBuilderNew.sendAsync()
.thenAccept(messageIdInDLQ -> {
possibleSendToDeadLetterTopicMessages.remove(messageId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,6 @@ public TypedMessageBuilder<T> properties(Map<String, String> properties) {

@Override
public TypedMessageBuilder<T> eventTime(long timestamp) {
checkArgument(timestamp > 0, "Invalid timestamp : '%s'", timestamp);
msgMetadata.setEventTime(timestamp);
return this;
}
Expand Down

0 comments on commit 8303b96

Please sign in to comment.