Skip to content

Commit

Permalink
Fix semaphore release duplicated in ProducerImpl (#16972)
Browse files Browse the repository at this point in the history
Co-authored-by: nicklixinyang <nicklixinyang@didiglobal.com>
  • Loading branch information
Nicklee007 and nicklixinyang committed Aug 8, 2022
1 parent 9abab5b commit 43a8436
Show file tree
Hide file tree
Showing 2 changed files with 43 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,18 @@
*/
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;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.util.FutureUtil;
import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
Expand Down Expand Up @@ -225,4 +228,44 @@ public void testEnsureNotBlockOnThePendingQueue() throws Exception {
FutureUtil.waitForAll(futures).get();
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize);
}

@Test(timeOut = 10_000)
public void testBatchMessageSendTimeoutProducerSemaphoreRelease() throws Exception {
final int pendingQueueSize = 10;
@Cleanup
ProducerImpl<byte[]> producer =
(ProducerImpl<byte[]>) pulsarClient.newProducer()
.topic("testProducerSemaphoreRelease")
.sendTimeout(2, TimeUnit.SECONDS)
.maxPendingMessages(pendingQueueSize)
.enableBatching(true)
.batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS)
.batchingMaxBytes(15)
.create();
this.stopBroker();
try {
ProducerImpl<byte[]> spyProducer = Mockito.spy(producer);
// Make the pendingMessages not empty
spyProducer.newMessage().value("semaphore-test".getBytes(StandardCharsets.UTF_8)).sendAsync();
spyProducer.newMessage().value("semaphore-test".getBytes(StandardCharsets.UTF_8)).sendAsync();

Field batchMessageContainerField = ProducerImpl.class.getDeclaredField("batchMessageContainer");
batchMessageContainerField.setAccessible(true);
BatchMessageContainerImpl batchMessageContainer =
(BatchMessageContainerImpl) batchMessageContainerField.get(spyProducer);
batchMessageContainer.setProducer(spyProducer);
Mockito.doThrow(new PulsarClientException.CryptoException("crypto error")).when(spyProducer)
.encryptMessage(any(), any());

try {
spyProducer.newMessage().value("memory-test".getBytes(StandardCharsets.UTF_8)).sendAsync().get();
} catch (Exception e) {
throw PulsarClientException.unwrap(e);
}

throw new IllegalStateException("can not reach here");
} catch (PulsarClientException.TimeoutException ex) {
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1672,10 +1672,7 @@ private void batchMessageAndSend() {
for (OpSendMsg opSendMsg : opSendMsgs) {
processOpSendMsg(opSendMsg);
}
} catch (PulsarClientException e) {
semaphore.release(batchMessageContainer.getNumMessagesInBatch());
} catch (Throwable t) {
semaphore.release(batchMessageContainer.getNumMessagesInBatch());
log.warn("[{}] [{}] error while create opSendMsg by batch message container", topic, producerName, t);
}
}
Expand Down

0 comments on commit 43a8436

Please sign in to comment.