Skip to content

Commit 1439529

Browse files
[fix][client] Fix concurrent lookup with properties might have different results (#23260)
1 parent 246647f commit 1439529

File tree

2 files changed

+55
-8
lines changed

2 files changed

+55
-8
lines changed

pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,23 +19,30 @@
1919
package org.apache.pulsar.client.api;
2020

2121
import java.net.InetSocketAddress;
22+
import java.util.ArrayList;
2223
import java.util.Collections;
24+
import java.util.List;
2325
import java.util.Map;
2426
import java.util.Optional;
2527
import java.util.Properties;
2628
import java.util.Set;
2729
import java.util.concurrent.CompletableFuture;
2830
import java.util.stream.Collectors;
31+
import java.util.stream.IntStream;
2932
import lombok.Cleanup;
3033
import lombok.extern.slf4j.Slf4j;
3134
import org.apache.pulsar.broker.MultiBrokerBaseTest;
3235
import org.apache.pulsar.broker.ServiceConfiguration;
3336
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;
37+
import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
3438
import org.apache.pulsar.broker.namespace.LookupOptions;
39+
import org.apache.pulsar.client.impl.LookupTopicResult;
3540
import org.apache.pulsar.client.impl.PartitionedProducerImpl;
3641
import org.apache.pulsar.client.impl.ProducerImpl;
3742
import org.apache.pulsar.client.impl.PulsarClientImpl;
3843
import org.apache.pulsar.common.naming.ServiceUnitId;
44+
import org.apache.pulsar.common.naming.TopicName;
45+
import org.apache.pulsar.common.util.FutureUtil;
3946
import org.testng.Assert;
4047
import org.testng.annotations.Test;
4148

@@ -72,6 +79,7 @@ private static ServiceConfiguration addCustomConfigs(ServiceConfiguration config
7279

7380
@Test
7481
public void testLookupProperty() throws Exception {
82+
admin.namespaces().unload("public/default");
7583
final var topic = "test-lookup-property";
7684
admin.topics().createPartitionedTopic(topic, 16);
7785
@Cleanup final var client = (PulsarClientImpl) PulsarClient.builder()
@@ -89,7 +97,35 @@ public void testLookupProperty() throws Exception {
8997
Assert.assertEquals(port, additionalBrokers.get(0).getBrokerListenPort().orElseThrow());
9098
}
9199

100+
@Test
101+
public void testConcurrentLookupProperties() throws Exception {
102+
@Cleanup final var client = (PulsarClientImpl) PulsarClient.builder()
103+
.serviceUrl(pulsar.getBrokerServiceUrl())
104+
.build();
105+
final var futures = new ArrayList<CompletableFuture<LookupTopicResult>>();
106+
BrokerIdAwareLoadManager.clientIdList.clear();
107+
108+
final var clientIdList = IntStream.range(0, 10).mapToObj(i -> "key-" + i).toList();
109+
for (var clientId : clientIdList) {
110+
client.getConfiguration().setLookupProperties(Collections.singletonMap(CLIENT_KEY, clientId));
111+
futures.add(client.getLookup().getBroker(TopicName.get("test-concurrent-lookup-properties")));
112+
client.getConfiguration().setLookupProperties(Collections.emptyMap());
113+
}
114+
FutureUtil.waitForAll(futures).get();
115+
Assert.assertEquals(clientIdList, BrokerIdAwareLoadManager.clientIdList);
116+
}
117+
92118
public static class BrokerIdAwareLoadManager extends ExtensibleLoadManagerImpl {
119+
120+
static final List<String> clientIdList = Collections.synchronizedList(new ArrayList<>());
121+
122+
@Override
123+
public CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic,
124+
ServiceUnitId serviceUnit, LookupOptions options) {
125+
getClientId(options).ifPresent(clientIdList::add);
126+
return super.assign(topic, serviceUnit, options);
127+
}
128+
93129
@Override
94130
public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle, Set<String> excludeBrokerSet,
95131
LookupOptions options) {
@@ -106,5 +142,12 @@ public CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle, Set
106142
.orElseGet(() -> super.selectAsync(bundle, excludeBrokerSet, options));
107143
});
108144
}
145+
146+
private static Optional<String> getClientId(LookupOptions options) {
147+
if (options.getProperties() == null) {
148+
return Optional.empty();
149+
}
150+
return Optional.ofNullable(options.getProperties().get(CLIENT_KEY));
151+
}
109152
}
110153
}

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

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import io.opentelemetry.api.common.Attributes;
2525
import java.net.InetSocketAddress;
2626
import java.net.URI;
27+
import java.util.Map;
2728
import java.util.Optional;
2829
import java.util.concurrent.CompletableFuture;
2930
import java.util.concurrent.ConcurrentHashMap;
@@ -32,6 +33,7 @@
3233
import java.util.concurrent.TimeUnit;
3334
import java.util.concurrent.atomic.AtomicLong;
3435
import org.apache.commons.lang3.mutable.MutableObject;
36+
import org.apache.commons.lang3.tuple.Pair;
3537
import org.apache.pulsar.client.api.PulsarClientException;
3638
import org.apache.pulsar.client.api.SchemaSerializationException;
3739
import org.apache.pulsar.client.impl.metrics.LatencyHistogram;
@@ -60,7 +62,7 @@ public class BinaryProtoLookupService implements LookupService {
6062
private final String listenerName;
6163
private final int maxLookupRedirects;
6264

63-
private final ConcurrentHashMap<TopicName, CompletableFuture<LookupTopicResult>>
65+
private final ConcurrentHashMap<Pair<TopicName, Map<String, String>>, CompletableFuture<LookupTopicResult>>
6466
lookupInProgress = new ConcurrentHashMap<>();
6567

6668
private final ConcurrentHashMap<TopicName, CompletableFuture<PartitionedTopicMetadata>>
@@ -118,10 +120,12 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException {
118120
public CompletableFuture<LookupTopicResult> getBroker(TopicName topicName) {
119121
long startTime = System.nanoTime();
120122
final MutableObject<CompletableFuture> newFutureCreated = new MutableObject<>();
123+
final Pair<TopicName, Map<String, String>> key = Pair.of(topicName,
124+
client.getConfiguration().getLookupProperties());
121125
try {
122-
return lookupInProgress.computeIfAbsent(topicName, tpName -> {
123-
CompletableFuture<LookupTopicResult> newFuture =
124-
findBroker(serviceNameResolver.resolveHost(), false, topicName, 0);
126+
return lookupInProgress.computeIfAbsent(key, tpName -> {
127+
CompletableFuture<LookupTopicResult> newFuture = findBroker(serviceNameResolver.resolveHost(), false,
128+
topicName, 0, key.getRight());
125129
newFutureCreated.setValue(newFuture);
126130

127131
newFuture.thenRun(() -> {
@@ -135,7 +139,7 @@ public CompletableFuture<LookupTopicResult> getBroker(TopicName topicName) {
135139
} finally {
136140
if (newFutureCreated.getValue() != null) {
137141
newFutureCreated.getValue().whenComplete((v, ex) -> {
138-
lookupInProgress.remove(topicName, newFutureCreated.getValue());
142+
lookupInProgress.remove(key, newFutureCreated.getValue());
139143
});
140144
}
141145
}
@@ -167,7 +171,7 @@ public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(
167171
}
168172

169173
private CompletableFuture<LookupTopicResult> findBroker(InetSocketAddress socketAddress,
170-
boolean authoritative, TopicName topicName, final int redirectCount) {
174+
boolean authoritative, TopicName topicName, final int redirectCount, Map<String, String> properties) {
171175
CompletableFuture<LookupTopicResult> addressFuture = new CompletableFuture<>();
172176

173177
if (maxLookupRedirects > 0 && redirectCount > maxLookupRedirects) {
@@ -179,7 +183,7 @@ private CompletableFuture<LookupTopicResult> findBroker(InetSocketAddress socket
179183
client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> {
180184
long requestId = client.newRequestId();
181185
ByteBuf request = Commands.newLookup(topicName.toString(), listenerName, authoritative, requestId,
182-
client.getConfiguration().getLookupProperties());
186+
properties);
183187
clientCnx.newLookup(request, requestId).whenComplete((r, t) -> {
184188
if (t != null) {
185189
// lookup failed
@@ -204,7 +208,7 @@ private CompletableFuture<LookupTopicResult> findBroker(InetSocketAddress socket
204208

205209
// (2) redirect to given address if response is: redirect
206210
if (r.redirect) {
207-
findBroker(responseBrokerAddress, r.authoritative, topicName, redirectCount + 1)
211+
findBroker(responseBrokerAddress, r.authoritative, topicName, redirectCount + 1, properties)
208212
.thenAccept(addressFuture::complete)
209213
.exceptionally((lookupException) -> {
210214
Throwable cause = FutureUtil.unwrapCompletionException(lookupException);

0 commit comments

Comments
 (0)