Skip to content

Commit 43a8436

Browse files
Nicklee007nicklixinyang
and
nicklixinyang
authored
Fix semaphore release duplicated in ProducerImpl (#16972)
Co-authored-by: nicklixinyang <[email protected]>
1 parent 9abab5b commit 43a8436

File tree

2 files changed

+43
-3
lines changed

2 files changed

+43
-3
lines changed

pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSemaphoreTest.java

+43
Original file line numberDiff line numberDiff line change
@@ -18,15 +18,18 @@
1818
*/
1919
package org.apache.pulsar.client.impl;
2020

21+
import static org.mockito.ArgumentMatchers.any;
2122
import java.lang.reflect.Field;
2223
import java.nio.charset.StandardCharsets;
24+
import java.util.concurrent.TimeUnit;
2325
import lombok.Cleanup;
2426
import org.apache.pulsar.client.api.MessageId;
2527
import org.apache.pulsar.client.api.ProducerConsumerBase;
2628
import org.apache.pulsar.client.api.PulsarClientException;
2729
import org.apache.pulsar.client.api.Schema;
2830
import org.apache.pulsar.common.api.proto.PulsarApi;
2931
import org.apache.pulsar.common.util.FutureUtil;
32+
import org.mockito.Mockito;
3033
import org.testng.Assert;
3134
import org.testng.annotations.AfterMethod;
3235
import org.testng.annotations.BeforeMethod;
@@ -225,4 +228,44 @@ public void testEnsureNotBlockOnThePendingQueue() throws Exception {
225228
FutureUtil.waitForAll(futures).get();
226229
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize);
227230
}
231+
232+
@Test(timeOut = 10_000)
233+
public void testBatchMessageSendTimeoutProducerSemaphoreRelease() throws Exception {
234+
final int pendingQueueSize = 10;
235+
@Cleanup
236+
ProducerImpl<byte[]> producer =
237+
(ProducerImpl<byte[]>) pulsarClient.newProducer()
238+
.topic("testProducerSemaphoreRelease")
239+
.sendTimeout(2, TimeUnit.SECONDS)
240+
.maxPendingMessages(pendingQueueSize)
241+
.enableBatching(true)
242+
.batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS)
243+
.batchingMaxBytes(15)
244+
.create();
245+
this.stopBroker();
246+
try {
247+
ProducerImpl<byte[]> spyProducer = Mockito.spy(producer);
248+
// Make the pendingMessages not empty
249+
spyProducer.newMessage().value("semaphore-test".getBytes(StandardCharsets.UTF_8)).sendAsync();
250+
spyProducer.newMessage().value("semaphore-test".getBytes(StandardCharsets.UTF_8)).sendAsync();
251+
252+
Field batchMessageContainerField = ProducerImpl.class.getDeclaredField("batchMessageContainer");
253+
batchMessageContainerField.setAccessible(true);
254+
BatchMessageContainerImpl batchMessageContainer =
255+
(BatchMessageContainerImpl) batchMessageContainerField.get(spyProducer);
256+
batchMessageContainer.setProducer(spyProducer);
257+
Mockito.doThrow(new PulsarClientException.CryptoException("crypto error")).when(spyProducer)
258+
.encryptMessage(any(), any());
259+
260+
try {
261+
spyProducer.newMessage().value("memory-test".getBytes(StandardCharsets.UTF_8)).sendAsync().get();
262+
} catch (Exception e) {
263+
throw PulsarClientException.unwrap(e);
264+
}
265+
266+
throw new IllegalStateException("can not reach here");
267+
} catch (PulsarClientException.TimeoutException ex) {
268+
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize);
269+
}
270+
}
228271
}

pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java

-3
Original file line numberDiff line numberDiff line change
@@ -1672,10 +1672,7 @@ private void batchMessageAndSend() {
16721672
for (OpSendMsg opSendMsg : opSendMsgs) {
16731673
processOpSendMsg(opSendMsg);
16741674
}
1675-
} catch (PulsarClientException e) {
1676-
semaphore.release(batchMessageContainer.getNumMessagesInBatch());
16771675
} catch (Throwable t) {
1678-
semaphore.release(batchMessageContainer.getNumMessagesInBatch());
16791676
log.warn("[{}] [{}] error while create opSendMsg by batch message container", topic, producerName, t);
16801677
}
16811678
}

0 commit comments

Comments
 (0)