Skip to content

Commit 6180556

Browse files
author
fanjianye
committed
resolve serviceUrl host separately when getConnection of each topic partition
1 parent f0e97f4 commit 6180556

File tree

14 files changed

+243
-6
lines changed

14 files changed

+243
-6
lines changed

pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -119,10 +119,13 @@ protected static String getTlsFileForClient(String name) {
119119
protected PulsarService pulsar;
120120
protected PulsarAdmin admin;
121121
protected PulsarClient pulsarClient;
122+
protected PulsarClient pulsarClientHttpUrlNotAllAvailable;
123+
protected PulsarClient pulsarClientserviceUrlNotAllAvailable;
122124
protected PortForwarder brokerGateway;
123125
protected boolean enableBrokerGateway = false;
124126
protected URL brokerUrl;
125127
protected URL brokerUrlTls;
128+
protected String brokerServiceUrl;
126129

127130
protected URI lookupUrl;
128131

@@ -164,6 +167,8 @@ protected final void internalSetup() throws Exception {
164167
}
165168
}
166169
pulsarClient = newPulsarClient(lookupUrl.toString(), 0);
170+
pulsarClientHttpUrlNotAllAvailable = newPulsarClient(brokerUrl.toString() + ",localhost:5678,localhost:5677,localhost:5676", 0);
171+
pulsarClientserviceUrlNotAllAvailable = newPulsarClient(brokerServiceUrl + ",localhost:5678,localhost:5677,localhost:5676", 0);
167172
}
168173

169174
protected final void internalSetup(ServiceConfiguration serviceConfiguration) throws Exception {
@@ -319,6 +324,7 @@ protected void startBroker() throws Exception {
319324

320325
brokerUrl = pulsar.getWebServiceAddress() != null ? new URL(pulsar.getWebServiceAddress()) : null;
321326
brokerUrlTls = pulsar.getWebServiceAddressTls() != null ? new URL(pulsar.getWebServiceAddressTls()) : null;
327+
brokerServiceUrl = pulsar.getBrokerServiceUrl() != null ? pulsar.getBrokerServiceUrl() : null;
322328

323329
if (admin != null) {
324330
admin.close();

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

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -191,4 +191,26 @@ public void testInitialSubscriptionCreationWithAutoCreationDisable()
191191

192192
Assert.assertFalse(admin.topics().getSubscriptions(topic.toString()).contains(initialSubscriptionName));
193193
}
194+
195+
@Test
196+
public void testCreateWhenServiceUrlNotAllAvailable() throws Exception {
197+
198+
final TopicName topic =
199+
TopicName.get("persistent", "public", "default", "testCreateInitialSubscriptionOnPartitionedTopic");
200+
admin.topics().createPartitionedTopic(topic.toString(), 20);
201+
202+
// use pulsar serviceUrl with unavailable host to new producer
203+
Producer<byte[]> producer = pulsarClientserviceUrlNotAllAvailable.newProducer()
204+
.topic(topic.toString())
205+
.create();
206+
207+
producer.close();
208+
209+
// use pulsar httpServiceUrl with unavailable host to new producer
210+
Producer<byte[]> producer2 = pulsarClientHttpUrlNotAllAvailable.newProducer()
211+
.topic(topic.toString())
212+
.create();
213+
214+
producer2.close();
215+
}
194216
}

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

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -95,6 +95,11 @@ public CompletableFuture<Pair<InetSocketAddress, InetSocketAddress>> getBroker(T
9595
return findBroker(serviceNameResolver.resolveHost(), false, topicName, 0);
9696
}
9797

98+
public CompletableFuture<Pair<InetSocketAddress, InetSocketAddress>> getBroker(TopicName topicName, int currentIndex) {
99+
InetSocketAddress socketAddress = serviceNameResolver.resolveHost(currentIndex);
100+
return findBroker(socketAddress, false, topicName, 0);
101+
}
102+
98103
/**
99104
* calls broker binaryProto-lookup api to get metadata of partitioned-topic.
100105
*
@@ -259,6 +264,10 @@ public String getServiceUrl() {
259264
return serviceNameResolver.getServiceUrl();
260265
}
261266

267+
public List<InetSocketAddress> getAddressList() {
268+
return serviceNameResolver.getAddressList();
269+
}
270+
262271
@Override
263272
public InetSocketAddress resolveHost() {
264273
return serviceNameResolver.resolveHost();

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

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,10 @@
1919
package org.apache.pulsar.client.impl;
2020

2121
import java.net.InetSocketAddress;
22+
import java.util.List;
2223
import java.util.concurrent.CompletableFuture;
2324
import java.util.concurrent.TimeUnit;
25+
import java.util.concurrent.atomic.AtomicInteger;
2426
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
2527
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
2628
import org.apache.pulsar.client.api.PulsarClientException;
@@ -41,6 +43,7 @@ public class ConnectionHandler {
4143
// Start with -1L because it gets incremented before sending on the first connection
4244
private volatile long epoch = -1L;
4345
protected volatile long lastConnectionClosedTimestamp = 0L;
46+
private AtomicInteger serviceResolverIndex;
4447

4548
interface Connection {
4649
void connectionFailed(PulsarClientException exception);
@@ -49,11 +52,18 @@ interface Connection {
4952

5053
protected Connection connection;
5154

52-
protected ConnectionHandler(HandlerState state, Backoff backoff, Connection connection) {
55+
protected ConnectionHandler(HandlerState state, Backoff backoff, Connection connection, List<InetSocketAddress> addressList) {
5356
this.state = state;
5457
this.connection = connection;
5558
this.backoff = backoff;
5659
CLIENT_CNX_UPDATER.set(this, null);
60+
this.serviceResolverIndex = new AtomicInteger(randomIndex(addressList.size()));
61+
}
62+
63+
private static int randomIndex(int numAddresses) {
64+
return numAddresses == 1
65+
?
66+
0 : io.netty.util.internal.PlatformDependent.threadLocalRandom().nextInt(numAddresses);
5767
}
5868

5969
protected void grabCnx() {
@@ -79,7 +89,7 @@ protected void grabCnx() {
7989
} else if (state.topic == null) {
8090
cnxFuture = state.client.getConnectionToServiceUrl();
8191
} else {
82-
cnxFuture = state.client.getConnection(state.topic); //
92+
cnxFuture = state.client.getConnection(state.topic, serviceResolverIndex.getAndIncrement());
8393
}
8494
cnxFuture.thenAccept(cnx -> connection.connectionOpened(cnx)) //
8595
.exceptionally(this::handleConnectionError);

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -332,7 +332,7 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat
332332
.setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS)
333333
.setMandatoryStop(0, TimeUnit.MILLISECONDS)
334334
.create(),
335-
this);
335+
this, client.getLookup().getAddressList());
336336

337337
this.topicName = TopicName.get(topic);
338338
if (this.topicName.isPersistent()) {

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

Lines changed: 86 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -264,4 +264,90 @@ public <T> CompletableFuture<T> get(String path, Class<T> clazz) {
264264

265265
return future;
266266
}
267+
268+
public <T> CompletableFuture<T> get(String path, Class<T> clazz, int currentIndex) {
269+
final CompletableFuture<T> future = new CompletableFuture<>();
270+
try {
271+
URI hostUri = serviceNameResolver.resolveHostUri(currentIndex);
272+
String requestUrl = new URL(hostUri.toURL(), path).toString();
273+
String remoteHostName = hostUri.getHost();
274+
AuthenticationDataProvider authData = authentication.getAuthData(remoteHostName);
275+
276+
CompletableFuture<Map<String, String>> authFuture = new CompletableFuture<>();
277+
278+
// bring a authenticationStage for sasl auth.
279+
if (authData.hasDataForHttp()) {
280+
authentication.authenticationStage(requestUrl, authData, null, authFuture);
281+
} else {
282+
authFuture.complete(null);
283+
}
284+
285+
// auth complete, do real request
286+
authFuture.whenComplete((respHeaders, ex) -> {
287+
if (ex != null) {
288+
log.warn("[{}] Failed to perform http request at authentication stage: {}",
289+
requestUrl, ex.getMessage());
290+
future.completeExceptionally(new PulsarClientException(ex));
291+
return;
292+
}
293+
294+
// auth complete, use a new builder
295+
BoundRequestBuilder builder = httpClient.prepareGet(requestUrl)
296+
.setHeader("Accept", "application/json");
297+
298+
if (authData.hasDataForHttp()) {
299+
Set<Entry<String, String>> headers;
300+
try {
301+
headers = authentication.newRequestHeader(requestUrl, authData, respHeaders);
302+
} catch (Exception e) {
303+
log.warn("[{}] Error during HTTP get headers: {}", requestUrl, e.getMessage());
304+
future.completeExceptionally(new PulsarClientException(e));
305+
return;
306+
}
307+
if (headers != null) {
308+
headers.forEach(entry -> builder.addHeader(entry.getKey(), entry.getValue()));
309+
}
310+
}
311+
312+
builder.execute().toCompletableFuture().whenComplete((response2, t) -> {
313+
if (t != null) {
314+
log.warn("[{}] Failed to perform http request: {}", requestUrl, t.getMessage());
315+
future.completeExceptionally(new PulsarClientException(t));
316+
return;
317+
}
318+
319+
// request not success
320+
if (response2.getStatusCode() != HttpURLConnection.HTTP_OK) {
321+
log.warn("[{}] HTTP get request failed: {}", requestUrl, response2.getStatusText());
322+
Exception e;
323+
if (response2.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
324+
e = new NotFoundException("Not found: " + response2.getStatusText());
325+
} else {
326+
e = new PulsarClientException("HTTP get request failed: " + response2.getStatusText());
327+
}
328+
future.completeExceptionally(e);
329+
return;
330+
}
331+
332+
try {
333+
T data = ObjectMapperFactory.getMapper().reader().readValue(
334+
response2.getResponseBodyAsBytes(), clazz);
335+
future.complete(data);
336+
} catch (Exception e) {
337+
log.warn("[{}] Error during HTTP get request: {}", requestUrl, e.getMessage());
338+
future.completeExceptionally(new PulsarClientException(e));
339+
}
340+
});
341+
});
342+
} catch (Exception e) {
343+
log.warn("[{}]PulsarClientImpl: {}", path, e.getMessage());
344+
if (e instanceof PulsarClientException) {
345+
future.completeExceptionally(e);
346+
} else {
347+
future.completeExceptionally(new PulsarClientException(e));
348+
}
349+
}
350+
351+
return future;
352+
}
267353
}

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

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,41 @@ public CompletableFuture<Pair<InetSocketAddress, InetSocketAddress>> getBroker(T
111111
});
112112
}
113113

114+
@Override
115+
public CompletableFuture<Pair<InetSocketAddress, InetSocketAddress>> getBroker(TopicName topicName, int currentIndex) {
116+
String basePath = topicName.isV2() ? BasePathV2 : BasePathV1;
117+
String path = basePath + topicName.getLookupName();
118+
path = StringUtils.isBlank(listenerName) ? path : path + "?listenerName=" + Codec.encode(listenerName);
119+
return httpClient.get(path, LookupData.class, currentIndex)
120+
.thenCompose(lookupData -> {
121+
// Convert LookupData into as SocketAddress, handling exceptions
122+
URI uri = null;
123+
try {
124+
if (useTls) {
125+
uri = new URI(lookupData.getBrokerUrlTls());
126+
} else {
127+
String serviceUrl = lookupData.getBrokerUrl();
128+
if (serviceUrl == null) {
129+
serviceUrl = lookupData.getNativeUrl();
130+
}
131+
uri = new URI(serviceUrl);
132+
}
133+
134+
InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort());
135+
return CompletableFuture.completedFuture(Pair.of(brokerAddress, brokerAddress));
136+
} catch (Exception e) {
137+
// Failed to parse url
138+
log.warn("[{}] Lookup Failed due to invalid url {}, {}", topicName, uri, e.getMessage());
139+
return FutureUtil.failedFuture(e);
140+
}
141+
});
142+
}
143+
144+
@Override
145+
public List<InetSocketAddress> getAddressList() {
146+
return httpClient.serviceNameResolver.getAddressList();
147+
}
148+
114149
@Override
115150
public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(TopicName topicName) {
116151
String format = topicName.isV2() ? "admin/v2/%s/partitions" : "admin/%s/partitions";

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

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.pulsar.client.impl;
2020

2121
import java.net.InetSocketAddress;
22+
import java.util.List;
2223
import java.util.Optional;
2324
import java.util.concurrent.CompletableFuture;
2425
import org.apache.commons.lang3.tuple.Pair;
@@ -58,6 +59,18 @@ public interface LookupService extends AutoCloseable {
5859
*/
5960
CompletableFuture<Pair<InetSocketAddress, InetSocketAddress>> getBroker(TopicName topicName);
6061

62+
/**
63+
* Calls broker lookup-api to get broker {@link InetSocketAddress} which serves namespace bundle that contains given
64+
* topic. use currentIndex to select the indexed address of the serviceUrl address list
65+
*
66+
* @param topicName
67+
* topic-name
68+
* @param currentIndex
69+
* index of multi-serviceUrl
70+
* @return a pair of addresses, representing the logical and physical address of the broker that serves given topic
71+
*/
72+
CompletableFuture<Pair<InetSocketAddress, InetSocketAddress>> getBroker(TopicName topicName, int currentIndex);
73+
6174
/**
6275
* Returns {@link PartitionedTopicMetadata} for a given topic.
6376
*
@@ -90,6 +103,13 @@ public interface LookupService extends AutoCloseable {
90103
*/
91104
String getServiceUrl();
92105

106+
/**
107+
* Returns serviceUrl address list.
108+
*
109+
* @return
110+
*/
111+
List<InetSocketAddress> getAddressList();
112+
93113
/**
94114
* Resolves pulsar service url.
95115
*

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -273,7 +273,7 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration
273273
.setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS)
274274
.setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS)
275275
.create(),
276-
this);
276+
this, client.getLookup().getAddressList());
277277

278278
grabCnx();
279279
}

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

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -950,6 +950,12 @@ public CompletableFuture<ClientCnx> getConnection(final String topic) {
950950
.thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight()));
951951
}
952952

953+
public CompletableFuture<ClientCnx> getConnection(final String topic, int currentIndex) {
954+
TopicName topicName = TopicName.get(topic);
955+
return lookup.getBroker(topicName, currentIndex)
956+
.thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight()));
957+
}
958+
953959
public CompletableFuture<ClientCnx> getConnectionToServiceUrl() {
954960
if (!(lookup instanceof BinaryProtoLookupService)) {
955961
return FutureUtil.failedFuture(new PulsarClientException.InvalidServiceURL(

0 commit comments

Comments
 (0)