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] Cannot access message data inside ProducerInterceptor#onSendAcknowledgement #23791

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 @@ -21,10 +21,12 @@
import com.google.common.collect.Sets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -216,6 +218,48 @@ public void onSendAcknowledgement(Producer<String> producer, Message<String> mes
producer.close();
}

@Test
public void testProducerInterceptorAccessMessageData() throws PulsarClientException {
List<String> messageDataInBeforeSend = Collections.synchronizedList(new ArrayList<>());
List<String> messageDataOnSendAcknowledgement = Collections.synchronizedList(new ArrayList<>());
ProducerInterceptor<String> interceptor = new ProducerInterceptor<>() {
@Override
public void close() {
}

@Override
public Message<String> beforeSend(Producer<String> producer, Message<String> message) {
messageDataInBeforeSend.add(new String(message.getData()));
return message;
}

@Override
public void onSendAcknowledgement(Producer<String> producer, Message<String> message, MessageId msgId,
Throwable exception) {
messageDataOnSendAcknowledgement.add(new String(message.getData()));
}
};
@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.topic("persistent://my-property/my-ns/my-topic")
.intercept(interceptor)
.create();

final String messageValue = UUID.randomUUID().toString();
try {
producer.newMessage().value(messageValue).send();
} catch (Exception ignore) {
}
Assert.assertEquals(messageDataInBeforeSend.size(), 1,
"Message data should be available in beforeSend");
Assert.assertEquals(messageDataInBeforeSend.get(0), messageValue,
"Message data should be available in beforeSend");
Assert.assertEquals(messageDataOnSendAcknowledgement.size(), 1,
"Message data should be available in onSendAcknowledgement");
Assert.assertEquals(messageDataOnSendAcknowledgement.get(0), messageValue,
"Message data should be available in onSendAcknowledgement");
}

@Test
public void testConsumerInterceptorWithErrors() throws PulsarClientException {
ConsumerInterceptor<String> interceptor = new ConsumerInterceptor<String>() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -436,20 +436,22 @@ private void onSendComplete(Throwable e, SendCallback sendCallback, MessageImpl<
if (payload == null) {
log.error("[{}] [{}] Payload is null when calling onSendComplete, which is not expected.",
topic, producerName);
} else {
ReferenceCountUtil.safeRelease(payload);
}
if (e != null) {
latencyHistogram.recordFailure(latencyNanos);
stats.incrementSendFailed();
onSendAcknowledgement(msg, null, e);
sendCallback.getFuture().completeExceptionally(e);
} else {
latencyHistogram.recordSuccess(latencyNanos);
publishedBytesCounter.add(msgSize);
stats.incrementNumAcksReceived(latencyNanos);
onSendAcknowledgement(msg, msg.getMessageId(), null);
sendCallback.getFuture().complete(msg.getMessageId());
try {
if (e != null) {
latencyHistogram.recordFailure(latencyNanos);
stats.incrementSendFailed();
onSendAcknowledgement(msg, null, e);
sendCallback.getFuture().completeExceptionally(e);
} else {
latencyHistogram.recordSuccess(latencyNanos);
publishedBytesCounter.add(msgSize);
stats.incrementNumAcksReceived(latencyNanos);
onSendAcknowledgement(msg, msg.getMessageId(), null);
sendCallback.getFuture().complete(msg.getMessageId());
}
} finally {
ReferenceCountUtil.safeRelease(payload);
thetumbled marked this conversation as resolved.
Show resolved Hide resolved
}
}

Expand Down
Loading