Skip to content

Commit

Permalink
[fix][client]Fix deadlock issue of consumer while using multiple IO t…
Browse files Browse the repository at this point in the history
…hreads (apache#20669)
  • Loading branch information
codelipenghui authored and Technoboy- committed Jun 29, 2023
1 parent 526e216 commit 39cf8d7
Show file tree
Hide file tree
Showing 2 changed files with 41 additions and 17 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.testng.Assert.assertTrue;
import com.google.common.collect.Lists;
import java.util.HashMap;
import java.util.Map;
Expand All @@ -36,8 +37,10 @@
import lombok.Cleanup;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.impl.ClientBuilderImpl;
import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.common.naming.TopicName;
import org.mockito.AdditionalAnswers;
import org.mockito.Mockito;
import org.slf4j.Logger;
Expand Down Expand Up @@ -65,6 +68,11 @@ protected void cleanup() throws Exception {
super.internalCleanup();
}

@Override
protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) {
clientBuilder.ioThreads(4).connectionsPerBroker(4);
}

// test that reproduces the issue https://github.com/apache/pulsar/issues/12024
// where closing the consumer leads to an endless receive loop
@Test
Expand Down Expand Up @@ -192,4 +200,19 @@ public void testShouldMaintainOrderForIndividualTopicInMultiTopicsConsumer()
}
Assert.assertEquals(numPartitions * numMessages, receivedCount);
}

@Test(invocationCount = 10, timeOut = 30000)
public void testMultipleIOThreads() throws PulsarAdminException, PulsarClientException {
final String topic = TopicName.get(newTopicName()).toString();
final int numPartitions = 100;
admin.topics().createPartitionedTopic(topic, numPartitions);
for (int i = 0; i < 100; i++) {
admin.topics().createNonPartitionedTopic(topic + "-" + i);
}
@Cleanup
final Consumer<Integer> consumer = pulsarClient.newConsumer(Schema.INT32).topicsPattern(topic + ".*")
.subscriptionName("sub").subscribe();
assertTrue(consumer instanceof MultiTopicsConsumerImpl);
assertTrue(consumer.isConnected());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ public class MultiTopicsConsumerImpl<T> extends ConsumerBase<T> {
// sum of topicPartitions, simple topic has 1, partitioned topic equals to partition number.
AtomicInteger allTopicPartitionsNumber;

private boolean paused = false;
private volatile boolean paused = false;
private final Object pauseMutex = new Object();
// timeout related to auto check and subscribe partition increasement
private volatile Timeout partitionsAutoUpdateTimeout = null;
Expand Down Expand Up @@ -1052,25 +1052,26 @@ private void doSubscribeTopicPartitions(Schema<T> schema,

CompletableFuture<Consumer<T>> subFuture = new CompletableFuture<>();

consumers.compute(topicName, (key, existingValue) -> {
if (existingValue != null) {
String errorMessage = String.format("[%s] Failed to subscribe for topic [%s] in topics consumer. "
+ "Topic is already being subscribed for in other thread.", topic, topicName);
log.warn(errorMessage);
subscribeResult.completeExceptionally(new PulsarClientException(errorMessage));
return existingValue;
} else {
ConsumerImpl<T> newConsumer = createInternalConsumer(internalConfig, topicName,
-1, subFuture, createIfDoesNotExist, schema);
synchronized (pauseMutex) {
synchronized (pauseMutex) {
consumers.compute(topicName, (key, existingValue) -> {
if (existingValue != null) {
String errorMessage =
String.format("[%s] Failed to subscribe for topic [%s] in topics consumer. "
+ "Topic is already being subscribed for in other thread.", topic, topicName);
log.warn(errorMessage);
subscribeResult.completeExceptionally(new PulsarClientException(errorMessage));
return existingValue;
} else {
internalConfig.setStartPaused(paused);
ConsumerImpl<T> newConsumer = createInternalConsumer(internalConfig, topicName,
-1, subFuture, createIfDoesNotExist, schema);
if (paused) {
newConsumer.pause();
}
return newConsumer;
}
return newConsumer;
}
});

});
}
futureList = Collections.singletonList(subFuture);
}

Expand Down Expand Up @@ -1394,7 +1395,7 @@ private CompletableFuture<Void> subscribeIncreasedTopicPartitions(String topicNa
}
if (log.isDebugEnabled()) {
log.debug("[{}] create consumer {} for partitionName: {}",
topicName, newConsumer.getTopic(), partitionName);
topicName, newConsumer.getTopic(), partitionName);
}
return subFuture;
})
Expand Down

0 comments on commit 39cf8d7

Please sign in to comment.