Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,7 @@
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.transaction.TransactionBufferClient;
import org.apache.pulsar.client.impl.DnsResolverGroupImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils;
Expand Down Expand Up @@ -266,6 +267,7 @@ public class PulsarService implements AutoCloseable, ShutdownService {
private final ScheduledExecutorProvider brokerClientSharedScheduledExecutorProvider;
private final Timer brokerClientSharedTimer;
private final ExecutorProvider brokerClientSharedLookupExecutorProvider;
private final DnsResolverGroupImpl brokerClientSharedDnsResolverGroup;

private MetricsGenerator metricsGenerator;
private final PulsarBrokerOpenTelemetry openTelemetry;
Expand Down Expand Up @@ -398,6 +400,9 @@ public PulsarService(ServiceConfiguration config,
new HashedWheelTimer(new DefaultThreadFactory("broker-client-shared-timer"), 1, TimeUnit.MILLISECONDS);
this.brokerClientSharedLookupExecutorProvider =
new ScheduledExecutorProvider(1, "broker-client-shared-lookup-executor");
this.brokerClientSharedDnsResolverGroup =
new DnsResolverGroupImpl(this.ioEventLoopGroup,
loadBrokerClientProperties(new ClientConfigurationData()));

// here in the constructor we don't have the offloader scheduler yet
this.offloaderStats = LedgerOffloaderStats.create(false, false, null, 0);
Expand Down Expand Up @@ -697,6 +702,7 @@ public CompletableFuture<Void> closeAsync(boolean waitForWebServiceToStop) {
brokerClientSharedInternalExecutorProvider.shutdownNow();
brokerClientSharedScheduledExecutorProvider.shutdownNow();
brokerClientSharedLookupExecutorProvider.shutdownNow();
brokerClientSharedDnsResolverGroup.close();
brokerClientSharedTimer.stop();
if (monotonicClock instanceof AutoCloseable c) {
c.close();
Expand Down Expand Up @@ -1711,7 +1717,8 @@ public PulsarClientImpl createClientImpl(ClientConfigurationData conf,
.internalExecutorProvider(brokerClientSharedInternalExecutorProvider)
.externalExecutorProvider(brokerClientSharedExternalExecutorProvider)
.scheduledExecutorProvider(brokerClientSharedScheduledExecutorProvider)
.lookupExecutorProvider(brokerClientSharedLookupExecutorProvider);
.lookupExecutorProvider(brokerClientSharedLookupExecutorProvider)
.dnsResolverGroup(brokerClientSharedDnsResolverGroup);
if (customizer != null) {
customizer.accept(pulsarClientImplBuilder);
}
Expand Down Expand Up @@ -1740,10 +1747,7 @@ protected ClientConfigurationData createClientConfigurationData()
// Apply all arbitrary configuration. This must be called before setting any fields annotated as
// @Secret on the ClientConfigurationData object because of the way they are serialized.
// See https://github.com/apache/pulsar/issues/8509 for more information.
Map<String, Object> overrides = PropertiesUtils
.filterAndMapProperties(this.getConfiguration().getProperties(), "brokerClient_");
ClientConfigurationData conf =
ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
ClientConfigurationData conf = loadBrokerClientProperties(initialConf);

// Disabled auto release useless connections
// The automatic release connection feature is not yet perfect for transaction scenarios, so turn it
Expand Down Expand Up @@ -1789,6 +1793,15 @@ protected ClientConfigurationData createClientConfigurationData()
return conf;
}

// load plain brokerClient_ properties without complete initialization
private ClientConfigurationData loadBrokerClientProperties(ClientConfigurationData initialConf) {
Map<String, Object> overrides = PropertiesUtils
.filterAndMapProperties(this.getConfiguration().getProperties(), "brokerClient_");
ClientConfigurationData conf =
ConfigurationDataUtils.loadData(overrides, initialConf, ClientConfigurationData.class);
return conf;
}

public synchronized PulsarAdmin getAdminClient() throws PulsarServerException {
if (this.adminClient == null) {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs;
import io.netty.channel.EventLoopGroup;
import io.netty.resolver.AbstractAddressResolver;
import io.netty.resolver.AddressResolver;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.netty.util.concurrent.Promise;
import java.net.InetSocketAddress;
Expand Down Expand Up @@ -260,7 +261,8 @@ protected void doResolveAll(SocketAddress socketAddress, Promise promise) throws
ConnectionPool pool =
spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop,
(Supplier<ClientCnx>) () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop),
Optional.of(resolver), scheduledExecutorService);
Optional.<Supplier<AddressResolver<InetSocketAddress>>>of(() -> resolver),
scheduledExecutorService);


ClientCnx cnx = pool.getConnection(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,6 @@
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.resolver.AddressResolver;
import io.netty.resolver.dns.DnsAddressResolverGroup;
import io.netty.resolver.dns.DnsNameResolverBuilder;
import io.netty.resolver.dns.SequentialDnsServerAddressStreamProvider;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.ScheduledFuture;
import io.opentelemetry.api.common.Attributes;
Expand All @@ -51,6 +48,8 @@
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import lombok.Builder;
import lombok.NonNull;
import lombok.Value;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.client.api.PulsarClientException;
Expand All @@ -61,7 +60,6 @@
import org.apache.pulsar.client.impl.metrics.Unit;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.netty.DnsResolverUtil;
import org.apache.pulsar.common.util.netty.EventLoopUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -80,6 +78,7 @@ public class ConnectionPool implements AutoCloseable {
private final boolean isSniProxy;

protected final AddressResolver<InetSocketAddress> addressResolver;
private DnsResolverGroupImpl dnsResolverGroup;
private final boolean shouldCloseDnsResolver;


Expand All @@ -106,8 +105,7 @@ private static class Key {
public ConnectionPool(InstrumentProvider instrumentProvider,
ClientConfigurationData conf, EventLoopGroup eventLoopGroup,
ScheduledExecutorService scheduledExecutorService) throws PulsarClientException {
this(instrumentProvider, conf, eventLoopGroup, () -> new ClientCnx(instrumentProvider, conf, eventLoopGroup),
scheduledExecutorService);
this(instrumentProvider, conf, eventLoopGroup, null, scheduledExecutorService);
}

public ConnectionPool(InstrumentProvider instrumentProvider,
Expand All @@ -118,12 +116,16 @@ public ConnectionPool(InstrumentProvider instrumentProvider,
scheduledExecutorService);
}

public ConnectionPool(InstrumentProvider instrumentProvider,
ClientConfigurationData conf, EventLoopGroup eventLoopGroup,
@Builder(builderClassName = "ConnectionPoolBuilder")
public ConnectionPool(@NonNull InstrumentProvider instrumentProvider,
@NonNull ClientConfigurationData conf, @NonNull EventLoopGroup eventLoopGroup,
Supplier<ClientCnx> clientCnxSupplier,
Optional<AddressResolver<InetSocketAddress>> addressResolver,
@NonNull Optional<Supplier<AddressResolver<InetSocketAddress>>> addressResolverSupplier,
ScheduledExecutorService scheduledExecutorService)
throws PulsarClientException {
if (clientCnxSupplier == null) {
clientCnxSupplier = () -> new ClientCnx(instrumentProvider, conf, eventLoopGroup);
}
this.eventLoopGroup = eventLoopGroup;
this.clientConfig = conf;
this.maxConnectionsPerHosts = conf.getConnectionsPerBroker();
Expand Down Expand Up @@ -152,8 +154,9 @@ public ConnectionPool(InstrumentProvider instrumentProvider,
throw new PulsarClientException(e);
}

this.shouldCloseDnsResolver = !addressResolver.isPresent();
this.addressResolver = addressResolver.orElseGet(() -> createAddressResolver(conf, eventLoopGroup));
this.shouldCloseDnsResolver = !addressResolverSupplier.isPresent();
this.addressResolver =
addressResolverSupplier.orElseGet(() -> createAddressResolver(conf, eventLoopGroup)).get();
// Auto release useless connections. see: https://github.com/apache/pulsar/issues/15516.
this.connectionMaxIdleSeconds = conf.getConnectionMaxIdleSeconds();
this.autoReleaseIdleConnectionsEnabled = connectionMaxIdleSeconds > 0;
Expand Down Expand Up @@ -185,26 +188,12 @@ public ConnectionPool(InstrumentProvider instrumentProvider,
Attributes.builder().put("pulsar.failure.type", "handshake").build());
}

private static AddressResolver<InetSocketAddress> createAddressResolver(ClientConfigurationData conf,
EventLoopGroup eventLoopGroup) {
DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder()
.traceEnabled(true)
.channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup))
.socketChannelType(EventLoopUtil.getClientSocketChannelClass(eventLoopGroup), true);
if (conf.getDnsLookupBindAddress() != null) {
InetSocketAddress addr = new InetSocketAddress(conf.getDnsLookupBindAddress(),
conf.getDnsLookupBindPort());
dnsNameResolverBuilder.localAddress(addr);
}
List<InetSocketAddress> serverAddresses = conf.getDnsServerAddresses();
if (serverAddresses != null && !serverAddresses.isEmpty()) {
dnsNameResolverBuilder.nameServerProvider(new SequentialDnsServerAddressStreamProvider(serverAddresses));
private Supplier<AddressResolver<InetSocketAddress>> createAddressResolver(ClientConfigurationData conf,
EventLoopGroup eventLoopGroup) {
if (dnsResolverGroup == null) {
dnsResolverGroup = new DnsResolverGroupImpl(eventLoopGroup, conf);
}
DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder);
// use DnsAddressResolverGroup to create the AddressResolver since it contains a solution
// to prevent cache stampede / thundering herds problem when a DNS entry expires while the system
// is under high load
return new DnsAddressResolverGroup(dnsNameResolverBuilder).getResolver(eventLoopGroup.next());
return () -> dnsResolverGroup.createAddressResolver(eventLoopGroup);
}

private static final Random random = new Random();
Expand Down Expand Up @@ -479,6 +468,9 @@ public void close() throws Exception {
if (shouldCloseDnsResolver) {
addressResolver.close();
}
if (dnsResolverGroup != null) {
dnsResolverGroup.close();
}
if (asyncReleaseUselessConnectionsTask != null && !asyncReleaseUselessConnectionsTask.isCancelled()) {
asyncReleaseUselessConnectionsTask.cancel(false);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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 io.netty.channel.EventLoopGroup;
import io.netty.resolver.AddressResolver;
import io.netty.resolver.dns.DnsAddressResolverGroup;
import io.netty.resolver.dns.DnsNameResolverBuilder;
import io.netty.resolver.dns.DnsServerAddressStreamProvider;
import io.netty.resolver.dns.SequentialDnsServerAddressStreamProvider;
import java.net.InetSocketAddress;
import java.util.List;
import java.util.Optional;
import java.util.function.Predicate;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.common.util.netty.DnsResolverUtil;
import org.apache.pulsar.common.util.netty.EventLoopUtil;

/**
* An abstraction to manage a group of Netty {@link AddressResolver} instances.
* Uses {@link io.netty.resolver.dns.DnsAddressResolverGroup} to create the {@link AddressResolver} instance
* since it contains a shared DNS cache and a solution to prevent cache stampede / thundering herds problem
* when a DNS entry expires while the system is under high load.
*/
public class DnsResolverGroupImpl implements AutoCloseable {
private final DnsAddressResolverGroup dnsAddressResolverGroup;

public DnsResolverGroupImpl(EventLoopGroup eventLoopGroup, ClientConfigurationData conf) {
Optional<InetSocketAddress> bindAddress = Optional.ofNullable(conf.getDnsLookupBindAddress())
.map(addr -> new InetSocketAddress(addr, conf.getDnsLookupBindPort()));
Optional<DnsServerAddressStreamProvider> dnsServerAddresses = Optional.ofNullable(conf.getDnsServerAddresses())
.filter(Predicate.not(List::isEmpty))
.map(SequentialDnsServerAddressStreamProvider::new);
this.dnsAddressResolverGroup = createAddressResolverGroup(eventLoopGroup, bindAddress, dnsServerAddresses);
}

public DnsResolverGroupImpl(EventLoopGroup eventLoopGroup, Optional<InetSocketAddress> bindAddress,
Optional<DnsServerAddressStreamProvider> dnsServerAddresses) {
this.dnsAddressResolverGroup = createAddressResolverGroup(eventLoopGroup, bindAddress, dnsServerAddresses);
}

private static DnsAddressResolverGroup createAddressResolverGroup(EventLoopGroup eventLoopGroup,
Optional<InetSocketAddress> bindAddress,
Optional<DnsServerAddressStreamProvider>
dnsServerAddresses) {
DnsNameResolverBuilder dnsNameResolverBuilder = createDnsNameResolverBuilder(eventLoopGroup);
bindAddress.ifPresent(dnsNameResolverBuilder::localAddress);
dnsServerAddresses.ifPresent(dnsNameResolverBuilder::nameServerProvider);

return new DnsAddressResolverGroup(dnsNameResolverBuilder);
}

private static DnsNameResolverBuilder createDnsNameResolverBuilder(EventLoopGroup eventLoopGroup) {
DnsNameResolverBuilder dnsNameResolverBuilder = new DnsNameResolverBuilder()
.traceEnabled(true)
.channelType(EventLoopUtil.getDatagramChannelClass(eventLoopGroup))
.socketChannelType(EventLoopUtil.getClientSocketChannelClass(eventLoopGroup), true);
DnsResolverUtil.applyJdkDnsCacheSettings(dnsNameResolverBuilder);
return dnsNameResolverBuilder;
}

@Override
public void close() {
this.dnsAddressResolverGroup.close();
}

public AddressResolver<InetSocketAddress> createAddressResolver(EventLoopGroup eventLoopGroup) {
return dnsAddressResolverGroup.getResolver(eventLoopGroup.next());
}
}
Loading
Loading