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] fix the beforeConsume() method earlier hit with message listener #23578

Merged
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 @@ -476,6 +476,84 @@ public void onAckTimeoutSend(Consumer<String> consumer, Set<MessageId> messageId
consumer.close();
}

@Test(dataProvider = "topicPartition")
public void testDoNotEarlierHitBeforeConsumerWithMessageListener(int partitions) throws Exception {

AtomicInteger beforeConsumeCount = new AtomicInteger(0);
PulsarClient client = PulsarClient.builder()
.serviceUrl(lookupUrl.toString())
.listenerThreads(1)
.build();

ConsumerInterceptor<String> interceptor = new ConsumerInterceptor<>() {
@Override
public void close() {
}

@Override
public Message<String> beforeConsume(Consumer<String> consumer, Message<String> message) {
beforeConsumeCount.incrementAndGet();
log.info("beforeConsume messageId: {}", message.getMessageId());
return message;
}

@Override
public void onAcknowledge(Consumer<String> consumer, MessageId messageId, Throwable cause) {
}

@Override
public void onAcknowledgeCumulative(Consumer<String> consumer, MessageId messageId, Throwable cause) {
}

@Override
public void onNegativeAcksSend(Consumer<String> consumer, Set<MessageId> messageIds) {
}

@Override
public void onAckTimeoutSend(Consumer<String> consumer, Set<MessageId> messageIds) {
}
};

final String topicName = "persistent://my-property/my-ns/my-topic";

if (partitions > 0) {
admin.topics().createPartitionedTopic(topicName, partitions);
} else {
admin.topics().createNonPartitionedTopic(topicName);
}

Consumer<String> consumer = client.newConsumer(Schema.STRING)
.topic(topicName)
.subscriptionType(SubscriptionType.Shared)
.intercept(interceptor)
.subscriptionName("my-subscription")
.messageListener((c, m) -> {
// Simulate a long processing time
try {
Thread.sleep(60000);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
})
.subscribe();

Producer<String> producer = client.newProducer(Schema.STRING)
.topic("persistent://my-property/my-ns/my-topic")
.create();

final int messages = 10;
for (int i = 0; i < messages; i++) {
producer.newMessage().value("Hello Pulsar!").send();
}
Awaitility.await().untilAsserted(() -> {
// Ensure that the interceptor is not hit before the message listener
Assert.assertEquals(beforeConsumeCount.get(), 1);
});
producer.close();
consumer.close();
client.close();
}

@Test
public void testConsumerInterceptorWithPatternTopicSubscribe() throws PulsarClientException {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1180,6 +1180,7 @@ protected void callMessageListener(Message<T> msg) {
id = msg.getMessageId();
}
unAckedMessageTracker.add(id, msg.getRedeliveryCount());
beforeConsume(msg);
listener.received(ConsumerBase.this, msg);
} catch (Throwable t) {
log.error("[{}][{}] Message listener error in processing message: {}", topic, subscription,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -542,7 +542,8 @@ protected Message<T> internalReceive(long timeout, TimeUnit unit) throws PulsarC
return null;
}
messageProcessed(message);
return beforeConsume(message);
message = listener == null ? beforeConsume(message) : message;
return message;
} catch (InterruptedException e) {
ExceptionHandler.handleInterruptedException(e);
State state = getState();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,7 @@ protected Message<T> internalReceive(long timeout, TimeUnit unit) throws PulsarC
decreaseIncomingMessageSize(message);
checkArgument(message instanceof TopicMessageImpl);
trackUnAckedMsgIfNoListener(message.getMessageId(), message.getRedeliveryCount());
message = beforeConsume(message);
message = listener == null ? beforeConsume(message) : message;
}
resumeReceivingFromPausedConsumersIfNeeded();
return message;
Expand Down
Loading