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 semaphore and memory leak when chunks failed to enqueue #13454

Merged
Show file tree
Hide file tree
Changes from 3 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 @@ -20,24 +20,28 @@

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.ClientBuilder;
import org.apache.pulsar.client.api.CompressionType;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
Expand All @@ -47,6 +51,7 @@
import org.apache.pulsar.client.api.ProducerBuilder;
import org.apache.pulsar.client.api.ProducerConsumerBase;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.SizeUnit;
import org.apache.pulsar.client.impl.MessageImpl.SchemaState;
import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
import org.apache.pulsar.common.api.proto.MessageMetadata;
Expand Down Expand Up @@ -369,6 +374,43 @@ public void testExpireIncompleteChunkMessage() throws Exception{
producer = null; // clean reference of mocked producer
}

@Test
public void testChunksEnqueueFailed() throws Exception {
final String topicName = "persistent://my-property/my-ns/test-chunks-enqueue-failed";
log.info("-- Starting {} test --", methodName);
this.conf.setMaxMessageSize(5);

final MemoryLimitController controller = ((PulsarClientImpl) pulsarClient).getMemoryLimitController();
assertEquals(controller.currentUsage(), 0);

final int maxPendingMessages = 10;

@Cleanup
Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topicName)
.maxPendingMessages(maxPendingMessages)
.enableChunking(true)
.enableBatching(false)
.create();
assertTrue(producer instanceof ProducerImpl);
Semaphore semaphore = ((ProducerImpl<byte[]>) producer).getSemaphore().orElse(null);
assertNotNull(semaphore);
assertEquals(semaphore.availablePermits(), maxPendingMessages);
producer.send(createMessagePayload(1).getBytes());
try {
producer.send(createMessagePayload(100).getBytes(StandardCharsets.UTF_8));
BewareMyPower marked this conversation as resolved.
Show resolved Hide resolved
} catch (PulsarClientException e) {
assertTrue(e instanceof PulsarClientException.ProducerQueueIsFullError);
assertEquals(controller.currentUsage(), 0);
assertEquals(semaphore.availablePermits(), maxPendingMessages);
}
}

@Override
protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) {
clientBuilder.memoryLimit(10000L, SizeUnit.BYTES);
}

private String createMessagePayload(int size) {
StringBuilder str = new StringBuilder();
Random rand = new Random();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -467,6 +467,8 @@ public void sendAsync(Message<?> message, SendCallback callback) {
// chunked message also sent individually so, try to acquire send-permits
for (int i = 0; i < (totalChunks - 1); i++) {
if (!canEnqueueRequest(callback, message.getSequenceId(), 0 /* The memory was already reserved */)) {
client.getMemoryLimitController().releaseMemory(uncompressedSize);
semaphoreRelease(i + 1);
return;
}
}
Expand Down