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 client side memory leak when call MessageImpl.create and fix imprecise client-side metrics: pendingMessagesUpDownCounter, pendingBytesUpDownCounter, latencyHistogram #22393

Merged
merged 9 commits into from
Apr 7, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -99,11 +99,13 @@
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl;
import org.apache.pulsar.client.impl.PartitionedProducerImpl;
import org.apache.pulsar.client.impl.ProducerBase;
import org.apache.pulsar.client.impl.ProducerImpl;
import org.apache.pulsar.client.impl.TopicMessageImpl;
import org.apache.pulsar.client.impl.TypedMessageBuilderImpl;
import org.apache.pulsar.client.impl.crypto.MessageCryptoBc;
import org.apache.pulsar.client.impl.schema.writer.AvroWriter;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.api.EncryptionContext;
import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey;
import org.apache.pulsar.common.api.proto.MessageMetadata;
Expand Down Expand Up @@ -4692,4 +4694,48 @@ public void flush(ChannelHandlerContext ctx) throws Exception {
consumer.close();
admin.topics().delete(topic, false);
}

@Test
public void testPublishWithCreateMessageManually() throws Exception {
final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp");
admin.topics().createNonPartitionedTopic(topic);
ProducerBase producerBase = (ProducerBase) pulsarClient.newProducer().topic(topic).enableBatching(false).create();

// Create message payload, refCnf = 1 now.
ByteBuf payload1 = PulsarByteBufAllocator.DEFAULT.heapBuffer(1);
payload1.writeByte(1);
ByteBuf payload2 = PulsarByteBufAllocator.DEFAULT.heapBuffer(1);
payload1.writeByte(2);
log.info("payload_1.refCnf 1st: {}", payload1.refCnt());
log.info("payload_2.refCnf 1st: {}", payload2.refCnt());
// refCnf = 2 now.
payload1.retain();
payload2.retain();
log.info("payload_1.refCnf 2nd: {}", payload1.refCnt());
log.info("payload_2.refCnf 2nd: {}", payload2.refCnt());
MessageMetadata messageMetadata = new MessageMetadata();
messageMetadata.setUncompressedSize(1);

// Publish message.
// Note: "ProducerBase.sendAsync" is not equals to "Producer.sendAsync".
MessageImpl<byte[]> message1 = MessageImpl.create(topic, null, messageMetadata, payload1, Optional.empty(),
null, Schema.BYTES, 0, true, 0);
MessageImpl<byte[]> message2 = MessageImpl.create(topic, null, messageMetadata, payload2, Optional.empty(),
null, Schema.BYTES, 0, true, 0);
// Release ByteBuf the first time, refCnf = 1 now.
producerBase.sendAsync(message1).thenAccept(ignore_ -> message1.release());
producerBase.sendAsync(message2).thenAccept(ignore_ -> message2.release()).join();

// Assert payload's refCnf.
log.info("payload_1.refCnf 3rd: {}", payload1.refCnt());
log.info("payload_2.refCnf 3rd: {}", payload2.refCnt());
assertEquals(payload1.refCnt(), 1);
assertEquals(payload2.refCnt(), 1);

// cleanup.
payload1.release();
payload2.release();
producerBase.close();
admin.topics().delete(topic, false);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -404,6 +404,7 @@ public void sendComplete(Exception e) {
pendingMessagesUpDownCounter.decrement();
pendingBytesUpDownCounter.subtract(msgSize);

ByteBuf payloadInCurrentMsg = interceptorMessage.getDataBuffer();
try {
if (e != null) {
latencyHistogram.recordFailure(latencyNanos);
Expand All @@ -418,16 +419,17 @@ public void sendComplete(Exception e) {
stats.incrementNumAcksReceived(latencyNanos);
}
} finally {
interceptorMessage.getDataBuffer().release();
ReferenceCountUtil.safeRelease(payloadInCurrentMsg);
codelipenghui marked this conversation as resolved.
Show resolved Hide resolved
}

while (nextCallback != null) {
SendCallback sendCallback = nextCallback;
MessageImpl<?> msg = nextMsg;
// Retain the buffer used by interceptors callback to get message. Buffer will release after
// complete interceptors.
ByteBuf payloadInNextMsg = msg.getDataBuffer();
payloadInNextMsg.retain();
try {
msg.getDataBuffer().retain();
if (e != null) {
stats.incrementSendFailed();
onSendAcknowledgement(msg, null, e);
Expand All @@ -440,7 +442,7 @@ public void sendComplete(Exception e) {
nextMsg = nextCallback.getNextMessage();
nextCallback = nextCallback.getNextSendCallback();
} finally {
msg.getDataBuffer().release();
ReferenceCountUtil.safeRelease(payloadInNextMsg);
}
}
}
Expand Down
Loading