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 memory limit currentUsage leak and semaphore release duplicated in ProducerImpl #16837

Merged
merged 5 commits into from
Jul 30, 2022
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,35 @@ public void testProducerTimeoutMemoryRelease() throws Exception {

}

@Test(timeOut = 10_000)
public void testProducerBatchSendTimeoutMemoryRelease() throws Exception {
initClientWithMemoryLimit();
@Cleanup
ProducerImpl<byte[]> producer = (ProducerImpl<byte[]>) pulsarClient.newProducer()
.topic("testProducerMemoryLimit")
.sendTimeout(5, TimeUnit.SECONDS)
.maxPendingMessages(0)
.enableBatching(true)
.batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS)
.batchingMaxBytes(12)
.create();
this.stopBroker();
try {
producer.newMessage().value("memory-test".getBytes(StandardCharsets.UTF_8)).sendAsync();
try {
producer.newMessage().value("memory-test".getBytes(StandardCharsets.UTF_8)).sendAsync().get();
Nicklee007 marked this conversation as resolved.
Show resolved Hide resolved
} catch (Exception e) {
throw PulsarClientException.unwrap(e);
}

throw new IllegalStateException("can not reach here");
} catch (PulsarClientException.TimeoutException ex) {
PulsarClientImpl clientImpl = (PulsarClientImpl) this.pulsarClient;
final MemoryLimitController memoryLimitController = clientImpl.getMemoryLimitController();
Assert.assertEquals(memoryLimitController.currentUsage(), 0);
}
}

@Test(timeOut = 10_000)
public void testProducerCloseMemoryRelease() throws Exception {
initClientWithMemoryLimit();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,10 @@
*/
package org.apache.pulsar.client.impl;

import static org.mockito.ArgumentMatchers.any;
import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.ProducerConsumerBase;
Expand All @@ -33,7 +37,6 @@
import org.testng.annotations.Test;

import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -229,4 +232,35 @@ public void testEnsureNotBlockOnThePendingQueue() throws Exception {
Assert.assertEquals(producer.getSemaphore().get().availablePermits(), pendingQueueSize);
Assert.assertFalse(producer.isErrorStat());
}

@Test(timeOut = 10_000)
public void testBatchMessageSendTimeoutProducerSemaphoreRelease() throws Exception {
final int pendingQueueSize = 10;
@Cleanup
ProducerImpl<byte[]> producer =
(ProducerImpl<byte[]>) pulsarClient.newProducer()
.topic("testProducerSemaphoreRelease")
.sendTimeout(5, TimeUnit.SECONDS)
.maxPendingMessages(pendingQueueSize)
.enableBatching(true)
.batchingMaxPublishDelay(500, TimeUnit.MILLISECONDS)
.batchingMaxBytes(12)
.create();
this.stopBroker();
try {
ProducerImpl<byte[]> spyProducer = Mockito.spy(producer);
Mockito.doThrow(new PulsarClientException.CryptoException("crypto error")).when(spyProducer)
.encryptMessage(any(),any());

Field batchMessageContainerField = ProducerImpl.class.getDeclaredField("batchMessageContainer");
batchMessageContainerField.setAccessible(true);
BatchMessageContainerImpl batchMessageContainer = (BatchMessageContainerImpl) batchMessageContainerField.get(spyProducer);
batchMessageContainer.setProducer(spyProducer);
spyProducer.send("semaphore-test".getBytes(StandardCharsets.UTF_8));

throw new IllegalStateException("can not reach here");
} catch (PulsarClientException.TimeoutException ex) {
Assert.assertEquals(producer.getSemaphore().get().availablePermits(), 10);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2029,8 +2029,10 @@ private void failPendingBatchMessages(PulsarClientException ex) {
return;
}
final int numMessagesInBatch = batchMessageContainer.getNumMessagesInBatch();
final long currentBatchSize = batchMessageContainer.getCurrentBatchSize();
batchMessageContainer.discard(ex);
semaphoreRelease(numMessagesInBatch);
client.getMemoryLimitController().releaseMemory(currentBatchSize);
}

@Override
Expand Down Expand Up @@ -2113,10 +2115,7 @@ private void batchMessageAndSend(boolean shouldScheduleNextBatchFlush) {
for (OpSendMsg opSendMsg : opSendMsgs) {
processOpSendMsg(opSendMsg);
}
} catch (PulsarClientException e) {
semaphoreRelease(batchMessageContainer.getNumMessagesInBatch());
} catch (Throwable t) {
semaphoreRelease(batchMessageContainer.getNumMessagesInBatch());
log.warn("[{}] [{}] error while create opSendMsg by batch message container", topic, producerName, t);
} finally {
if (shouldScheduleNextBatchFlush) {
Expand Down