forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Issue 5585][pulsar-client] Fix producer Semaphore release error (apa…
…che#5587) ### Motivation Fixes apache#5585 This PR is try to fix the dead lock issue which mentioned in apache#5585. The core problem is the semaphore release count calculation in replicate scenario. Replicator send a batch message(from the original producer) without batcher, so that the batch message acquire 1 each batch message but release it by number of messages in the batch. This PR will handle the semaphore release in replicate scenario. ### Modifications Release only one semaphore in replicate scenario.
- Loading branch information
1 parent
006f291
commit f2f801a
Showing
2 changed files
with
157 additions
and
9 deletions.
There are no files selected for viewing
135 changes: 135 additions & 0 deletions
135
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSemaphoreTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,135 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.pulsar.client.impl; | ||
|
||
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.testng.Assert; | ||
import org.testng.annotations.AfterMethod; | ||
import org.testng.annotations.BeforeMethod; | ||
import org.testng.annotations.Test; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.ExecutionException; | ||
|
||
public class ProducerSemaphoreTest extends ProducerConsumerBase { | ||
|
||
@Override | ||
@BeforeMethod | ||
public void setup() throws Exception { | ||
super.internalSetup(); | ||
super.producerBaseSetup(); | ||
} | ||
|
||
@Override | ||
@AfterMethod | ||
public void cleanup() throws Exception { | ||
super.internalCleanup(); | ||
} | ||
|
||
@Test(timeOut = 30000) | ||
public void testProducerSemaphoreAcquireAndRelease() throws PulsarClientException, ExecutionException, InterruptedException { | ||
|
||
final int pendingQueueSize = 100; | ||
|
||
@Cleanup | ||
ProducerImpl<byte[]> producer = (ProducerImpl<byte[]>) pulsarClient.newProducer() | ||
.topic("testProducerSemaphoreAcquire") | ||
.maxPendingMessages(pendingQueueSize) | ||
.enableBatching(false) | ||
.create(); | ||
|
||
final int messages = 10; | ||
final List<CompletableFuture<MessageId>> futures = new ArrayList<>(messages); | ||
for (int i = 0; i < messages; i++) { | ||
futures.add(producer.newMessage().value(("Semaphore-test-" + i).getBytes()).sendAsync()); | ||
} | ||
|
||
FutureUtil.waitForAll(futures).get(); | ||
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize); | ||
futures.clear(); | ||
|
||
// Simulate replicator, non batching message but `numMessagesInBatch` of message metadata > 1 | ||
for (int i = 0; i < messages / 2; i++) { | ||
PulsarApi.MessageMetadata.Builder builder = PulsarApi.MessageMetadata.newBuilder(); | ||
builder.setNumMessagesInBatch(10); | ||
MessageImpl<byte[]> msg = MessageImpl.create(builder, ByteBuffer.wrap(new byte[0]), Schema.BYTES); | ||
futures.add(producer.sendAsync(msg)); | ||
} | ||
FutureUtil.waitForAll(futures).get(); | ||
|
||
// Here must ensure that the semaphore available permits is 0 | ||
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize); | ||
|
||
// Acquire 5 and not wait the send ack call back | ||
for (int i = 0; i < messages / 2; i++) { | ||
producer.newMessage().value(("Semaphore-test-" + i).getBytes()).sendAsync(); | ||
} | ||
|
||
// Here must ensure that the Semaphore a acquired 5 | ||
Assert.assertEquals(producer.getSemaphore().availablePermits(), pendingQueueSize - messages / 2); | ||
|
||
} | ||
|
||
/** | ||
* We use semaphore to limit the pending send, so we must ensure that the thread of sending message never block | ||
* at the pending message queue. If not, the dead lock might occur. Here is the related issue to describe the | ||
* dead lock happens {https://github.com/apache/pulsar/issues/5585} | ||
*/ | ||
@Test(timeOut = 30000) | ||
public void testEnsureNotBlockOnThePendingQueue() throws PulsarClientException { | ||
|
||
final int pendingQueueSize = 10; | ||
|
||
@Cleanup | ||
ProducerImpl<byte[]> producer = (ProducerImpl<byte[]>) pulsarClient.newProducer() | ||
.topic("testProducerSemaphoreAcquire") | ||
.maxPendingMessages(pendingQueueSize) | ||
.enableBatching(false) | ||
.blockIfQueueFull(true) | ||
.create(); | ||
|
||
final int messages = 20; | ||
final List<CompletableFuture<MessageId>> futures = new ArrayList<>(messages); | ||
|
||
// Simulate replicator, non batching message but `numMessagesInBatch` of message metadata > 1 | ||
for (int i = 0; i < messages; i++) { | ||
PulsarApi.MessageMetadata.Builder builder = PulsarApi.MessageMetadata.newBuilder(); | ||
builder.setNumMessagesInBatch(10); | ||
MessageImpl<byte[]> msg = MessageImpl.create(builder, ByteBuffer.wrap(new byte[0]), Schema.BYTES); | ||
futures.add(producer.sendAsync(msg)); | ||
} | ||
|
||
FutureUtil.waitForAll(futures); | ||
futures.clear(); | ||
|
||
for (int i = 0; i < messages; i++) { | ||
futures.add(producer.newMessage().value(("Semaphore-test-" + i).getBytes()).sendAsync()); | ||
} | ||
Assert.assertEquals(producer.getSemaphore().availablePermits(), 0); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters