From e5be40963be93ec3e7cf34bd681863e1392b61b9 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Thu, 4 Dec 2025 17:32:36 +0000 Subject: [PATCH 01/30] KAFKA-14648: Add boostrap timeout config --- .../org/apache/kafka/clients/ClientUtils.java | 11 +- .../kafka/clients/CommonClientConfigs.java | 6 + .../org/apache/kafka/clients/KafkaClient.java | 5 + .../kafka/clients/ManualMetadataUpdater.java | 11 + .../apache/kafka/clients/MetadataUpdater.java | 12 + .../apache/kafka/clients/NetworkClient.java | 102 +++- .../kafka/clients/admin/KafkaAdminClient.java | 54 +- .../admin/internals/AdminMetadataManager.java | 11 + .../clients/consumer/ConsumerConfig.java | 10 + .../internals/ClassicKafkaConsumer.java | 3 - .../kafka/clients/producer/KafkaProducer.java | 3 - .../network/BootstrapResolutionException.java | 35 ++ .../org/apache/kafka/clients/MockClient.java | 5 + .../kafka/clients/NetworkClientTest.java | 516 +++++++++--------- .../internals/FetchRequestManagerTest.java | 2 +- .../consumer/internals/FetcherTest.java | 2 +- .../clients/producer/KafkaProducerTest.java | 20 +- .../producer/internals/SenderTest.java | 3 +- .../distributed/WorkerGroupMember.java | 10 +- .../main/java/kafka/server/NetworkUtils.java | 4 +- .../TransactionMarkerChannelManager.scala | 4 +- .../scala/kafka/raft/KafkaRaftManager.scala | 5 +- .../kafka/server/BrokerBlockingSender.scala | 4 +- .../NodeToControllerChannelManager.scala | 3 +- .../kafka/tools/BrokerApiVersionsCommand.java | 4 +- .../kafka/tools/ReplicaVerificationTool.java | 9 +- .../workload/ConnectionStressWorker.java | 4 +- 27 files changed, 550 insertions(+), 308 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index c295713df2370..1952020f47750 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -41,6 +41,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.closeQuietly; @@ -219,6 +220,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, ClientTelemetrySender clientTelemetrySender) { ChannelBuilder channelBuilder = null; Selector selector = null; + NetworkClient.BootstrapConfiguration bootstrapConfiguration = null; try { channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); @@ -228,6 +230,12 @@ public static NetworkClient createNetworkClient(AbstractConfig config, metricsGroupPrefix, channelBuilder, logContext); + bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + time, + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ); return new NetworkClient(metadataUpdater, metadata, selector, @@ -248,7 +256,8 @@ public static NetworkClient createNetworkClient(AbstractConfig config, hostResolver, clientTelemetrySender, config.getLong(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG), - MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)) + MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), + Optional.of(bootstrapConfiguration) ); } catch (Throwable t) { closeQuietly(selector, "Selector"); diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 08b861673e3d7..6c249b61a54bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -60,6 +60,12 @@ public class CommonClientConfigs { + "resolve each bootstrap address into a list of canonical names. After " + "the bootstrap phase, this behaves the same as use_all_dns_ips."; + public static final String BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG = "bootstrap.resolve.timeout.ms"; + public static final long DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS = 2 * 60 * 1000L; + public static final String BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC = "Maximum amount of time clients can spend trying to" + + " resolve for the bootstrap server address. If the resolution cannot be completed within this timeframe, a " + + "BootstrapResolutionException will be thrown."; + public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 46b6498606412..791ffaa5d638e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -213,4 +213,9 @@ ClientRequest newClientRequest(String nodeId, */ boolean active(); + /** + * Returns true if the client has already been bootstrapped. Returns false if the client has not + * been bootstrapped or if bootstrap servers must be resolved again. + */ + boolean isBootstrapped(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java index 3d5154994fcda..ed909850ecf03 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.RequestHeader; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -81,6 +82,16 @@ public void handleSuccessfulResponse(RequestHeader requestHeader, long now, Meta // Do nothing } + @Override + public boolean isBootstrapped() { + return false; + } + + @Override + public void bootstrap(List addresses) { + + } + @Override public void close() { } diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java index 825d2e67f70d7..add46cb7fcbca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.requests.RequestHeader; import java.io.Closeable; +import java.net.InetSocketAddress; import java.util.List; import java.util.Optional; @@ -102,6 +103,17 @@ default boolean needsRebootstrap(long now, long rebootstrapTriggerMs) { */ default void rebootstrap(long now) {} + /** + * Returns true if the metadata has been bootstrapped. + */ + boolean isBootstrapped(); + + /** + * Bootstrap the metadata cache with the given addresses. + * @param addresses list of addresses for the bootstrap servers + */ + void bootstrap(List addresses); + /** * Close this updater. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 692847a8b1553..d8f6242a48754 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.network.BootstrapResolutionException; import org.apache.kafka.common.network.ChannelState; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkSend; @@ -47,6 +48,7 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetrySender; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -137,6 +139,8 @@ private enum State { private final AtomicReference state; + private final BootstrapConfiguration bootstrapConfiguration; + private final TelemetrySender telemetrySender; public NetworkClient(Selectable selector, @@ -154,7 +158,8 @@ public NetworkClient(Selectable selector, boolean discoverBrokerVersions, ApiVersions apiVersions, LogContext logContext, - MetadataRecoveryStrategy metadataRecoveryStrategy) { + MetadataRecoveryStrategy metadataRecoveryStrategy, + Optional bootstrapConfiguration) { this(selector, metadata, clientId, @@ -171,7 +176,8 @@ public NetworkClient(Selectable selector, apiVersions, logContext, Long.MAX_VALUE, - metadataRecoveryStrategy); + metadataRecoveryStrategy, + bootstrapConfiguration); } public NetworkClient(Selectable selector, @@ -190,7 +196,8 @@ public NetworkClient(Selectable selector, ApiVersions apiVersions, LogContext logContext, long rebootstrapTriggerMs, - MetadataRecoveryStrategy metadataRecoveryStrategy) { + MetadataRecoveryStrategy metadataRecoveryStrategy, + Optional bootstrapConfiguration) { this(null, metadata, selector, @@ -211,7 +218,8 @@ public NetworkClient(Selectable selector, new DefaultHostResolver(), null, rebootstrapTriggerMs, - metadataRecoveryStrategy); + metadataRecoveryStrategy, + bootstrapConfiguration); } public NetworkClient(Selectable selector, @@ -230,7 +238,8 @@ public NetworkClient(Selectable selector, ApiVersions apiVersions, Sensor throttleTimeSensor, LogContext logContext, - MetadataRecoveryStrategy metadataRecoveryStrategy) { + MetadataRecoveryStrategy metadataRecoveryStrategy, + Optional bootstrapConfiguration) { this(null, metadata, selector, @@ -251,7 +260,8 @@ public NetworkClient(Selectable selector, new DefaultHostResolver(), null, Long.MAX_VALUE, - metadataRecoveryStrategy); + metadataRecoveryStrategy, + bootstrapConfiguration); } public NetworkClient(Selectable selector, @@ -269,7 +279,8 @@ public NetworkClient(Selectable selector, boolean discoverBrokerVersions, ApiVersions apiVersions, LogContext logContext, - MetadataRecoveryStrategy metadataRecoveryStrategy) { + MetadataRecoveryStrategy metadataRecoveryStrategy, + Optional bootstrapConfiguration) { this(metadataUpdater, null, selector, @@ -290,7 +301,8 @@ public NetworkClient(Selectable selector, new DefaultHostResolver(), null, Long.MAX_VALUE, - metadataRecoveryStrategy); + metadataRecoveryStrategy, + bootstrapConfiguration); } public NetworkClient(MetadataUpdater metadataUpdater, @@ -313,7 +325,8 @@ public NetworkClient(MetadataUpdater metadataUpdater, HostResolver hostResolver, ClientTelemetrySender clientTelemetrySender, long rebootstrapTriggerMs, - MetadataRecoveryStrategy metadataRecoveryStrategy) { + MetadataRecoveryStrategy metadataRecoveryStrategy, + Optional bootstrapConfiguration) { /* It would be better if we could pass `DefaultMetadataUpdater` from the public constructor, but it's not * possible because `DefaultMetadataUpdater` is an inner class and it can only be instantiated after the * super constructor is invoked. @@ -346,6 +359,7 @@ public NetworkClient(MetadataUpdater metadataUpdater, this.telemetrySender = (clientTelemetrySender != null) ? new TelemetrySender(clientTelemetrySender) : null; this.rebootstrapTriggerMs = rebootstrapTriggerMs; this.metadataRecoveryStrategy = metadataRecoveryStrategy; + this.bootstrapConfiguration = bootstrapConfiguration.orElse(null); } /** @@ -629,6 +643,7 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long @Override public List poll(long timeout, long now) { ensureActive(); + ensureBootstrapped(); if (!abortedSends.isEmpty()) { // If there are aborted sends because of unsupported version exceptions or disconnects, @@ -1171,6 +1186,64 @@ private boolean isTelemetryApi(ApiKeys apiKey) { return apiKey == ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS || apiKey == ApiKeys.PUSH_TELEMETRY; } + public static class BootstrapConfiguration { + private final Timer timer; + private final List bootstrapServers; + private final ClientDnsLookup clientDnsLookup; + private final long bootstrapResolveTimeoutMs; + private boolean isBootstrapped = false; + + public BootstrapConfiguration(Time time, + List bootstrapServers, + ClientDnsLookup clientDnsLookup, + long bootstrapResolveTimeoutMs) { + this.timer = time.timer(bootstrapResolveTimeoutMs); + this.bootstrapServers = bootstrapServers; + this.clientDnsLookup = clientDnsLookup; + this.bootstrapResolveTimeoutMs = bootstrapResolveTimeoutMs; + } + + private void checkTimerExpiration() { + if (timer.isExpired()) { + throw new BootstrapResolutionException("Unable to Resolve Address within the configured period " + + bootstrapResolveTimeoutMs + " ms."); + } + } + + private List tryResolveAddresses() { + List addresses = ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookup); + + if (!addresses.isEmpty()) { + timer.reset(bootstrapResolveTimeoutMs); + isBootstrapped = true; + return addresses; + } else { + return List.of(); + } + } + } + + void ensureBootstrapped() { + if (isBootstrapped() || null == bootstrapConfiguration) + return; + + bootstrapConfiguration.timer.update(time.milliseconds()); + bootstrapConfiguration.checkTimerExpiration(); + + List servers = bootstrapConfiguration.tryResolveAddresses(); + if (!servers.isEmpty()) { + metadataUpdater.bootstrap(servers); + } + } + + public boolean isBootstrapped() { + if (null == bootstrapConfiguration) + return false; + else + return bootstrapConfiguration.isBootstrapped; + } + + class DefaultMetadataUpdater implements MetadataUpdater { /* the current cluster metadata */ @@ -1195,6 +1268,7 @@ class DefaultMetadataUpdater implements MetadataUpdater { @Override public List fetchNodes() { + ensureBootstrapped(); return metadata.fetch().nodes(); } @@ -1321,6 +1395,16 @@ public void rebootstrap(long now) { metadataAttemptStartMs = Optional.of(now); } + @Override + public boolean isBootstrapped() { + return metadata.fetch().isBootstrapConfigured(); + } + + @Override + public void bootstrap(List addresses) { + metadata.bootstrap(addresses); + } + @Override public void close() { this.metadata.close(); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index fa9440dfb9102..7e0960865dea0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientUtils; @@ -176,6 +177,8 @@ import org.apache.kafka.common.metrics.MetricsContext; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.network.ChannelBuilder; +import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; @@ -416,6 +419,11 @@ public class KafkaAdminClient extends AdminClient { */ private Uuid clientInstanceId; + /** + * Addresses for the admin client to bootstrap + */ + private static AdminBootstrapAddresses adminAddresses; + /** * Get or create a list value from a map. * @@ -523,6 +531,10 @@ static KafkaAdminClient createInternal( ) { Metrics metrics = null; NetworkClient networkClient = null; + NetworkClient.BootstrapConfiguration bootstrapConfiguration; + ChannelBuilder channelBuilder; + Selector selector; + List bootstrapAddresses = new ArrayList<>(); Time time = Time.SYSTEM; String clientId = generateClientId(config); ApiVersions apiVersions = new ApiVersions(); @@ -532,12 +544,11 @@ static KafkaAdminClient createInternal( try { // Since we only request node information, it's safe to pass true for allowAutoTopicCreation (and it // simplifies communication with older brokers) - AdminBootstrapAddresses adminAddresses = AdminBootstrapAddresses.fromConfig(config); + adminAddresses = AdminBootstrapAddresses.fromConfig(config); AdminMetadataManager metadataManager = new AdminMetadataManager(logContext, config.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), config.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), adminAddresses.usingBootstrapControllers()); - metadataManager.update(Cluster.bootstrap(adminAddresses.addresses()), time.milliseconds()); List reporters = CommonClientConfigs.metricsReporters(clientId, config); clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); clientTelemetryReporter.ifPresent(reporters::add); @@ -549,20 +560,45 @@ static KafkaAdminClient createInternal( MetricsContext metricsContext = new KafkaMetricsContext(JMX_PREFIX, config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)); metrics = new Metrics(metricConfig, reporters, time, metricsContext); - networkClient = ClientUtils.createNetworkClient(config, - clientId, + adminAddresses.addresses().forEach(inetSocketAddress -> + bootstrapAddresses.add(inetSocketAddress.getHostString() + ":" + inetSocketAddress.getPort())); + channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); + selector = new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, + time, "admin-client", - logContext, - apiVersions, + channelBuilder, + logContext); + bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( time, + bootstrapAddresses, + ClientDnsLookup.forConfig(config.getString(AdminClientConfig.CLIENT_DNS_LOOKUP_CONFIG)), + 2 * 60 * 1000 + ); + networkClient = new NetworkClient(metadataManager.updater(), + null, + selector, + clientId, 1, - (int) TimeUnit.HOURS.toMillis(1), + config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG), + config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG), + config.getInt(CommonClientConfigs.SEND_BUFFER_CONFIG), + config.getInt(CommonClientConfigs.RECEIVE_BUFFER_CONFIG), + config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG), + config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG), + config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG), + time, + true, + apiVersions, null, - metadataManager.updater(), + logContext, (hostResolver == null) ? new DefaultHostResolver() : hostResolver, null, - clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); + config.getLong(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG), + MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), + Optional.of(bootstrapConfiguration) + ); + return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, timeoutProcessorFactory, logContext, clientTelemetryReporter); } catch (Throwable exc) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java index 0ac5419991e1b..43242e7fcdbf7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -148,6 +149,16 @@ public void rebootstrap(long now) { AdminMetadataManager.this.rebootstrap(now); } + @Override + public boolean isBootstrapped() { + return false; + } + + @Override + public void bootstrap(List addresses) { + // do nothing + } + @Override public void close() { } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 7ad52c2122627..7d326297d0930 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -133,6 +133,11 @@ public class ConsumerConfig extends AbstractConfig { /** client.dns.lookup */ public static final String CLIENT_DNS_LOOKUP_CONFIG = CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG; + /** + * bootstrap.resolve.timeout.ms + */ + public static final String BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG = CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG; + /** * enable.auto.commit */ @@ -426,6 +431,11 @@ public class ConsumerConfig extends AbstractConfig { ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY.toString()), Importance.MEDIUM, CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC) + .define(BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, + Type.LONG, + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC) .define(GROUP_ID_CONFIG, Type.STRING, null, Importance.HIGH, GROUP_ID_DOC) .define(GROUP_INSTANCE_ID_CONFIG, Type.STRING, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java index c227a9511b7b6..fa549df9c540a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java @@ -63,7 +63,6 @@ import org.slf4j.Logger; import org.slf4j.event.Level; -import java.net.InetSocketAddress; import java.time.Duration; import java.util.Arrays; import java.util.Collection; @@ -188,8 +187,6 @@ public class ClassicKafkaConsumer implements ConsumerDelegate { interceptorList, Arrays.asList(this.deserializers.keyDeserializer(), this.deserializers.valueDeserializer())); this.metadata = new ConsumerMetadata(config, subscriptions, logContext, clusterResourceListeners); - List addresses = ClientUtils.parseAndValidateAddresses(config); - this.metadata.bootstrap(addresses); FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics); FetchConfig fetchConfig = new FetchConfig(config); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 0ea2e66f57d6a..9fa174c131b90 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -85,7 +85,6 @@ import org.slf4j.Logger; -import java.net.InetSocketAddress; import java.time.Duration; import java.util.Arrays; import java.util.Collections; @@ -452,7 +451,6 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali transactionManager, new BufferPool(this.totalMemorySize, batchSize, metrics, time, PRODUCER_METRIC_GROUP_NAME)); - List addresses = ClientUtils.parseAndValidateAddresses(config); if (metadata != null) { this.metadata = metadata; } else { @@ -463,7 +461,6 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali logContext, clusterResourceListeners, Time.SYSTEM); - this.metadata.bootstrap(addresses); } this.errors = this.metrics.sensor("errors"); this.sender = newSender(logContext, kafkaClient, this.metadata); diff --git a/clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java b/clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java new file mode 100644 index 0000000000000..db92127a0553a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java @@ -0,0 +1,35 @@ +/* + * 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.kafka.common.network; + +import org.apache.kafka.common.KafkaException; + +/** + * Indicates that the {@link org.apache.kafka.clients.NetworkClient} was unable to resolve a DNS address within + * the time specified by {@link org.apache.kafka.clients.CommonClientConfigs#BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG} + * + * @see org.apache.kafka.clients.CommonClientConfigs + */ +public class BootstrapResolutionException extends KafkaException { + public BootstrapResolutionException(String message) { + super(message); + } + + public BootstrapResolutionException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index fca0a9ca2121b..132c22146dbea 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -588,6 +588,11 @@ public boolean active() { return active; } + @Override + public boolean isBootstrapped() { + return false; + } + @Override public void close() { active = false; diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index e8dcf5843dcb8..bb8946ef54f90 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -91,26 +91,32 @@ import static org.mockito.Mockito.when; public class NetworkClientTest { + protected static final MockTime TIME = new MockTime(); + private static final List BOOTSTRAP_ADDRESSES = new ArrayList<>(Arrays.asList( + "127.0.0.1:8000", + "127.0.0.2:8000")); + private static List initialAddresses; + private static List newAddresses; + private static NetworkClient.BootstrapConfiguration bootstrapConfiguration = + new NetworkClient.BootstrapConfiguration( + TIME, + BOOTSTRAP_ADDRESSES, + ClientDnsLookup.USE_ALL_DNS_IPS, + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + 100000000); protected final int defaultRequestTimeoutMs = 1000; - protected final MockTime time = new MockTime(); - protected final MockSelector selector = new MockSelector(time); + protected final MockSelector selector = new MockSelector(TIME); protected final Node node = TestUtils.singletonCluster().nodes().iterator().next(); protected final long reconnectBackoffMsTest = 10 * 1000; protected final long reconnectBackoffMaxMsTest = 10 * 10000; protected final long connectionSetupTimeoutMsTest = 5 * 1000; protected final long connectionSetupTimeoutMaxMsTest = 127 * 1000; - private final int reconnectBackoffExpBase = ClusterConnectionStates.RECONNECT_BACKOFF_EXP_BASE; - private final double reconnectBackoffJitter = ClusterConnectionStates.RECONNECT_BACKOFF_JITTER; private final TestMetadataUpdater metadataUpdater = new TestMetadataUpdater(Collections.singletonList(node)); private final NetworkClient client = createNetworkClient(reconnectBackoffMaxMsTest); private final NetworkClient clientWithNoExponentialBackoff = createNetworkClient(reconnectBackoffMsTest); private final NetworkClient clientWithStaticNodes = createNetworkClientWithStaticNodes(); private final NetworkClient clientWithNoVersionDiscovery = createNetworkClientWithNoVersionDiscovery(); - private static ArrayList initialAddresses; - private static ArrayList newAddresses; - static { try { initialAddresses = new ArrayList<>(Arrays.asList( @@ -131,16 +137,16 @@ public class NetworkClientTest { private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) { return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); } private NetworkClient createNetworkClientWithMaxInFlightRequestsPerConnection( int maxInFlightRequestsPerConnection, long reconnectBackoffMaxMs) { return new NetworkClient(selector, metadataUpdater, "mock", maxInFlightRequestsPerConnection, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); } private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoffMaxMs, long connectionSetupTimeoutMsTest, int nodeNumber) { @@ -148,41 +154,47 @@ private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoff TestMetadataUpdater metadataUpdater = new TestMetadataUpdater(nodes); return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); } private NetworkClient createNetworkClientWithStaticNodes() { return new NetworkClient(selector, metadataUpdater, "mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, - connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE); + connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); } private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata) { return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); } private NetworkClient createNetworkClientWithNoVersionDiscovery() { return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, - connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE); + connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); } @BeforeEach public void setup() { selector.reset(); + bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + TIME, + BOOTSTRAP_ADDRESSES, + ClientDnsLookup.USE_ALL_DNS_IPS, + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ); } @Test public void testSendToUnreadyNode() { MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.singletonList("test"), true); - long now = time.milliseconds(); + long now = TIME.milliseconds(); ClientRequest request = client.newClientRequest("5", builder, now, false); assertThrows(IllegalStateException.class, () -> client.send(request, now)); } @@ -205,22 +217,22 @@ public void testSimpleRequestResponseWithNoBrokerDiscovery() { @Test public void testDnsLookupFailure() { /* Fail cleanly when the node has a bad hostname */ - assertFalse(client.ready(new Node(1234, "badhost", 1234), time.milliseconds())); + assertFalse(client.ready(new Node(1234, "badhost", 1234), TIME.milliseconds())); } @Test public void testClose() { - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); awaitReady(client, node); - client.poll(1, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); + client.poll(1, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds()), "The client should be ready"); ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) .setTimeoutMs(1000)); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); - client.send(request, time.milliseconds()); + ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true); + client.send(request, TIME.milliseconds()); assertEquals(1, client.inFlightRequestCount(node.idString()), "There should be 1 in-flight request after send"); assertTrue(client.hasInFlightRequests(node.idString())); @@ -239,7 +251,7 @@ public void testUnsupportedVersionDuringInternalMetadataRequest() { // disabling auto topic creation for versions less than 4 is not supported MetadataRequest.Builder builder = new MetadataRequest.Builder(topics, false, (short) 3); - client.sendInternalMetadataRequest(builder, node.idString(), time.milliseconds()); + client.sendInternalMetadataRequest(builder, node.idString(), TIME.milliseconds()); assertEquals(UnsupportedVersionException.class, metadataUpdater.getAndClearFailure().getClass()); } @@ -257,42 +269,42 @@ public synchronized void rebootstrap() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), rebootstrapTriggerMs, - MetadataRecoveryStrategy.REBOOTSTRAP); + MetadataRecoveryStrategy.REBOOTSTRAP, Optional.of(bootstrapConfiguration)); MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); metadata.bootstrap(Collections.singletonList(new InetSocketAddress("localhost", 9999))); metadata.requestUpdate(true); - client.poll(0, time.milliseconds()); - time.sleep(rebootstrapTriggerMs + 1); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); + TIME.sleep(rebootstrapTriggerMs + 1); + client.poll(0, TIME.milliseconds()); assertEquals(1, rebootstrapCount.get()); - time.sleep(1); - client.poll(0, time.milliseconds()); + TIME.sleep(1); + client.poll(0, TIME.milliseconds()); assertEquals(1, rebootstrapCount.get()); metadata.requestUpdate(true); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(1, rebootstrapCount.get()); - metadataUpdater.handleFailedRequest(time.milliseconds(), Optional.of(new KafkaException())); - client.poll(0, time.milliseconds()); + metadataUpdater.handleFailedRequest(TIME.milliseconds(), Optional.of(new KafkaException())); + client.poll(0, TIME.milliseconds()); assertEquals(1, rebootstrapCount.get()); - time.sleep(rebootstrapTriggerMs); - client.poll(0, time.milliseconds()); + TIME.sleep(rebootstrapTriggerMs); + client.poll(0, TIME.milliseconds()); assertEquals(2, rebootstrapCount.get()); metadata.requestUpdate(true); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(2, rebootstrapCount.get()); MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); + ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true); MetadataResponse rebootstrapResponse = (MetadataResponse) builder.build().getErrorResponse(0, new RebootstrapRequiredException("rebootstrap")); - metadataUpdater.handleSuccessfulResponse(request.makeHeader(builder.latestAllowedVersion()), time.milliseconds(), rebootstrapResponse); + metadataUpdater.handleSuccessfulResponse(request.makeHeader(builder.latestAllowedVersion()), TIME.milliseconds(), rebootstrapResponse); assertEquals(2, rebootstrapCount.get()); - time.sleep(50); - client.poll(0, time.milliseconds()); + TIME.sleep(50); + client.poll(0, TIME.milliseconds()); assertEquals(3, rebootstrapCount.get()); } @@ -312,31 +324,31 @@ public synchronized void rebootstrap() { metadata.bootstrap(Collections.singletonList(new InetSocketAddress("localhost", 9999))); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), - rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP); + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP, Optional.of(bootstrapConfiguration)); MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); List nodes = metadata.fetch().nodes(); nodes.forEach(node -> { - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); awaitReady(client, node); }); // Queue a request sendEmptyProduceRequest(client, nodes.get(0).idString()); - List responses = client.poll(0, time.milliseconds()); + List responses = client.poll(0, TIME.milliseconds()); assertEquals(0, responses.size()); assertEquals(1, client.inFlightRequestCount()); // Trigger rebootstrap metadata.requestUpdate(true); - time.sleep(refreshBackoffMs); - responses = client.poll(0, time.milliseconds()); + TIME.sleep(refreshBackoffMs); + responses = client.poll(0, TIME.milliseconds()); assertEquals(0, responses.size()); assertEquals(2, client.inFlightRequestCount()); - time.sleep(rebootstrapTriggerMs + 1); - responses = client.poll(0, time.milliseconds()); + TIME.sleep(rebootstrapTriggerMs + 1); + responses = client.poll(0, TIME.milliseconds()); // Verify that inflight produce request was aborted with disconnection assertEquals(1, responses.size()); @@ -356,15 +368,15 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = networkClient.newClientRequest(node.idString(), builder, time.milliseconds(), + ClientRequest request = networkClient.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, defaultRequestTimeoutMs, handler); - networkClient.send(request, time.milliseconds()); - networkClient.poll(1, time.milliseconds()); + networkClient.send(request, TIME.milliseconds()); + networkClient.poll(1, TIME.milliseconds()); assertEquals(1, networkClient.inFlightRequestCount()); ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData()); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, requestVersion, request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); - List responses = networkClient.poll(1, time.milliseconds()); + List responses = networkClient.poll(1, TIME.milliseconds()); assertEquals(1, responses.size()); assertTrue(handler.executed, "The handler should have executed."); assertTrue(handler.response.hasResponse(), "Should have a response body."); @@ -387,18 +399,18 @@ private void awaitReady(NetworkClient client, Node node) { setExpectedApiVersionsResponse(TestUtils.defaultApiVersionsResponse( ApiMessageType.ListenerType.BROKER)); } - while (!client.ready(node, time.milliseconds())) - client.poll(1, time.milliseconds()); + while (!client.ready(node, TIME.milliseconds())) + client.poll(1, TIME.milliseconds()); selector.clear(); } @Test public void testInvalidApiVersionsRequest() { // initiate the connection - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); // handle the connection, send the ApiVersionsRequest - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // check that the ApiVersionsRequest has been initiated assertTrue(client.hasInFlightRequests(node.idString())); @@ -412,22 +424,22 @@ public void testInvalidApiVersionsRequest() { )); // handle completed receives - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); // various assertions - assertFalse(client.isReady(node, time.milliseconds())); + assertFalse(client.isReady(node, TIME.milliseconds())); } @Test public void testApiVersionsRequest() { // initiate the connection - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); // handle the connection, send the ApiVersionsRequest - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // check that the ApiVersionsRequest has been initiated assertTrue(client.hasInFlightRequests(node.idString())); @@ -436,28 +448,28 @@ public void testApiVersionsRequest() { delayedApiVersionsResponse(0, ApiKeys.API_VERSIONS.latestVersion(), defaultApiVersionsResponse()); // handle completed receives - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); // various assertions - assertTrue(client.isReady(node, time.milliseconds())); + assertTrue(client.isReady(node, TIME.milliseconds())); } @Test public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { // initiate the connection - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); // handle the connection, initiate first ApiVersionsRequest - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); // completes initiated sends - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(1, selector.completedSends().size()); ByteBuffer buffer = selector.completedSendBuffers().get(0).buffer(); @@ -479,7 +491,7 @@ public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { )); // handle ApiVersionResponse, initiate second ApiVersionRequest - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); @@ -493,7 +505,7 @@ public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { selector.completedReceives().clear(); // completes initiated sends - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // ApiVersionsRequest has been sent assertEquals(1, selector.completedSends().size()); @@ -507,29 +519,29 @@ public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { delayedApiVersionsResponse(1, (short) 0, defaultApiVersionsResponse()); // handle completed receives - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); assertEquals(1, selector.completedReceives().size()); // the client is ready - assertTrue(client.isReady(node, time.milliseconds())); + assertTrue(client.isReady(node, TIME.milliseconds())); } @Test public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() { // initiate the connection - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); // handle the connection, initiate first ApiVersionsRequest - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); // completes initiated sends - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(1, selector.completedSends().size()); ByteBuffer buffer = selector.completedSendBuffers().get(0).buffer(); @@ -545,7 +557,7 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() )); // handle ApiVersionResponse, initiate second ApiVersionRequest - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); @@ -559,7 +571,7 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() selector.completedReceives().clear(); // completes initiated sends - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // ApiVersionsRequest has been sent assertEquals(1, selector.completedSends().size()); @@ -573,14 +585,14 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() delayedApiVersionsResponse(1, (short) 0, defaultApiVersionsResponse()); // handle completed receives - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); assertEquals(1, selector.completedReceives().size()); // the client is ready - assertTrue(client.isReady(node, time.milliseconds())); + assertTrue(client.isReady(node, TIME.milliseconds())); } @Test @@ -611,7 +623,7 @@ public void testDefaultRequestTimeout() { private void testRequestTimeout(int requestTimeoutMs) { Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); NetworkClient client = createNetworkClientWithNoVersionDiscovery(metadata); @@ -637,20 +649,20 @@ private ClientResponse produce(NetworkClient client, int requestTimeoutMs, boole .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, requestTimeoutMs, handler); - client.send(request, time.milliseconds()); + client.send(request, TIME.milliseconds()); if (shouldEmulateTimeout) { // For a delay of slightly more than our timeout threshold to emulate the request timing out. - time.sleep(requestTimeoutMs + 1); + TIME.sleep(requestTimeoutMs + 1); } else { ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData()); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, PRODUCE.latestVersion(), request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); } - List responses = client.poll(0, time.milliseconds()); + List responses = client.poll(0, TIME.milliseconds()); assertEquals(1, responses.size()); return responses.get(0); } @@ -663,18 +675,18 @@ public void testConnectionSetupTimeout() { final Node node0 = cluster.nodeById(0); final Node node1 = cluster.nodeById(1); - client.ready(node0, time.milliseconds()); + client.ready(node0, TIME.milliseconds()); selector.serverConnectionBlocked(node0.idString()); - client.ready(node1, time.milliseconds()); + client.ready(node1, TIME.milliseconds()); selector.serverConnectionBlocked(node1.idString()); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertFalse(client.connectionFailed(node), "The connections should not fail before the socket connection setup timeout elapsed"); - time.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); - client.poll(0, time.milliseconds()); + TIME.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); + client.poll(0, TIME.milliseconds()); assertTrue(client.connectionFailed(node), "Expected the connections to fail due to the socket connection setup timeout"); } @@ -691,21 +703,21 @@ public void testConnectionTimeoutAfterThrottling() { .setAcks((short) 1) .setTimeoutMs(timeoutMs)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest r1 = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + ClientRequest r1 = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(r1, time.milliseconds()); - client.poll(0, time.milliseconds()); + client.send(r1, TIME.milliseconds()); + client.poll(0, TIME.milliseconds()); // Throttle long enough to ensure other inFlight requests timeout. ProduceResponse pr = new ProduceResponse(new ProduceResponseData().setThrottleTimeMs(timeoutMs)); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(pr, requestVersion, r1.correlationId()); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); - ClientRequest r2 = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + ClientRequest r2 = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(r2, time.milliseconds()); - time.sleep(timeoutMs); - client.poll(0, time.milliseconds()); + client.send(r2, TIME.milliseconds()); + TIME.sleep(timeoutMs); + client.poll(0, TIME.milliseconds()); assertEquals(1, client.inFlightRequestCount(node.idString())); assertFalse(client.connectionFailed(node), "Connection should not have failed due to the extra time spent throttling."); @@ -723,29 +735,29 @@ public void testConnectionThrottling() { .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, + ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(request, time.milliseconds()); - client.poll(1, time.milliseconds()); + client.send(request, TIME.milliseconds()); + client.poll(1, TIME.milliseconds()); int throttleTime = 100; ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData().setThrottleTimeMs(throttleTime)); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, requestVersion, request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); - client.poll(1, time.milliseconds()); + client.poll(1, TIME.milliseconds()); // The connection is not ready due to throttling. - assertFalse(client.ready(node, time.milliseconds())); - assertEquals(100, client.throttleDelayMs(node, time.milliseconds())); + assertFalse(client.ready(node, TIME.milliseconds())); + assertEquals(100, client.throttleDelayMs(node, TIME.milliseconds())); // After 50ms, the connection is not ready yet. - time.sleep(50); - assertFalse(client.ready(node, time.milliseconds())); - assertEquals(50, client.throttleDelayMs(node, time.milliseconds())); + TIME.sleep(50); + assertFalse(client.ready(node, TIME.milliseconds())); + assertEquals(50, client.throttleDelayMs(node, TIME.milliseconds())); // After another 50ms, the throttling is done and the connection becomes ready again. - time.sleep(50); - assertTrue(client.ready(node, time.milliseconds())); - assertEquals(0, client.throttleDelayMs(node, time.milliseconds())); + TIME.sleep(50); + assertTrue(client.ready(node, TIME.milliseconds())); + assertEquals(0, client.throttleDelayMs(node, TIME.milliseconds())); } private int sendEmptyProduceRequest() { @@ -758,9 +770,9 @@ private int sendEmptyProduceRequest(NetworkClient client, String nodeId) { .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, + ClientRequest request = client.newClientRequest(nodeId, builder, TIME.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(request, time.milliseconds()); + client.send(request, TIME.milliseconds()); return request.correlationId(); } @@ -776,31 +788,31 @@ private void sendThrottledProduceResponse(int correlationId, int throttleMs, sho @Test public void testLeastLoadedNode() { - client.ready(node, time.milliseconds()); - assertFalse(client.isReady(node, time.milliseconds())); - LeastLoadedNode leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + client.ready(node, TIME.milliseconds()); + assertFalse(client.isReady(node, TIME.milliseconds())); + LeastLoadedNode leastLoadedNode = client.leastLoadedNode(TIME.milliseconds()); assertEquals(node, leastLoadedNode.node()); assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); awaitReady(client, node); - client.poll(1, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); + client.poll(1, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds()), "The client should be ready"); // leastloadednode should be our single node - leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + leastLoadedNode = client.leastLoadedNode(TIME.milliseconds()); assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); Node leastNode = leastLoadedNode.node(); assertEquals(leastNode.id(), node.id(), "There should be one leastloadednode"); // sleep for longer than reconnect backoff - time.sleep(reconnectBackoffMsTest); + TIME.sleep(reconnectBackoffMsTest); // CLOSE node selector.serverDisconnect(node.idString()); - client.poll(1, time.milliseconds()); - assertFalse(client.ready(node, time.milliseconds()), "After we forced the disconnection the client is no longer ready."); - leastLoadedNode = client.leastLoadedNode(time.milliseconds()); + client.poll(1, TIME.milliseconds()); + assertFalse(client.ready(node, TIME.milliseconds()), "After we forced the disconnection the client is no longer ready."); + leastLoadedNode = client.leastLoadedNode(TIME.milliseconds()); assertFalse(leastLoadedNode.hasNodeAvailableOrConnectionReady()); assertNull(leastLoadedNode.node(), "There should be NO leastloadednode"); } @@ -810,7 +822,7 @@ public void testHasNodeAvailableOrConnectionReady() { NetworkClient client = createNetworkClientWithMaxInFlightRequestsPerConnection(1, reconnectBackoffMaxMsTest); awaitReady(client, node); - long now = time.milliseconds(); + long now = TIME.milliseconds(); LeastLoadedNode leastLoadedNode = client.leastLoadedNode(now); assertEquals(node, leastLoadedNode.node()); assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); @@ -832,13 +844,13 @@ public void testLeastLoadedNodeProvideDisconnectedNodesPrioritizedByLastConnecti Set providedNodeIds = new HashSet<>(); for (int i = 0; i < nodeNumber * 10; i++) { - Node node = client.leastLoadedNode(time.milliseconds()).node(); + Node node = client.leastLoadedNode(TIME.milliseconds()).node(); assertNotNull(node, "Should provide a node"); providedNodeIds.add(node); - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); client.disconnect(node.idString()); - time.sleep(connectionSetupTimeoutMsTest + 1); - client.poll(0, time.milliseconds()); + TIME.sleep(connectionSetupTimeoutMsTest + 1); + client.poll(0, TIME.milliseconds()); // Define a round as nodeNumber of nodes have been provided // In each round every node should be provided exactly once if ((i + 1) % nodeNumber == 0) { @@ -854,7 +866,7 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); Metadata metadata = new Metadata(refreshBackoffMs, refreshBackoffMs, 5000, new LogContext(), new ClusterResourceListeners()); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); Cluster cluster = metadata.fetch(); Node node1 = cluster.nodes().get(0); @@ -865,18 +877,18 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { awaitReady(client, node1); metadata.requestUpdate(true); - time.sleep(refreshBackoffMs); + TIME.sleep(refreshBackoffMs); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); Optional nodeWithPendingMetadataOpt = cluster.nodes().stream() .filter(node -> client.hasInFlightRequests(node.idString())) .findFirst(); assertEquals(Optional.of(node1), nodeWithPendingMetadataOpt); - assertFalse(client.ready(node2, time.milliseconds())); + assertFalse(client.ready(node2, TIME.milliseconds())); selector.serverAuthenticationFailed(node2.idString()); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertNotNull(client.authenticationException(node2)); ByteBuffer requestBuffer = selector.completedSendBuffers().get(0).buffer(); @@ -887,30 +899,30 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { selector.delayedReceive(new DelayedReceive(node1.idString(), new NetworkReceive(node1.idString(), responseBuffer))); int initialUpdateVersion = metadata.updateVersion(); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(initialUpdateVersion + 1, metadata.updateVersion()); } @Test public void testLeastLoadedNodeConsidersThrottledConnections() { - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); awaitReady(client, node); - client.poll(1, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); + client.poll(1, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds()), "The client should be ready"); int correlationId = sendEmptyProduceRequest(); - client.poll(1, time.milliseconds()); + client.poll(1, TIME.milliseconds()); sendThrottledProduceResponse(correlationId, 100, PRODUCE.latestVersion()); - client.poll(1, time.milliseconds()); + client.poll(1, TIME.milliseconds()); // leastloadednode should return null since the node is throttled - assertNull(client.leastLoadedNode(time.milliseconds()).node()); + assertNull(client.leastLoadedNode(TIME.milliseconds()).node()); } @Test public void testConnectionDelayWithNoExponentialBackoff() { - long now = time.milliseconds(); + long now = TIME.milliseconds(); long delay = clientWithNoExponentialBackoff.connectionDelay(node, now); assertEquals(0, delay); @@ -920,7 +932,7 @@ public void testConnectionDelayWithNoExponentialBackoff() { public void testConnectionDelayConnectedWithNoExponentialBackoff() { awaitReady(clientWithNoExponentialBackoff, node); - long now = time.milliseconds(); + long now = TIME.milliseconds(); long delay = clientWithNoExponentialBackoff.connectionDelay(node, now); assertEquals(Long.MAX_VALUE, delay); @@ -931,19 +943,19 @@ public void testConnectionDelayDisconnectedWithNoExponentialBackoff() { awaitReady(clientWithNoExponentialBackoff, node); selector.serverDisconnect(node.idString()); - clientWithNoExponentialBackoff.poll(defaultRequestTimeoutMs, time.milliseconds()); - long delay = clientWithNoExponentialBackoff.connectionDelay(node, time.milliseconds()); + clientWithNoExponentialBackoff.poll(defaultRequestTimeoutMs, TIME.milliseconds()); + long delay = clientWithNoExponentialBackoff.connectionDelay(node, TIME.milliseconds()); assertEquals(reconnectBackoffMsTest, delay); // Sleep until there is no connection delay - time.sleep(delay); - assertEquals(0, clientWithNoExponentialBackoff.connectionDelay(node, time.milliseconds())); + TIME.sleep(delay); + assertEquals(0, clientWithNoExponentialBackoff.connectionDelay(node, TIME.milliseconds())); // Start connecting and disconnect before the connection is established - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); selector.serverDisconnect(node.idString()); - client.poll(defaultRequestTimeoutMs, time.milliseconds()); + client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); // Second attempt should have the same behaviour as exponential backoff is disabled assertEquals(reconnectBackoffMsTest, delay); @@ -951,7 +963,7 @@ public void testConnectionDelayDisconnectedWithNoExponentialBackoff() { @Test public void testConnectionDelay() { - long now = time.milliseconds(); + long now = TIME.milliseconds(); long delay = client.connectionDelay(node, now); assertEquals(0, delay); @@ -961,7 +973,7 @@ public void testConnectionDelay() { public void testConnectionDelayConnected() { awaitReady(client, node); - long now = time.milliseconds(); + long now = TIME.milliseconds(); long delay = client.connectionDelay(node, now); assertEquals(Long.MAX_VALUE, delay); @@ -973,24 +985,24 @@ public void testConnectionDelayDisconnected() { // First disconnection selector.serverDisconnect(node.idString()); - client.poll(defaultRequestTimeoutMs, time.milliseconds()); - long delay = client.connectionDelay(node, time.milliseconds()); + client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); + long delay = client.connectionDelay(node, TIME.milliseconds()); long expectedDelay = reconnectBackoffMsTest; double jitter = 0.3; assertEquals(expectedDelay, delay, expectedDelay * jitter); // Sleep until there is no connection delay - time.sleep(delay); - assertEquals(0, client.connectionDelay(node, time.milliseconds())); + TIME.sleep(delay); + assertEquals(0, client.connectionDelay(node, TIME.milliseconds())); // Start connecting and disconnect before the connection is established - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); selector.serverDisconnect(node.idString()); - client.poll(defaultRequestTimeoutMs, time.milliseconds()); + client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); // Second attempt should take twice as long with twice the jitter expectedDelay = Math.round(delay * 2); - delay = client.connectionDelay(node, time.milliseconds()); + delay = client.connectionDelay(node, TIME.milliseconds()); jitter = 0.6; assertEquals(expectedDelay, delay, expectedDelay * jitter); } @@ -1002,7 +1014,7 @@ public void testDisconnectDuringUserMetadataRequest() { awaitReady(client, node); MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); - long now = time.milliseconds(); + long now = TIME.milliseconds(); ClientRequest request = client.newClientRequest(node.idString(), builder, now, true); client.send(request, now); client.poll(defaultRequestTimeoutMs, now); @@ -1011,7 +1023,7 @@ public void testDisconnectDuringUserMetadataRequest() { assertTrue(client.hasInFlightRequests()); selector.close(node.idString()); - List responses = client.poll(defaultRequestTimeoutMs, time.milliseconds()); + List responses = client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); assertEquals(1, responses.size()); assertTrue(responses.iterator().next().wasDisconnected()); } @@ -1019,6 +1031,7 @@ public void testDisconnectDuringUserMetadataRequest() { @Test public void testServerDisconnectAfterInternalApiVersionRequest() throws Exception { final long numIterations = 5; + int reconnectBackoffExpBase = ClusterConnectionStates.RECONNECT_BACKOFF_EXP_BASE; double reconnectBackoffMaxExp = Math.log(reconnectBackoffMaxMsTest / (double) Math.max(reconnectBackoffMsTest, 1)) / Math.log(reconnectBackoffExpBase); for (int i = 0; i < numIterations; i++) { @@ -1027,18 +1040,19 @@ public void testServerDisconnectAfterInternalApiVersionRequest() throws Exceptio selector.serverDisconnect(node.idString()); // The failed ApiVersion request should not be forwarded to upper layers - List responses = client.poll(0, time.milliseconds()); + List responses = client.poll(0, TIME.milliseconds()); assertFalse(client.hasInFlightRequests(node.idString())); assertTrue(responses.isEmpty()); long expectedBackoff = Math.round(Math.pow(reconnectBackoffExpBase, Math.min(i, reconnectBackoffMaxExp)) * reconnectBackoffMsTest); - long delay = client.connectionDelay(node, time.milliseconds()); + long delay = client.connectionDelay(node, TIME.milliseconds()); + double reconnectBackoffJitter = ClusterConnectionStates.RECONNECT_BACKOFF_JITTER; assertEquals(expectedBackoff, delay, reconnectBackoffJitter * expectedBackoff); if (i == numIterations - 1) { break; } - time.sleep(delay + 1); + TIME.sleep(delay + 1); } } @@ -1049,7 +1063,7 @@ public void testClientDisconnectAfterInternalApiVersionRequest() throws Exceptio assertFalse(client.hasInFlightRequests(node.idString())); // The failed ApiVersion request should not be forwarded to upper layers - List responses = client.poll(0, time.milliseconds()); + List responses = client.poll(0, TIME.milliseconds()); assertTrue(responses.isEmpty()); } @@ -1057,11 +1071,11 @@ public void testClientDisconnectAfterInternalApiVersionRequest() throws Exceptio public void testDisconnectWithMultipleInFlights() { NetworkClient client = this.clientWithNoVersionDiscovery; awaitReady(client, node); - assertTrue(client.isReady(node, time.milliseconds()), + assertTrue(client.isReady(node, TIME.milliseconds()), "Expected NetworkClient to be ready to send to node " + node.idString()); MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); - long now = time.milliseconds(); + long now = TIME.milliseconds(); final List callbackResponses = new ArrayList<>(); RequestCompletionHandler callback = callbackResponses::add; @@ -1081,7 +1095,7 @@ public void testDisconnectWithMultipleInFlights() { client.disconnect(node.idString()); - List responses = client.poll(0, time.milliseconds()); + List responses = client.poll(0, TIME.milliseconds()); assertEquals(2, responses.size()); assertEquals(responses, callbackResponses); assertEquals(0, client.inFlightRequestCount()); @@ -1100,22 +1114,22 @@ public void testDisconnectWithMultipleInFlights() { @Test public void testCallDisconnect() { awaitReady(client, node); - assertTrue(client.isReady(node, time.milliseconds()), + assertTrue(client.isReady(node, TIME.milliseconds()), "Expected NetworkClient to be ready to send to node " + node.idString()); assertFalse(client.connectionFailed(node), "Did not expect connection to node " + node.idString() + " to be failed"); client.disconnect(node.idString()); - assertFalse(client.isReady(node, time.milliseconds()), + assertFalse(client.isReady(node, TIME.milliseconds()), "Expected node " + node.idString() + " to be disconnected."); assertTrue(client.connectionFailed(node), "Expected connection to node " + node.idString() + " to be failed after disconnect"); - assertFalse(client.canConnect(node, time.milliseconds())); + assertFalse(client.canConnect(node, TIME.milliseconds())); // ensure disconnect does not reset backoff period if already disconnected - time.sleep(reconnectBackoffMaxMsTest); - assertTrue(client.canConnect(node, time.milliseconds())); + TIME.sleep(reconnectBackoffMaxMsTest); + assertTrue(client.canConnect(node, TIME.milliseconds())); client.disconnect(node.idString()); - assertTrue(client.canConnect(node, time.milliseconds())); + assertTrue(client.canConnect(node, TIME.milliseconds())); } @Test @@ -1134,7 +1148,7 @@ public void testReconnectAfterAddressChange() { initialAddresses.toArray(new InetAddress[0]), newAddresses.toArray(new InetAddress[0])); AtomicInteger initialAddressConns = new AtomicInteger(); AtomicInteger newAddressConns = new AtomicInteger(); - MockSelector selector = new MockSelector(this.time, inetSocketAddress -> { + MockSelector selector = new MockSelector(TIME, inetSocketAddress -> { InetAddress inetAddress = inetSocketAddress.getAddress(); if (initialAddresses.contains(inetAddress)) { initialAddressConns.incrementAndGet(); @@ -1151,35 +1165,35 @@ public void testReconnectAfterAddressChange() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE); + TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); // Connect to one the initial addresses, then change the addresses and disconnect - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); // First poll should try to update the node but couldn't because node remains in connecting state // i.e. connection handling is completed after telemetry update. assertNull(client.telemetryConnectedNode()); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); mockHostResolver.changeAddresses(); selector.serverDisconnect(node.idString()); - client.poll(0, time.milliseconds()); - assertFalse(client.isReady(node, time.milliseconds())); + client.poll(0, TIME.milliseconds()); + assertFalse(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); - time.sleep(reconnectBackoffMaxMsTest); - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + TIME.sleep(reconnectBackoffMaxMsTest); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); // We should have tried to connect to one initial address and one new address, and resolved DNS twice @@ -1195,7 +1209,7 @@ public void testFailedConnectionToFirstAddress() { initialAddresses.toArray(new InetAddress[0]), newAddresses.toArray(new InetAddress[0])); AtomicInteger initialAddressConns = new AtomicInteger(); AtomicInteger newAddressConns = new AtomicInteger(); - MockSelector selector = new MockSelector(this.time, inetSocketAddress -> { + MockSelector selector = new MockSelector(this.TIME, inetSocketAddress -> { InetAddress inetAddress = inetSocketAddress.getAddress(); if (initialAddresses.contains(inetAddress)) { initialAddressConns.incrementAndGet(); @@ -1212,26 +1226,26 @@ public void testFailedConnectionToFirstAddress() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE); + TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); // First connection attempt should fail - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertFalse(client.isReady(node, time.milliseconds())); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertFalse(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); // Second connection attempt should succeed - time.sleep(reconnectBackoffMaxMsTest); - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + TIME.sleep(reconnectBackoffMaxMsTest); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); // Next client poll after handling connection setup should update telemetry node. - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); // We should have tried to connect to two of the initial addresses, none of the new address, and should @@ -1248,7 +1262,7 @@ public void testFailedConnectionToFirstAddressAfterReconnect() { initialAddresses.toArray(new InetAddress[0]), newAddresses.toArray(new InetAddress[0])); AtomicInteger initialAddressConns = new AtomicInteger(); AtomicInteger newAddressConns = new AtomicInteger(); - MockSelector selector = new MockSelector(this.time, inetSocketAddress -> { + MockSelector selector = new MockSelector(TIME, inetSocketAddress -> { InetAddress inetAddress = inetSocketAddress.getAddress(); if (initialAddresses.contains(inetAddress)) { initialAddressConns.incrementAndGet(); @@ -1265,43 +1279,43 @@ public void testFailedConnectionToFirstAddressAfterReconnect() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE); + TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); // Connect to one the initial addresses, then change the addresses and disconnect - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); // Next client poll after handling connection setup should update telemetry node. - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); mockHostResolver.changeAddresses(); selector.serverDisconnect(node.idString()); - client.poll(0, time.milliseconds()); - assertFalse(client.isReady(node, time.milliseconds())); + client.poll(0, TIME.milliseconds()); + assertFalse(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); // First connection attempt to new addresses should fail - time.sleep(reconnectBackoffMaxMsTest); - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertFalse(client.isReady(node, time.milliseconds())); + TIME.sleep(reconnectBackoffMaxMsTest); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertFalse(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); // Second connection attempt to new addresses should succeed - time.sleep(reconnectBackoffMaxMsTest); - client.ready(node, time.milliseconds()); - time.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + TIME.sleep(reconnectBackoffMaxMsTest); + client.ready(node, TIME.milliseconds()); + TIME.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); assertNull(client.telemetryConnectedNode()); // Next client poll after handling connection setup should update telemetry node. - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); // We should have tried to connect to one of the initial addresses and two of the new addresses (the first one @@ -1317,32 +1331,32 @@ public void testCloseConnectingNode() { Cluster cluster = TestUtils.clusterWith(2); Node node0 = cluster.nodeById(0); Node node1 = cluster.nodeById(1); - client.ready(node0, time.milliseconds()); + client.ready(node0, TIME.milliseconds()); selector.serverConnectionBlocked(node0.idString()); - client.poll(1, time.milliseconds()); + client.poll(1, TIME.milliseconds()); client.close(node0.idString()); // Poll without any connections should return without exceptions - client.poll(0, time.milliseconds()); - assertFalse(NetworkClientUtils.isReady(client, node0, time.milliseconds())); - assertFalse(NetworkClientUtils.isReady(client, node1, time.milliseconds())); + client.poll(0, TIME.milliseconds()); + assertFalse(NetworkClientUtils.isReady(client, node0, TIME.milliseconds())); + assertFalse(NetworkClientUtils.isReady(client, node1, TIME.milliseconds())); // Connection to new node should work - client.ready(node1, time.milliseconds()); + client.ready(node1, TIME.milliseconds()); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(defaultApiVersionsResponse(), ApiKeys.API_VERSIONS.latestVersion(), 0); selector.delayedReceive(new DelayedReceive(node1.idString(), new NetworkReceive(node1.idString(), buffer))); - while (!client.ready(node1, time.milliseconds())) - client.poll(1, time.milliseconds()); - assertTrue(client.isReady(node1, time.milliseconds())); + while (!client.ready(node1, TIME.milliseconds())) + client.poll(1, TIME.milliseconds()); + assertTrue(client.isReady(node1, TIME.milliseconds())); selector.clear(); // New connection to node closed earlier should work - client.ready(node0, time.milliseconds()); + client.ready(node0, TIME.milliseconds()); buffer = RequestTestUtils.serializeResponseWithHeader(defaultApiVersionsResponse(), ApiKeys.API_VERSIONS.latestVersion(), 1); selector.delayedReceive(new DelayedReceive(node0.idString(), new NetworkReceive(node0.idString(), buffer))); - while (!client.ready(node0, time.milliseconds())) - client.poll(1, time.milliseconds()); - assertTrue(client.isReady(node0, time.milliseconds())); + while (!client.ready(node0, TIME.milliseconds())) + client.poll(1, TIME.milliseconds()); + assertTrue(client.isReady(node0, TIME.milliseconds())); } @Test @@ -1351,18 +1365,18 @@ public void testConnectionDoesNotRemainStuckInCheckingApiVersionsStateIfChannelN final Node node = cluster.nodeById(0); // Channel is ready by default so we mark it as not ready. - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); selector.channelNotReady(node.idString()); // Channel should not be ready. - client.poll(0, time.milliseconds()); - assertFalse(NetworkClientUtils.isReady(client, node, time.milliseconds())); + client.poll(0, TIME.milliseconds()); + assertFalse(NetworkClientUtils.isReady(client, node, TIME.milliseconds())); // Connection should time out if the channel does not become ready within // the connection setup timeout. This ensures that the client does not remain // stuck in the CHECKING_API_VERSIONS state. - time.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); - client.poll(0, time.milliseconds()); + TIME.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); + client.poll(0, TIME.milliseconds()); assertTrue(client.connectionFailed(node)); } @@ -1374,18 +1388,18 @@ public void testTelemetryRequest() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE); + TIME, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender, + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); // Send the ApiVersionsRequest - client.ready(node, time.milliseconds()); - client.poll(0, time.milliseconds()); + client.ready(node, TIME.milliseconds()); + client.poll(0, TIME.milliseconds()); assertNull(client.telemetryConnectedNode()); assertTrue(client.hasInFlightRequests(node.idString())); delayedApiVersionsResponse(0, ApiKeys.API_VERSIONS.latestVersion(), TestUtils.defaultApiVersionsResponse( ApiMessageType.ListenerType.BROKER)); // handle ApiVersionsResponse - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); selector.clear(); @@ -1399,8 +1413,8 @@ time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver() selector.completeReceive(new NetworkReceive(node.idString(), buffer)); // Initiate poll to send GetTelemetrySubscriptions request - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); assertEquals(node, client.telemetryConnectedNode()); verify(mockClientTelemetrySender, times(1)).handleResponse(any(GetTelemetrySubscriptionsResponse.class)); selector.clear(); @@ -1414,8 +1428,8 @@ time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver() selector.completeReceive(new NetworkReceive(node.idString(), pushBuffer)); // Initiate poll to send PushTelemetry request - client.poll(0, time.milliseconds()); - assertTrue(client.isReady(node, time.milliseconds())); + client.poll(0, TIME.milliseconds()); + assertTrue(client.isReady(node, TIME.milliseconds())); assertEquals(node, client.telemetryConnectedNode()); verify(mockClientTelemetrySender, times(1)).handleResponse(any(PushTelemetryResponse.class)); verify(mockClientTelemetrySender, times(4)).timeToNextUpdate(anyLong()); @@ -1428,12 +1442,12 @@ private RequestHeader parseHeader(ByteBuffer buffer) { } private void awaitInFlightApiVersionRequest() throws Exception { - client.ready(node, time.milliseconds()); + client.ready(node, TIME.milliseconds()); TestUtils.waitForCondition(() -> { - client.poll(0, time.milliseconds()); + client.poll(0, TIME.milliseconds()); return client.hasInFlightRequests(node.idString()); }, 1000, ""); - assertFalse(client.isReady(node, time.milliseconds())); + assertFalse(client.isReady(node, TIME.milliseconds())); } private ApiVersionsResponse defaultApiVersionsResponse() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index f8e53148b0100..60daacdceab56 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -1906,7 +1906,7 @@ public void testQuotaMetrics() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(), - MetadataRecoveryStrategy.NONE); + MetadataRecoveryStrategy.NONE, Optional.empty()); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.BROKER); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 93da0a4433af0..f3283eb277b9b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -1893,7 +1893,7 @@ public void testQuotaMetrics() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(), - MetadataRecoveryStrategy.NONE); + MetadataRecoveryStrategy.NONE, Optional.empty()); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.BROKER); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 5d15229a838a6..ce67053933e93 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -559,24 +559,6 @@ public void testNoSerializerProvided() { }); } - @Test - public void testConstructorFailureCloseResource() { - Properties props = new Properties(); - props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar.local:9999"); - props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); - - final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); - final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); - try (KafkaProducer ignored = new KafkaProducer<>(props, new ByteArraySerializer(), new ByteArraySerializer())) { - fail("should have caught an exception and returned"); - } catch (KafkaException e) { - assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); - assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); - assertEquals("Failed to construct kafka producer", e.getMessage()); - } - } - @Test public void testConstructorWithNotStringKey() { Properties props = new Properties(); @@ -1113,7 +1095,7 @@ private > void doTestHeaders(Class serializerCla metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, nowMs); KafkaProducer producer = kafkaProducer(configs, keySerializer, valueSerializer, metadata, - null, null, Time.SYSTEM); + new MockClient(new MockTime(), metadata), null, Time.SYSTEM); when(keySerializer.serialize(any(), any(), any())).then(invocation -> invocation.getArgument(2).getBytes()); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index a7ff324961516..9e01551995d21 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -111,6 +111,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; @@ -240,7 +241,7 @@ public void testQuotaMetrics() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, time, true, new ApiVersions(), throttleTimeSensor, logContext, - MetadataRecoveryStrategy.NONE); + MetadataRecoveryStrategy.NONE, Optional.empty()); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.BROKER); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index a283ce7cf3878..0bdc96e634dc9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.runtime.distributed; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.GroupRebalanceConfig; @@ -49,6 +50,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; @@ -106,6 +108,11 @@ public WorkerGroupMember(DistributedConfig config, metadata.bootstrap(addresses); String metricGrpPrefix = "connect"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + time, + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS); NetworkClient netClient = new NetworkClient( new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, channelBuilder, logContext), metadata, @@ -123,7 +130,8 @@ public WorkerGroupMember(DistributedConfig config, new ApiVersions(), logContext, config.getLong(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG), - MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)) + MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), + Optional.of(bootstrapConfiguration) ); this.client = new ConsumerNetworkClient( logContext, diff --git a/core/src/main/java/kafka/server/NetworkUtils.java b/core/src/main/java/kafka/server/NetworkUtils.java index 5f084bfd98904..b41c41c257dfc 100644 --- a/core/src/main/java/kafka/server/NetworkUtils.java +++ b/core/src/main/java/kafka/server/NetworkUtils.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.utils.Time; import java.util.Map; +import java.util.Optional; public class NetworkUtils { @@ -85,7 +86,8 @@ public static NetworkClient buildNetworkClient(String prefix, true, new ApiVersions(), logContext, - MetadataRecoveryStrategy.NONE + MetadataRecoveryStrategy.NONE, + Optional.empty() ); } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 21d2876d82437..26a5a7cde1e9d 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -38,6 +38,7 @@ import org.apache.kafka.server.common.RequestLocal import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler} +import java.util.Optional import scala.collection.{concurrent, immutable} import scala.jdk.CollectionConverters._ import scala.jdk.javaapi.OptionConverters @@ -98,7 +99,8 @@ object TransactionMarkerChannelManager { false, new ApiVersions, logContext, - MetadataRecoveryStrategy.NONE + MetadataRecoveryStrategy.NONE, + Optional.empty ) new TransactionMarkerChannelManager(config, diff --git a/core/src/main/scala/kafka/raft/KafkaRaftManager.scala b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala index 86950e1ce2540..41b0dea8c079e 100644 --- a/core/src/main/scala/kafka/raft/KafkaRaftManager.scala +++ b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala @@ -20,7 +20,7 @@ import java.io.File import java.net.InetSocketAddress import java.nio.file.Files import java.nio.file.Paths -import java.util.{OptionalInt, Collection => JCollection, Map => JMap} +import java.util.{Optional, OptionalInt, Collection => JCollection, Map => JMap} import java.util.concurrent.CompletableFuture import kafka.server.KafkaConfig import kafka.utils.CoreUtils @@ -253,7 +253,8 @@ class KafkaRaftManager[T]( discoverBrokerVersions, apiVersions, logContext, - MetadataRecoveryStrategy.NONE + MetadataRecoveryStrategy.NONE, + Optional.empty ) (controllerListenerName, networkClient) diff --git a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala index c8405be6c6e6f..51450103c00c2 100644 --- a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala +++ b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.{Node, Reconfigurable} import org.apache.kafka.common.requests.AbstractRequest.Builder import org.apache.kafka.server.network.BrokerEndPoint +import java.util.Optional import scala.jdk.CollectionConverters._ trait BlockingSend { @@ -95,7 +96,8 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint, false, new ApiVersions, logContext, - MetadataRecoveryStrategy.NONE + MetadataRecoveryStrategy.NONE, + Optional.empty ) (networkClient, reconfigurableChannelBuilder) } diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 0caa03ec05299..cd8af23dce407 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -158,7 +158,8 @@ class NodeToControllerChannelManagerImpl( true, apiVersions, logContext, - MetadataRecoveryStrategy.NONE + MetadataRecoveryStrategy.NONE, + Optional.empty ) } val threadName = s"${threadNamePrefix}to-controller-${channelName}-channel-manager" diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index 9090fe46e4756..92fbe1f52b0e5 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -57,6 +57,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -198,7 +199,8 @@ static AdminClient create(AbstractConfig config) { true, new ApiVersions(), logContext, - MetadataRecoveryStrategy.NONE); + MetadataRecoveryStrategy.NONE, + Optional.empty()); ConsumerNetworkClient highLevelClient = new ConsumerNetworkClient( logContext, networkClient, diff --git a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java index 3de356b55ee95..f1f7983957d2d 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java @@ -17,6 +17,7 @@ package org.apache.kafka.tools; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientUtils; @@ -672,6 +673,11 @@ private static class ReplicaFetcherBlockingSend { channelBuilder, logContext ); + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + time, + consumerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(consumerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + consumerConfig.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG)); this.networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -688,7 +694,8 @@ private static class ReplicaFetcherBlockingSend { false, new ApiVersions(), logContext, - MetadataRecoveryStrategy.forName(consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)) + MetadataRecoveryStrategy.forName(consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), + Optional.of(bootstrapConfiguration) ); } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index 9452c03ec9937..9254b1b20209e 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -52,6 +52,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.List; +import java.util.Optional; import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -179,7 +180,8 @@ public boolean tryConnect() { false, new ApiVersions(), logContext, - MetadataRecoveryStrategy.NONE)) { + MetadataRecoveryStrategy.NONE, + Optional.empty())) { NetworkClientUtils.awaitReady(client, targetNode, Time.SYSTEM, 500); } } From e1f8466e6e1dfe7e98c53a1a24a7db5b72e24b29 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 5 Dec 2025 17:16:05 +0000 Subject: [PATCH 02/30] fix the error test --- .../java/org/apache/kafka/clients/admin/KafkaAdminClient.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 7e0960865dea0..aaa77bb16eb57 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -549,6 +549,7 @@ static KafkaAdminClient createInternal( config.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), config.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), adminAddresses.usingBootstrapControllers()); + metadataManager.update(Cluster.bootstrap(adminAddresses.addresses()), time.milliseconds()); List reporters = CommonClientConfigs.metricsReporters(clientId, config); clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); clientTelemetryReporter.ifPresent(reporters::add); From 4a066205f513d75c38bb49cbc2a6666264b092a0 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Mon, 15 Dec 2025 15:29:51 +0000 Subject: [PATCH 03/30] Seperate config and state --- .../org/apache/kafka/clients/ClientUtils.java | 6 +- .../org/apache/kafka/clients/KafkaClient.java | 6 - .../apache/kafka/clients/NetworkClient.java | 106 ++++++++++-------- .../kafka/clients/admin/KafkaAdminClient.java | 53 ++------- .../org/apache/kafka/clients/MockClient.java | 5 - .../kafka/clients/NetworkClientTest.java | 26 ++--- .../internals/FetchRequestManagerTest.java | 4 +- .../consumer/internals/FetcherTest.java | 5 +- .../clients/producer/KafkaProducerTest.java | 20 +++- .../producer/internals/SenderTest.java | 4 +- .../distributed/WorkerGroupMember.java | 4 +- .../main/java/kafka/server/NetworkUtils.java | 10 +- .../TransactionMarkerChannelManager.scala | 8 +- .../scala/kafka/raft/KafkaRaftManager.scala | 12 +- .../kafka/server/BrokerBlockingSender.scala | 9 +- .../NodeToControllerChannelManager.scala | 8 +- .../kafka/tools/BrokerApiVersionsCommand.java | 8 +- .../kafka/tools/ReplicaVerificationTool.java | 5 +- .../workload/ConnectionStressWorker.java | 10 +- 19 files changed, 165 insertions(+), 144 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 1952020f47750..c3a9e971cec85 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -41,7 +41,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.closeQuietly; @@ -220,7 +219,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, ClientTelemetrySender clientTelemetrySender) { ChannelBuilder channelBuilder = null; Selector selector = null; - NetworkClient.BootstrapConfiguration bootstrapConfiguration = null; + NetworkClient.BootstrapConfiguration bootstrapConfiguration; try { channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); @@ -231,7 +230,6 @@ public static NetworkClient createNetworkClient(AbstractConfig config, channelBuilder, logContext); bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - time, config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS @@ -257,7 +255,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, clientTelemetrySender, config.getLong(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG), MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), - Optional.of(bootstrapConfiguration) + bootstrapConfiguration ); } catch (Throwable t) { closeQuietly(selector, "Selector"); diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 791ffaa5d638e..405644024c9bc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -212,10 +212,4 @@ ClientRequest newClientRequest(String nodeId, * was invoked for this client. */ boolean active(); - - /** - * Returns true if the client has already been bootstrapped. Returns false if the client has not - * been bootstrapped or if bootstrap servers must be resolved again. - */ - boolean isBootstrapped(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index d8f6242a48754..26bafbe9cbfa5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -139,7 +139,7 @@ private enum State { private final AtomicReference state; - private final BootstrapConfiguration bootstrapConfiguration; + private final BootstrapState bootstrapState; private final TelemetrySender telemetrySender; @@ -159,7 +159,7 @@ public NetworkClient(Selectable selector, ApiVersions apiVersions, LogContext logContext, MetadataRecoveryStrategy metadataRecoveryStrategy, - Optional bootstrapConfiguration) { + BootstrapConfiguration bootstrapConfiguration) { this(selector, metadata, clientId, @@ -197,7 +197,7 @@ public NetworkClient(Selectable selector, LogContext logContext, long rebootstrapTriggerMs, MetadataRecoveryStrategy metadataRecoveryStrategy, - Optional bootstrapConfiguration) { + BootstrapConfiguration bootstrapConfiguration) { this(null, metadata, selector, @@ -239,7 +239,7 @@ public NetworkClient(Selectable selector, Sensor throttleTimeSensor, LogContext logContext, MetadataRecoveryStrategy metadataRecoveryStrategy, - Optional bootstrapConfiguration) { + BootstrapConfiguration bootstrapConfiguration) { this(null, metadata, selector, @@ -280,7 +280,7 @@ public NetworkClient(Selectable selector, ApiVersions apiVersions, LogContext logContext, MetadataRecoveryStrategy metadataRecoveryStrategy, - Optional bootstrapConfiguration) { + BootstrapConfiguration bootstrapConfiguration) { this(metadataUpdater, null, selector, @@ -326,7 +326,7 @@ public NetworkClient(MetadataUpdater metadataUpdater, ClientTelemetrySender clientTelemetrySender, long rebootstrapTriggerMs, MetadataRecoveryStrategy metadataRecoveryStrategy, - Optional bootstrapConfiguration) { + BootstrapConfiguration bootstrapConfiguration) { /* It would be better if we could pass `DefaultMetadataUpdater` from the public constructor, but it's not * possible because `DefaultMetadataUpdater` is an inner class and it can only be instantiated after the * super constructor is invoked. @@ -359,7 +359,7 @@ public NetworkClient(MetadataUpdater metadataUpdater, this.telemetrySender = (clientTelemetrySender != null) ? new TelemetrySender(clientTelemetrySender) : null; this.rebootstrapTriggerMs = rebootstrapTriggerMs; this.metadataRecoveryStrategy = metadataRecoveryStrategy; - this.bootstrapConfiguration = bootstrapConfiguration.orElse(null); + this.bootstrapState = new BootstrapState(bootstrapConfiguration); } /** @@ -643,7 +643,7 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long @Override public List poll(long timeout, long now) { ensureActive(); - ensureBootstrapped(); + ensureBootstrapped(now); if (!abortedSends.isEmpty()) { // If there are aborted sends because of unsupported version exceptions or disconnects, @@ -1187,63 +1187,81 @@ private boolean isTelemetryApi(ApiKeys apiKey) { } public static class BootstrapConfiguration { - private final Timer timer; - private final List bootstrapServers; - private final ClientDnsLookup clientDnsLookup; - private final long bootstrapResolveTimeoutMs; - private boolean isBootstrapped = false; - - public BootstrapConfiguration(Time time, - List bootstrapServers, - ClientDnsLookup clientDnsLookup, - long bootstrapResolveTimeoutMs) { - this.timer = time.timer(bootstrapResolveTimeoutMs); + public final List bootstrapServers; + public final ClientDnsLookup clientDnsLookup; + public final long bootstrapResolveTimeoutMs; + private boolean isBootstrapDisabled; + + public BootstrapConfiguration(final List bootstrapServers, + final ClientDnsLookup clientDnsLookup, + final long bootstrapResolveTimeoutMs) { this.bootstrapServers = bootstrapServers; this.clientDnsLookup = clientDnsLookup; this.bootstrapResolveTimeoutMs = bootstrapResolveTimeoutMs; + this.isBootstrapDisabled = false; } - private void checkTimerExpiration() { - if (timer.isExpired()) { - throw new BootstrapResolutionException("Unable to Resolve Address within the configured period " + - bootstrapResolveTimeoutMs + " ms."); - } + public void disableBootstrap() { + this.isBootstrapDisabled = true; } + } - private List tryResolveAddresses() { - List addresses = ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookup); + private class BootstrapState { + private final Timer timer; + private final List bootstrapServers; + private final ClientDnsLookup clientDnsLookup; + private final long dnsResolutionTimeoutMs; + private final boolean isDisabled; + BootstrapState(BootstrapConfiguration bootstrapConfiguration) { + this.dnsResolutionTimeoutMs = bootstrapConfiguration.bootstrapResolveTimeoutMs; + this.timer = time.timer(bootstrapConfiguration.bootstrapResolveTimeoutMs); + this.bootstrapServers = bootstrapConfiguration.bootstrapServers; + this.clientDnsLookup = bootstrapConfiguration.clientDnsLookup; + this.isDisabled = bootstrapConfiguration.isBootstrapDisabled; + } + + List tryResolveAddresses(final long currentTimeMs) { + timer.update(currentTimeMs); + List addresses = ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookup); if (!addresses.isEmpty()) { - timer.reset(bootstrapResolveTimeoutMs); - isBootstrapped = true; + timer.reset(dnsResolutionTimeoutMs); return addresses; - } else { - return List.of(); } + + if (timer.isExpired()) { + throw new BootstrapResolutionException("Timeout while attempting to resolve bootstrap " + + "servers. "); + } + // TODO: why? + return ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookup); + } + + boolean isDisabled() { + return isDisabled; + } + + boolean isTimerExpired() { + return timer.isExpired(); } } - void ensureBootstrapped() { - if (isBootstrapped() || null == bootstrapConfiguration) + void ensureBootstrapped(final long currentTimeMs) { + if (bootstrapState.isDisabled() || metadataUpdater.isBootstrapped()) return; - bootstrapConfiguration.timer.update(time.milliseconds()); - bootstrapConfiguration.checkTimerExpiration(); - - List servers = bootstrapConfiguration.tryResolveAddresses(); + List servers = bootstrapState.tryResolveAddresses(currentTimeMs); if (!servers.isEmpty()) { metadataUpdater.bootstrap(servers); + return; } - } - public boolean isBootstrapped() { - if (null == bootstrapConfiguration) - return false; - else - return bootstrapConfiguration.isBootstrapped; + if (bootstrapState.timer.isExpired()) { + throw new BootstrapResolutionException("Unable to Resolve Address within the configured period " + + bootstrapState.dnsResolutionTimeoutMs + "ms."); + } } - class DefaultMetadataUpdater implements MetadataUpdater { /* the current cluster metadata */ @@ -1268,7 +1286,7 @@ class DefaultMetadataUpdater implements MetadataUpdater { @Override public List fetchNodes() { - ensureBootstrapped(); + ensureBootstrapped(time.milliseconds()); return metadata.fetch().nodes(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index aaa77bb16eb57..fa9440dfb9102 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -18,7 +18,6 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.clients.ApiVersions; -import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientUtils; @@ -177,8 +176,6 @@ import org.apache.kafka.common.metrics.MetricsContext; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.network.ChannelBuilder; -import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; @@ -419,11 +416,6 @@ public class KafkaAdminClient extends AdminClient { */ private Uuid clientInstanceId; - /** - * Addresses for the admin client to bootstrap - */ - private static AdminBootstrapAddresses adminAddresses; - /** * Get or create a list value from a map. * @@ -531,10 +523,6 @@ static KafkaAdminClient createInternal( ) { Metrics metrics = null; NetworkClient networkClient = null; - NetworkClient.BootstrapConfiguration bootstrapConfiguration; - ChannelBuilder channelBuilder; - Selector selector; - List bootstrapAddresses = new ArrayList<>(); Time time = Time.SYSTEM; String clientId = generateClientId(config); ApiVersions apiVersions = new ApiVersions(); @@ -544,7 +532,7 @@ static KafkaAdminClient createInternal( try { // Since we only request node information, it's safe to pass true for allowAutoTopicCreation (and it // simplifies communication with older brokers) - adminAddresses = AdminBootstrapAddresses.fromConfig(config); + AdminBootstrapAddresses adminAddresses = AdminBootstrapAddresses.fromConfig(config); AdminMetadataManager metadataManager = new AdminMetadataManager(logContext, config.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), config.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), @@ -561,45 +549,20 @@ static KafkaAdminClient createInternal( MetricsContext metricsContext = new KafkaMetricsContext(JMX_PREFIX, config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)); metrics = new Metrics(metricConfig, reporters, time, metricsContext); - adminAddresses.addresses().forEach(inetSocketAddress -> - bootstrapAddresses.add(inetSocketAddress.getHostString() + ":" + inetSocketAddress.getPort())); - channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); - selector = new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), + networkClient = ClientUtils.createNetworkClient(config, + clientId, metrics, - time, "admin-client", - channelBuilder, - logContext); - bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + logContext, + apiVersions, time, - bootstrapAddresses, - ClientDnsLookup.forConfig(config.getString(AdminClientConfig.CLIENT_DNS_LOOKUP_CONFIG)), - 2 * 60 * 1000 - ); - networkClient = new NetworkClient(metadataManager.updater(), - null, - selector, - clientId, 1, - config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG), - config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG), - config.getInt(CommonClientConfigs.SEND_BUFFER_CONFIG), - config.getInt(CommonClientConfigs.RECEIVE_BUFFER_CONFIG), - config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG), - config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG), - config.getLong(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG), - time, - true, - apiVersions, + (int) TimeUnit.HOURS.toMillis(1), null, - logContext, + metadataManager.updater(), (hostResolver == null) ? new DefaultHostResolver() : hostResolver, null, - config.getLong(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG), - MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), - Optional.of(bootstrapConfiguration) - ); - + clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, timeoutProcessorFactory, logContext, clientTelemetryReporter); } catch (Throwable exc) { diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 132c22146dbea..fca0a9ca2121b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -588,11 +588,6 @@ public boolean active() { return active; } - @Override - public boolean isBootstrapped() { - return false; - } - @Override public void close() { active = false; diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index bb8946ef54f90..7759fc5ba1fed 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -99,7 +99,6 @@ public class NetworkClientTest { private static List newAddresses; private static NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - TIME, BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + 100000000); @@ -138,7 +137,7 @@ private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) { return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } private NetworkClient createNetworkClientWithMaxInFlightRequestsPerConnection( @@ -146,7 +145,7 @@ private NetworkClient createNetworkClientWithMaxInFlightRequestsPerConnection( return new NetworkClient(selector, metadataUpdater, "mock", maxInFlightRequestsPerConnection, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoffMaxMs, long connectionSetupTimeoutMsTest, int nodeNumber) { @@ -155,21 +154,21 @@ private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoff return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } private NetworkClient createNetworkClientWithStaticNodes() { return new NetworkClient(selector, metadataUpdater, "mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata) { return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } private NetworkClient createNetworkClientWithNoVersionDiscovery() { @@ -177,14 +176,13 @@ private NetworkClient createNetworkClientWithNoVersionDiscovery() { reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } @BeforeEach public void setup() { selector.reset(); bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - TIME, BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS @@ -271,7 +269,7 @@ public synchronized void rebootstrap() { reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), rebootstrapTriggerMs, - MetadataRecoveryStrategy.REBOOTSTRAP, Optional.of(bootstrapConfiguration)); + MetadataRecoveryStrategy.REBOOTSTRAP, bootstrapConfiguration); MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); metadata.bootstrap(Collections.singletonList(new InetSocketAddress("localhost", 9999))); @@ -325,7 +323,7 @@ public synchronized void rebootstrap() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), - rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP, Optional.of(bootstrapConfiguration)); + rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP, bootstrapConfiguration); MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); @@ -1166,7 +1164,7 @@ public void testReconnectAfterAddressChange() { reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // Connect to one the initial addresses, then change the addresses and disconnect client.ready(node, TIME.milliseconds()); @@ -1227,7 +1225,7 @@ public void testFailedConnectionToFirstAddress() { reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // First connection attempt should fail client.ready(node, TIME.milliseconds()); @@ -1280,7 +1278,7 @@ public void testFailedConnectionToFirstAddressAfterReconnect() { reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // Connect to one the initial addresses, then change the addresses and disconnect client.ready(node, TIME.milliseconds()); @@ -1389,7 +1387,7 @@ public void testTelemetryRequest() { reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender, - Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, Optional.of(bootstrapConfiguration)); + Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // Send the ApiVersionsRequest client.ready(node, TIME.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 60daacdceab56..0184d0971627a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -200,6 +200,7 @@ public class FetchRequestManagerTest { private MemoryRecords nextRecords; private MemoryRecords emptyRecords; private MemoryRecords partialRecords; + private NetworkClient.BootstrapConfiguration bootstrapConfiguration; @BeforeEach public void setup() { @@ -208,6 +209,7 @@ public void setup() { emptyRecords = buildRecords(0L, 0, 0); partialRecords = buildRecords(4L, 1, 0); partialRecords.buffer().putInt(Records.SIZE_OFFSET, 10000); + bootstrapConfiguration = mock(NetworkClient.BootstrapConfiguration.class); } private void assignFromUser(Set partitions) { @@ -1906,7 +1908,7 @@ public void testQuotaMetrics() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.empty()); + MetadataRecoveryStrategy.NONE, bootstrapConfiguration); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.BROKER); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index f3283eb277b9b..27c2d81d31584 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -140,6 +140,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -197,6 +198,7 @@ public class FetcherTest { private MemoryRecords emptyRecords; private MemoryRecords partialRecords; private ExecutorService executorService; + private NetworkClient.BootstrapConfiguration mockBootstrapConfig; @BeforeEach public void setup() { @@ -206,6 +208,7 @@ public void setup() { emptyRecords = buildRecords(0L, 0, 0); partialRecords = buildRecords(4L, 1, 0); partialRecords.buffer().putInt(Records.SIZE_OFFSET, 10000); + mockBootstrapConfig = mock(NetworkClient.BootstrapConfiguration.class); } private void assignFromUser(Set partitions) { @@ -1893,7 +1896,7 @@ public void testQuotaMetrics() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogContext(), - MetadataRecoveryStrategy.NONE, Optional.empty()); + MetadataRecoveryStrategy.NONE, mockBootstrapConfig); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.BROKER); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index ce67053933e93..5d15229a838a6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -559,6 +559,24 @@ public void testNoSerializerProvided() { }); } + @Test + public void testConstructorFailureCloseResource() { + Properties props = new Properties(); + props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar.local:9999"); + props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + + final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); + final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); + try (KafkaProducer ignored = new KafkaProducer<>(props, new ByteArraySerializer(), new ByteArraySerializer())) { + fail("should have caught an exception and returned"); + } catch (KafkaException e) { + assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); + assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); + assertEquals("Failed to construct kafka producer", e.getMessage()); + } + } + @Test public void testConstructorWithNotStringKey() { Properties props = new Properties(); @@ -1095,7 +1113,7 @@ private > void doTestHeaders(Class serializerCla metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, nowMs); KafkaProducer producer = kafkaProducer(configs, keySerializer, valueSerializer, metadata, - new MockClient(new MockTime(), metadata), null, Time.SYSTEM); + null, null, Time.SYSTEM); when(keySerializer.serialize(any(), any(), any())).then(invocation -> invocation.getArgument(2).getBytes()); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 9e01551995d21..43d18bb71c6d2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -111,7 +111,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; @@ -179,6 +178,7 @@ public class SenderTest { private Sender sender = null; private SenderMetricsRegistry senderMetricsRegistry = null; private final LogContext logContext = new LogContext(); + private final NetworkClient.BootstrapConfiguration bootstrapConfig = mock(NetworkClient.BootstrapConfiguration.class); @BeforeEach public void setup() { @@ -241,7 +241,7 @@ public void testQuotaMetrics() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, time, true, new ApiVersions(), throttleTimeSensor, logContext, - MetadataRecoveryStrategy.NONE, Optional.empty()); + MetadataRecoveryStrategy.NONE, bootstrapConfig); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( 400, ApiMessageType.ListenerType.BROKER); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 0bdc96e634dc9..b6e3f41dacd31 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -50,7 +50,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; @@ -109,7 +108,6 @@ public WorkerGroupMember(DistributedConfig config, String metricGrpPrefix = "connect"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - time, config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS); @@ -131,7 +129,7 @@ public WorkerGroupMember(DistributedConfig config, logContext, config.getLong(CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_CONFIG), MetadataRecoveryStrategy.forName(config.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), - Optional.of(bootstrapConfiguration) + bootstrapConfiguration ); this.client = new ConsumerNetworkClient( logContext, diff --git a/core/src/main/java/kafka/server/NetworkUtils.java b/core/src/main/java/kafka/server/NetworkUtils.java index b41c41c257dfc..579d6765bd720 100644 --- a/core/src/main/java/kafka/server/NetworkUtils.java +++ b/core/src/main/java/kafka/server/NetworkUtils.java @@ -17,6 +17,8 @@ package kafka.server; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.ManualMetadataUpdater; import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.NetworkClient; @@ -32,7 +34,6 @@ import org.apache.kafka.common.utils.Time; import java.util.Map; -import java.util.Optional; public class NetworkUtils { @@ -70,6 +71,11 @@ public static NetworkClient buildNetworkClient(String prefix, ); String clientId = prefix + "-client-" + config.nodeId(); + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ); return new NetworkClient( selector, new ManualMetadataUpdater(), @@ -87,7 +93,7 @@ public static NetworkClient buildNetworkClient(String prefix, new ApiVersions(), logContext, MetadataRecoveryStrategy.NONE, - Optional.empty() + bootstrapConfiguration ); } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 26a5a7cde1e9d..e685fb32f66aa 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -38,7 +38,6 @@ import org.apache.kafka.server.common.RequestLocal import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler} -import java.util.Optional import scala.collection.{concurrent, immutable} import scala.jdk.CollectionConverters._ import scala.jdk.javaapi.OptionConverters @@ -83,6 +82,11 @@ object TransactionMarkerChannelManager { channelBuilder, logContext ) + val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ) val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -100,7 +104,7 @@ object TransactionMarkerChannelManager { new ApiVersions, logContext, MetadataRecoveryStrategy.NONE, - Optional.empty + bootstrapConfiguration ) new TransactionMarkerChannelManager(config, diff --git a/core/src/main/scala/kafka/raft/KafkaRaftManager.scala b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala index 41b0dea8c079e..c1b35d57cdb7a 100644 --- a/core/src/main/scala/kafka/raft/KafkaRaftManager.scala +++ b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala @@ -20,12 +20,12 @@ import java.io.File import java.net.InetSocketAddress import java.nio.file.Files import java.nio.file.Paths -import java.util.{Optional, OptionalInt, Collection => JCollection, Map => JMap} +import java.util.{OptionalInt, Collection => JCollection, Map => JMap} import java.util.concurrent.CompletableFuture import kafka.server.KafkaConfig import kafka.utils.CoreUtils import kafka.utils.Logging -import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} +import org.apache.kafka.clients.{ApiVersions, ClientDnsLookup, CommonClientConfigs, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid @@ -236,7 +236,11 @@ class KafkaRaftManager[T]( val reconnectBackoffMs = 50 val reconnectBackoffMsMs = 500 val discoverBrokerVersions = true - + val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ) val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -254,7 +258,7 @@ class KafkaRaftManager[T]( apiVersions, logContext, MetadataRecoveryStrategy.NONE, - Optional.empty + bootstrapConfiguration ) (controllerListenerName, networkClient) diff --git a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala index 51450103c00c2..32b729bb20864 100644 --- a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala +++ b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala @@ -28,7 +28,6 @@ import org.apache.kafka.common.{Node, Reconfigurable} import org.apache.kafka.common.requests.AbstractRequest.Builder import org.apache.kafka.server.network.BrokerEndPoint -import java.util.Optional import scala.jdk.CollectionConverters._ trait BlockingSend { @@ -80,6 +79,12 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint, channelBuilder, logContext ) + + val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + brokerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(brokerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ) val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -97,7 +102,7 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint, new ApiVersions, logContext, MetadataRecoveryStrategy.NONE, - Optional.empty + bootstrapConfiguration ) (networkClient, reconfigurableChannelBuilder) } diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index cd8af23dce407..7879f4dbd4430 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -142,6 +142,12 @@ class NodeToControllerChannelManagerImpl( channelBuilder, logContext ) + + val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ) new NetworkClient( selector, manualMetadataUpdater, @@ -159,7 +165,7 @@ class NodeToControllerChannelManagerImpl( apiVersions, logContext, MetadataRecoveryStrategy.NONE, - Optional.empty + bootstrapConfiguration ) } val threadName = s"${threadNamePrefix}to-controller-${channelName}-channel-manager" diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index 92fbe1f52b0e5..214469f161cd0 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -57,7 +57,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -183,6 +182,11 @@ static AdminClient create(AbstractConfig config) { "admin", ClientUtils.createChannelBuilder(config, time, logContext), logContext); + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ); NetworkClient networkClient = new NetworkClient( selector, metadata, @@ -200,7 +204,7 @@ static AdminClient create(AbstractConfig config) { new ApiVersions(), logContext, MetadataRecoveryStrategy.NONE, - Optional.empty()); + bootstrapConfiguration); ConsumerNetworkClient highLevelClient = new ConsumerNetworkClient( logContext, networkClient, diff --git a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java index f1f7983957d2d..97d691ab813b6 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java @@ -665,7 +665,7 @@ private static class ReplicaFetcherBlockingSend { metrics, time, "replica-fetcher", - new HashMap() {{ + new HashMap<>() {{ put("broker-id", sourceNode.idString()); put("fetcher-id", String.valueOf(fetcherId)); }}, @@ -674,7 +674,6 @@ private static class ReplicaFetcherBlockingSend { logContext ); NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - time, consumerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(consumerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), consumerConfig.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG)); @@ -695,7 +694,7 @@ private static class ReplicaFetcherBlockingSend { new ApiVersions(), logContext, MetadataRecoveryStrategy.forName(consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)), - Optional.of(bootstrapConfiguration) + bootstrapConfiguration ); } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index 9254b1b20209e..cbefc8c86c1e1 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -18,7 +18,9 @@ package org.apache.kafka.trogdor.workload; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.ManualMetadataUpdater; import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.NetworkClient; @@ -52,7 +54,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.List; -import java.util.Optional; import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -165,6 +166,11 @@ public boolean tryConnect() { try (Metrics metrics = new Metrics()) { try (Selector selector = new Selector(conf.getLong(AdminClientConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, Time.SYSTEM, "", channelBuilder, logContext)) { + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + conf.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(conf.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ); try (NetworkClient client = new NetworkClient(selector, updater, "ConnectionStressWorker", @@ -181,7 +187,7 @@ public boolean tryConnect() { new ApiVersions(), logContext, MetadataRecoveryStrategy.NONE, - Optional.empty())) { + bootstrapConfiguration)) { NetworkClientUtils.awaitReady(client, targetNode, Time.SYSTEM, 500); } } From b6fe307c1828c0fb572e82a173482934d509384e Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 16 Dec 2025 10:47:17 +0000 Subject: [PATCH 04/30] add disable instance to BootstrapConfiguration --- .../java/org/apache/kafka/clients/NetworkClient.java | 6 ++++++ core/src/main/java/kafka/server/NetworkUtils.java | 9 +-------- .../transaction/TransactionMarkerChannelManager.scala | 7 +------ core/src/main/scala/kafka/raft/KafkaRaftManager.scala | 9 ++------- .../kafka/server/NodeToControllerChannelManager.scala | 7 +------ 5 files changed, 11 insertions(+), 27 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 26bafbe9cbfa5..581d6a90e0c63 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1201,6 +1201,12 @@ public BootstrapConfiguration(final List bootstrapServers, this.isBootstrapDisabled = false; } + public static BootstrapConfiguration disabled() { + BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration(null, null, 0); + bootstrapConfiguration.disableBootstrap(); + return bootstrapConfiguration; + } + public void disableBootstrap() { this.isBootstrapDisabled = true; } diff --git a/core/src/main/java/kafka/server/NetworkUtils.java b/core/src/main/java/kafka/server/NetworkUtils.java index 579d6765bd720..b789bb30f02f7 100644 --- a/core/src/main/java/kafka/server/NetworkUtils.java +++ b/core/src/main/java/kafka/server/NetworkUtils.java @@ -17,8 +17,6 @@ package kafka.server; import org.apache.kafka.clients.ApiVersions; -import org.apache.kafka.clients.ClientDnsLookup; -import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.ManualMetadataUpdater; import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.NetworkClient; @@ -71,11 +69,6 @@ public static NetworkClient buildNetworkClient(String prefix, ); String clientId = prefix + "-client-" + config.nodeId(); - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS - ); return new NetworkClient( selector, new ManualMetadataUpdater(), @@ -93,7 +86,7 @@ public static NetworkClient buildNetworkClient(String prefix, new ApiVersions(), logContext, MetadataRecoveryStrategy.NONE, - bootstrapConfiguration + NetworkClient.BootstrapConfiguration.disabled() ); } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index e685fb32f66aa..421c41ccc2cf2 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -82,11 +82,6 @@ object TransactionMarkerChannelManager { channelBuilder, logContext ) - val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS - ) val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -104,7 +99,7 @@ object TransactionMarkerChannelManager { new ApiVersions, logContext, MetadataRecoveryStrategy.NONE, - bootstrapConfiguration + NetworkClient.BootstrapConfiguration.disabled ) new TransactionMarkerChannelManager(config, diff --git a/core/src/main/scala/kafka/raft/KafkaRaftManager.scala b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala index c1b35d57cdb7a..ee1afe3a5e7ad 100644 --- a/core/src/main/scala/kafka/raft/KafkaRaftManager.scala +++ b/core/src/main/scala/kafka/raft/KafkaRaftManager.scala @@ -25,7 +25,7 @@ import java.util.concurrent.CompletableFuture import kafka.server.KafkaConfig import kafka.utils.CoreUtils import kafka.utils.Logging -import org.apache.kafka.clients.{ApiVersions, ClientDnsLookup, CommonClientConfigs, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} +import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.Uuid @@ -236,11 +236,6 @@ class KafkaRaftManager[T]( val reconnectBackoffMs = 50 val reconnectBackoffMsMs = 500 val discoverBrokerVersions = true - val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS - ) val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -258,7 +253,7 @@ class KafkaRaftManager[T]( apiVersions, logContext, MetadataRecoveryStrategy.NONE, - bootstrapConfiguration + NetworkClient.BootstrapConfiguration.disabled ) (controllerListenerName, networkClient) diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 7879f4dbd4430..d478c0c766b61 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -143,11 +143,6 @@ class NodeToControllerChannelManagerImpl( logContext ) - val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS - ) new NetworkClient( selector, manualMetadataUpdater, @@ -165,7 +160,7 @@ class NodeToControllerChannelManagerImpl( apiVersions, logContext, MetadataRecoveryStrategy.NONE, - bootstrapConfiguration + NetworkClient.BootstrapConfiguration.disabled ) } val threadName = s"${threadNamePrefix}to-controller-${channelName}-channel-manager" From 2a146bed3370f08daa81a65cde77c0bc24d17452 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 16 Dec 2025 18:01:21 +0000 Subject: [PATCH 05/30] Fix tests --- .../org/apache/kafka/clients/ClientUtils.java | 35 ++++++++++++++++ .../apache/kafka/clients/NetworkClient.java | 7 ++-- .../internals/AsyncKafkaConsumer.java | 7 ---- .../kafka/clients/NetworkClientTest.java | 42 ++++++++++++++++--- 4 files changed, 74 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index c3a9e971cec85..b62ef5bf5aaed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -53,6 +53,41 @@ public final class ClientUtils { private ClientUtils() { } + public static List validateAddresses(List urls, ClientDnsLookup clientDnsLookup) { + List addresses = new ArrayList<>(); + urls.forEach(url -> { + final String host = getHost(url); + final Integer port = getPort(url); + + if (clientDnsLookup == ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY) { + InetAddress[] inetAddresses; + try { + inetAddresses = InetAddress.getAllByName(host); + } catch (UnknownHostException e) { + inetAddresses = new InetAddress[0]; + } + + for (InetAddress inetAddress : inetAddresses) { + String resolvedCanonicalName = inetAddress.getCanonicalHostName(); + InetSocketAddress address = new InetSocketAddress(resolvedCanonicalName, port); + if (address.isUnresolved()) { + log.warn("Couldn't resolve server {} from {} as DNS resolution of the canonical hostname {} failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, resolvedCanonicalName, host); + } else { + addresses.add(address); + } + } + } else { + InetSocketAddress address = new InetSocketAddress(host, port); + if (address.isUnresolved()) { + log.warn("Couldn't resolve server {} from {} as DNS resolution failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, host); + } else { + addresses.add(address); + } + } + }); + return addresses; + } + public static List parseAndValidateAddresses(AbstractConfig config) { List urls = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); String clientDnsLookupConfig = config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG); diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 581d6a90e0c63..29d288fdc87a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1049,7 +1049,7 @@ private void handleApiVersionsResponse(List responses, // the ApiVersionsRequest when an UNSUPPORTED_VERSION error is returned. // If not provided, the client falls back to version 0. short maxApiVersion = 0; - if (apiVersionsResponse.data().apiKeys().size() > 0) { + if (!apiVersionsResponse.data().apiKeys().isEmpty()) { ApiVersion apiVersion = apiVersionsResponse.data().apiKeys().find(ApiKeys.API_VERSIONS.id); if (apiVersion != null) { maxApiVersion = apiVersion.maxVersion(); @@ -1229,7 +1229,7 @@ private class BootstrapState { List tryResolveAddresses(final long currentTimeMs) { timer.update(currentTimeMs); - List addresses = ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookup); + List addresses = ClientUtils.validateAddresses(bootstrapServers, clientDnsLookup); if (!addresses.isEmpty()) { timer.reset(dnsResolutionTimeoutMs); return addresses; @@ -1239,8 +1239,7 @@ List tryResolveAddresses(final long currentTimeMs) { throw new BootstrapResolutionException("Timeout while attempting to resolve bootstrap " + "servers. "); } - // TODO: why? - return ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookup); + return ClientUtils.validateAddresses(bootstrapServers, clientDnsLookup); } boolean isDisabled() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index fee082e2481fe..b0dba2c21a5d1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -112,7 +112,6 @@ import org.slf4j.Logger; import org.slf4j.event.Level; -import java.net.InetSocketAddress; import java.time.Duration; import java.util.Arrays; import java.util.Collection; @@ -163,7 +162,6 @@ * {@link ConsumerNetworkThread network thread}. Visit * this document * for implementation detail. - * *

* * Note: this {@link Consumer} implementation is part of the revised consumer group protocol from KIP-848. @@ -413,8 +411,6 @@ public AsyncKafkaConsumer(final ConsumerConfig config, interceptorList, Arrays.asList(deserializers.keyDeserializer(), deserializers.valueDeserializer())); this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners); - final List addresses = ClientUtils.parseAndValidateAddresses(config); - metadata.bootstrap(addresses); FetchMetricsManager fetchMetricsManager = createFetchMetricsManager(metrics); FetchConfig fetchConfig = new FetchConfig(config); @@ -1983,7 +1979,6 @@ private boolean isCommittedOffsetsManagementEnabled() { * This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests} for the * pre-fetch case, i.e. right before {@link #poll(Duration)} exits. In the pre-fetch case, the application thread * will not wait for confirmation of the request creation before continuing. - * *

* * At the point this method is called, {@link KafkaConsumer#poll(Duration)} has data ready to return to the user, @@ -2193,7 +2188,6 @@ private void subscribeInternal(Collection topics, Optional * * As an example, take {@link #unsubscribe()}. To start unsubscribing, the application thread enqueues an diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 7759fc5ba1fed..205925c9a97cc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -92,7 +92,7 @@ public class NetworkClientTest { protected static final MockTime TIME = new MockTime(); - private static final List BOOTSTRAP_ADDRESSES = new ArrayList<>(Arrays.asList( + private static final List BOOTSTRAP_ADDRESSES = new ArrayList<>(List.of( "127.0.0.1:8000", "127.0.0.2:8000")); private static List initialAddresses; @@ -101,7 +101,7 @@ public class NetworkClientTest { new NetworkClient.BootstrapConfiguration( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + 100000000); + 10 * 1000); protected final int defaultRequestTimeoutMs = 1000; protected final MockSelector selector = new MockSelector(TIME); @@ -134,6 +134,7 @@ public class NetworkClientTest { } private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) { + bootstrapMetadataUpdater(metadataUpdater); return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), @@ -164,7 +165,10 @@ connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new A MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } - private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata) { + private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata, boolean disableBootstrap) { + if (disableBootstrap) { + bootstrapConfiguration.disableBootstrap(); + } return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), @@ -172,6 +176,7 @@ defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMax } private NetworkClient createNetworkClientWithNoVersionDiscovery() { + bootstrapMetadataUpdater(metadataUpdater); return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, @@ -324,6 +329,7 @@ public synchronized void rebootstrap() { reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP, bootstrapConfiguration); + client.poll(0, TIME.milliseconds()); MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); @@ -608,7 +614,6 @@ public void testDefaultRequestTimeout() { * second produce call is intentionally made to emulate a request timeout. In the case that a timeout occurs * during a request, we want to ensure that we {@link Metadata#requestUpdate(boolean) request a metadata update} so that * on a subsequent invocation of {@link NetworkClient#poll(long, long) poll}, the metadata request will be sent. - * *

* * The {@link MetadataUpdater} has a specific method to handle @@ -623,10 +628,12 @@ private void testRequestTimeout(int requestTimeoutMs) { MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); - NetworkClient client = createNetworkClientWithNoVersionDiscovery(metadata); + NetworkClient client = createNetworkClientWithNoVersionDiscovery(metadata, true); + client.poll(0, TIME.milliseconds()); // Send first produce without any timeout. ClientResponse clientResponse = produce(client, requestTimeoutMs, false); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); assertEquals(node.idString(), clientResponse.destination()); assertFalse(clientResponse.wasDisconnected(), "Expected response to succeed and not disconnect"); assertFalse(clientResponse.wasTimedOut(), "Expected response to succeed and not time out"); @@ -870,7 +877,7 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { Node node1 = cluster.nodes().get(0); Node node2 = cluster.nodes().get(1); - NetworkClient client = createNetworkClientWithNoVersionDiscovery(metadata); + NetworkClient client = createNetworkClientWithNoVersionDiscovery(metadata, true); awaitReady(client, node1); @@ -1491,4 +1498,27 @@ public KafkaException getAndClearFailure() { return failure; } } + + private void bootstrapMetadataWithNodes(Metadata metadata, List nodes) { + List serverAddresses = new ArrayList<>(); + nodes.forEach(node -> serverAddresses.add(new InetSocketAddress(node.host(), node.port()))); + metadata.bootstrap(serverAddresses); + } + + private void bootstrapMetadata(Metadata metadata) { + List serverAddresses = new ArrayList<>(List.of( + new InetSocketAddress("localhost0", 8000), + new InetSocketAddress("localhost1", 8000) + )); + metadata.bootstrap(serverAddresses); + } + + private void bootstrapMetadataUpdater(final MetadataUpdater metadataUpdater) { + List serverAddresses = new ArrayList<>(List.of( + new InetSocketAddress("localhost0", 8000), + new InetSocketAddress("localhost1", 8000) + )); + metadataUpdater.bootstrap(serverAddresses); + System.out.println("Bootstraping metadata------:" + metadataUpdater.isBootstrapped()); + } } From 9f1afdb7502fa8caa0b762439edb45c24ac875dd Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 2 Jan 2026 20:16:35 +0000 Subject: [PATCH 06/30] Fix the failed tests --- .../org/apache/kafka/clients/Metadata.java | 8 ++++-- .../kafka/clients/MetadataSnapshot.java | 17 +++++++---- .../java/org/apache/kafka/common/Cluster.java | 28 +++++++++++++++++++ .../clients/producer/KafkaProducerTest.java | 9 +++++- .../internals/RecordAccumulatorTest.java | 1 + 5 files changed, 54 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 0986d8a67bc36..c9b618edd0185 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -545,9 +545,13 @@ else if (metadata.error() == Errors.TOPIC_AUTHORIZATION_FAILED) return this.metadataSnapshot.mergeWith(metadataResponse.clusterId(), nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, (topic, isInternal) -> !topics.contains(topic) && retainTopic(topic, isInternal, nowMs)); - else + else { + // Preserve the bootstrap flag from the current snapshot when creating a new one + boolean isBootstrapConfigured = this.metadataSnapshot.cluster().isBootstrapConfigured(); return new MetadataSnapshot(metadataResponse.clusterId(), nodes, partitions, - unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds); + unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, + isBootstrapConfigured, null); + } } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java index 424ecb0d29196..551b8b1391ebf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java @@ -54,6 +54,7 @@ public class MetadataSnapshot { private final Map metadataByPartition; private final Map topicIds; private final Map topicNames; + private final boolean isBootstrapConfigured; private Cluster clusterInstance; public MetadataSnapshot(String clusterId, @@ -64,7 +65,7 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds) { - this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, null); + this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, false, null); } // Visible for testing @@ -76,6 +77,7 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds, + boolean isBootstrapConfigured, Cluster clusterInstance) { this.clusterId = clusterId; this.nodes = Collections.unmodifiableMap(nodes); @@ -87,6 +89,7 @@ public MetadataSnapshot(String clusterId, this.topicNames = Collections.unmodifiableMap( topicIds.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)) ); + this.isBootstrapConfigured = isBootstrapConfigured; Map tmpMetadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { @@ -199,8 +202,9 @@ MetadataSnapshot mergeWith(String newClusterId, Set newInvalidTopics = fillSet(addInvalidTopics, invalidTopics, shouldRetainTopic); Set newInternalTopics = fillSet(addInternalTopics, internalTopics, shouldRetainTopic); + // Preserve the bootstrap flag from the current snapshot during merge return new MetadataSnapshot(newClusterId, newNodes, newMetadataByPartition.values(), newUnauthorizedTopics, - newInvalidTopics, newInternalTopics, newController, newTopicIds); + newInvalidTopics, newInternalTopics, newController, newTopicIds, this.isBootstrapConfigured, null); } /** @@ -227,8 +231,9 @@ private void computeClusterView() { .stream() .map(metadata -> MetadataResponse.toPartitionInfo(metadata, nodes)) .collect(Collectors.toList()); - this.clusterInstance = new Cluster(clusterId, nodes.values(), partitionInfos, unauthorizedTopics, - invalidTopics, internalTopics, controller, topicIds); + // Use the factory method that preserves the bootstrap state + this.clusterInstance = Cluster.withBootstrapFlag(clusterId, nodes.values(), partitionInfos, + unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, isBootstrapConfigured); } static MetadataSnapshot bootstrap(List addresses) { @@ -240,12 +245,12 @@ static MetadataSnapshot bootstrap(List addresses) { } return new MetadataSnapshot(null, nodes, Collections.emptyList(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), - null, Collections.emptyMap(), Cluster.bootstrap(addresses)); + null, Collections.emptyMap(), true, Cluster.bootstrap(addresses)); } static MetadataSnapshot empty() { return new MetadataSnapshot(null, Collections.emptyMap(), Collections.emptyList(), - Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), Cluster.empty()); + Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), false, Cluster.empty()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 93f2f4225bc74..9c7acc7791931 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -217,6 +217,34 @@ public static Cluster bootstrap(List addresses) { Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap()); } + /** + * Create a cluster instance preserving the bootstrap state. + * This is used internally by MetadataSnapshot to maintain bootstrap flag across metadata updates. + * + * @param clusterId the cluster id + * @param nodes the nodes in the cluster + * @param partitions information about partitions + * @param unauthorizedTopics unauthorized topics + * @param invalidTopics invalid topics + * @param internalTopics internal topics + * @param controller the controller node + * @param topicIds topic IDs + * @param isBootstrapConfigured whether the cluster is bootstrapped + * @return a new Cluster instance with the specified bootstrap state + */ + public static Cluster withBootstrapFlag(String clusterId, + Collection nodes, + Collection partitions, + Set unauthorizedTopics, + Set invalidTopics, + Set internalTopics, + Node controller, + Map topicIds, + boolean isBootstrapConfigured) { + return new Cluster(clusterId, isBootstrapConfigured, nodes, partitions, + unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); + } + /** * Return a copy of this cluster combined with `partitions`. */ diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 474bb412af86c..848e1e179f53d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; +import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.LeastLoadedNode; @@ -563,8 +565,10 @@ public void testNoSerializerProvided() { public void testConstructorFailureCloseResource() { Properties props = new Properties(); props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar.local:9999"); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + // Use invalid interceptor class to cause constructor failure after metrics initialization + props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "non.existent.Interceptor"); final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); @@ -1107,6 +1111,9 @@ private > void doTestHeaders(Class serializerCla long nowMs = Time.SYSTEM.milliseconds(); String topic = "topic"; ProducerMetadata metadata = newMetadata(0, 0, 90000); + // Bootstrap the metadata to mark it as configured (required for lazy bootstrapping) + metadata.bootstrap(ClientUtils.parseAndValidateAddresses( + Collections.singletonList("localhost:9999"), ClientDnsLookup.USE_ALL_DNS_IPS)); metadata.add(topic, nowMs); MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1)); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index d6ac75a37a0e6..353307816b766 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -161,6 +161,7 @@ public void testDrainBatches() throws Exception { Collections.emptySet(), null, Collections.emptyMap(), + false, cluster); long batchSize = value.length + DefaultRecordBatch.RECORD_BATCH_OVERHEAD; RecordAccumulator accum = createTestRecordAccumulator((int) batchSize, Integer.MAX_VALUE, Compression.NONE, 10); From f7adedab065b6e78966e214d467d36c295d352ee Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Mon, 5 Jan 2026 09:49:24 +0000 Subject: [PATCH 07/30] Fix failed tests --- .../src/main/scala/kafka/server/BrokerBlockingSender.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala index 32b729bb20864..25845bef20118 100644 --- a/core/src/main/scala/kafka/server/BrokerBlockingSender.scala +++ b/core/src/main/scala/kafka/server/BrokerBlockingSender.scala @@ -80,11 +80,6 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint, logContext ) - val bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - brokerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(brokerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS - ) val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), @@ -102,7 +97,7 @@ class BrokerBlockingSender(sourceBroker: BrokerEndPoint, new ApiVersions, logContext, MetadataRecoveryStrategy.NONE, - bootstrapConfiguration + NetworkClient.BootstrapConfiguration.disabled ) (networkClient, reconfigurableChannelBuilder) } From 3168e32986b9d95f67898495d19ebb964608e23d Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Mon, 5 Jan 2026 14:04:48 +0000 Subject: [PATCH 08/30] Fix failed tests --- .../src/main/java/org/apache/kafka/clients/ClientUtils.java | 5 +++++ .../main/java/org/apache/kafka/clients/NetworkClient.java | 4 ++-- .../clients/consumer/internals/FetchRequestManagerTest.java | 2 +- .../apache/kafka/clients/consumer/internals/FetcherTest.java | 3 +-- .../apache/kafka/clients/producer/internals/SenderTest.java | 2 +- 5 files changed, 10 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index b62ef5bf5aaed..3899b4df0e053 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -55,6 +55,9 @@ private ClientUtils() { public static List validateAddresses(List urls, ClientDnsLookup clientDnsLookup) { List addresses = new ArrayList<>(); + if (urls == null) { + return addresses; + } urls.forEach(url -> { final String host = getHost(url); final Integer port = getPort(url); @@ -264,6 +267,8 @@ public static NetworkClient createNetworkClient(AbstractConfig config, metricsGroupPrefix, channelBuilder, logContext); + // Validate bootstrap servers immediately during construction + parseAndValidateAddresses(config); bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 29d288fdc87a4..163e5a9fc6cd2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1187,7 +1187,7 @@ private boolean isTelemetryApi(ApiKeys apiKey) { } public static class BootstrapConfiguration { - public final List bootstrapServers; + public List bootstrapServers; public final ClientDnsLookup clientDnsLookup; public final long bootstrapResolveTimeoutMs; private boolean isBootstrapDisabled; @@ -1202,7 +1202,7 @@ public BootstrapConfiguration(final List bootstrapServers, } public static BootstrapConfiguration disabled() { - BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration(null, null, 0); + BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration(List.of(), null, 0); bootstrapConfiguration.disableBootstrap(); return bootstrapConfiguration; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 0184d0971627a..215a6d2b5bac0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -209,7 +209,7 @@ public void setup() { emptyRecords = buildRecords(0L, 0, 0); partialRecords = buildRecords(4L, 1, 0); partialRecords.buffer().putInt(Records.SIZE_OFFSET, 10000); - bootstrapConfiguration = mock(NetworkClient.BootstrapConfiguration.class); + bootstrapConfiguration = NetworkClient.BootstrapConfiguration.disabled(); } private void assignFromUser(Set partitions) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 27c2d81d31584..f5d74bf994eab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -140,7 +140,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -208,7 +207,7 @@ public void setup() { emptyRecords = buildRecords(0L, 0, 0); partialRecords = buildRecords(4L, 1, 0); partialRecords.buffer().putInt(Records.SIZE_OFFSET, 10000); - mockBootstrapConfig = mock(NetworkClient.BootstrapConfiguration.class); + mockBootstrapConfig = NetworkClient.BootstrapConfiguration.disabled(); } private void assignFromUser(Set partitions) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 43d18bb71c6d2..a07afa8a78e39 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -178,7 +178,7 @@ public class SenderTest { private Sender sender = null; private SenderMetricsRegistry senderMetricsRegistry = null; private final LogContext logContext = new LogContext(); - private final NetworkClient.BootstrapConfiguration bootstrapConfig = mock(NetworkClient.BootstrapConfiguration.class); + private final NetworkClient.BootstrapConfiguration bootstrapConfig = NetworkClient.BootstrapConfiguration.disabled(); @BeforeEach public void setup() { From 71c285fc1409a58874f480a4ee011c85b3659d53 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Mon, 5 Jan 2026 16:48:32 +0000 Subject: [PATCH 09/30] Fix the failed tests --- .../org/apache/kafka/clients/ClientUtils.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 3899b4df0e053..ad3c4a6def213 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -267,11 +267,18 @@ public static NetworkClient createNetworkClient(AbstractConfig config, metricsGroupPrefix, channelBuilder, logContext); - // Validate bootstrap servers immediately during construction - parseAndValidateAddresses(config); + List bootstrapServers = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); + ClientDnsLookup dnsLookup = ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)); + + // Only validate if bootstrap servers are provided (non-empty list) + // This allows configurations that don't use bootstrap (e.g., broker-to-broker) to skip validation + if (bootstrapServers != null && !bootstrapServers.isEmpty()) { + parseAndValidateAddresses(bootstrapServers, dnsLookup); + } + bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + bootstrapServers, + dnsLookup, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS ); return new NetworkClient(metadataUpdater, From 39ee0b1bde269f6eeebc65d13448fa8431c09404 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 3 Feb 2026 14:16:10 +0000 Subject: [PATCH 10/30] fix conflict --- .../server/NodeToControllerChannelManagerImpl.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java index c54fb97a65bdc..b9b7089dab1e0 100644 --- a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java +++ b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java @@ -18,6 +18,8 @@ package org.apache.kafka.server; import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.ClientDnsLookup; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.ManualMetadataUpdater; import org.apache.kafka.clients.MetadataRecoveryStrategy; @@ -113,6 +115,13 @@ private KafkaClient buildNetworkClient(ControllerInformation controllerInfo) { channelBuilder, logContext ); + + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), + ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + ); + return new NetworkClient( selector, manualMetadataUpdater, @@ -129,7 +138,8 @@ private KafkaClient buildNetworkClient(ControllerInformation controllerInfo) { true, apiVersions, logContext, - MetadataRecoveryStrategy.NONE + MetadataRecoveryStrategy.NONE, + bootstrapConfiguration ); } From 0afecf5a197c20129f4a85302bd4cb627ca5db7d Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 3 Feb 2026 17:05:58 +0000 Subject: [PATCH 11/30] fix failed tests --- .../kafka/server/NodeToControllerChannelManagerImpl.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java index b9b7089dab1e0..617e75bd5e584 100644 --- a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java +++ b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java @@ -116,12 +116,6 @@ private KafkaClient buildNetworkClient(ControllerInformation controllerInfo) { logContext ); - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), - ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS - ); - return new NetworkClient( selector, manualMetadataUpdater, @@ -139,7 +133,7 @@ private KafkaClient buildNetworkClient(ControllerInformation controllerInfo) { apiVersions, logContext, MetadataRecoveryStrategy.NONE, - bootstrapConfiguration + NetworkClient.BootstrapConfiguration.disabled() ); } From f91a1968f3ad6fd1b449d5f4839248810f7573ff Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 25 Feb 2026 20:51:10 +0000 Subject: [PATCH 12/30] address comments --- .../org/apache/kafka/clients/ClientUtils.java | 91 ++++++++++--------- .../NodeToControllerChannelManagerImpl.java | 2 - 2 files changed, 48 insertions(+), 45 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index ad3c4a6def213..a7b0ace3a35f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -53,6 +53,49 @@ public final class ClientUtils { private ClientUtils() { } + /** + * Resolves a single URL to one or more InetSocketAddress based on the DNS lookup strategy. + * + * @param url the original URL string (for logging) + * @param host the hostname extracted from the URL + * @param port the port extracted from the URL + * @param clientDnsLookup the DNS lookup strategy + * @return list of resolved addresses (may be empty if addresses are unresolved) + * @throws UnknownHostException if DNS resolution fails + */ + private static List resolveAddress( + String url, + String host, + Integer port, + ClientDnsLookup clientDnsLookup) throws UnknownHostException { + + List addresses = new ArrayList<>(); + + if (clientDnsLookup == ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY) { + InetAddress[] inetAddresses = InetAddress.getAllByName(host); + for (InetAddress inetAddress : inetAddresses) { + String resolvedCanonicalName = inetAddress.getCanonicalHostName(); + InetSocketAddress address = new InetSocketAddress(resolvedCanonicalName, port); + if (address.isUnresolved()) { + log.warn("Couldn't resolve server {} from {} as DNS resolution of the canonical hostname {} failed for {}", + url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, resolvedCanonicalName, host); + } else { + addresses.add(address); + } + } + } else { + InetSocketAddress address = new InetSocketAddress(host, port); + if (address.isUnresolved()) { + log.warn("Couldn't resolve server {} from {} as DNS resolution failed for {}", + url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, host); + } else { + addresses.add(address); + } + } + + return addresses; + } + public static List validateAddresses(List urls, ClientDnsLookup clientDnsLookup) { List addresses = new ArrayList<>(); if (urls == null) { @@ -62,30 +105,10 @@ public static List validateAddresses(List urls, Clien final String host = getHost(url); final Integer port = getPort(url); - if (clientDnsLookup == ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY) { - InetAddress[] inetAddresses; - try { - inetAddresses = InetAddress.getAllByName(host); - } catch (UnknownHostException e) { - inetAddresses = new InetAddress[0]; - } - - for (InetAddress inetAddress : inetAddresses) { - String resolvedCanonicalName = inetAddress.getCanonicalHostName(); - InetSocketAddress address = new InetSocketAddress(resolvedCanonicalName, port); - if (address.isUnresolved()) { - log.warn("Couldn't resolve server {} from {} as DNS resolution of the canonical hostname {} failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, resolvedCanonicalName, host); - } else { - addresses.add(address); - } - } - } else { - InetSocketAddress address = new InetSocketAddress(host, port); - if (address.isUnresolved()) { - log.warn("Couldn't resolve server {} from {} as DNS resolution failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, host); - } else { - addresses.add(address); - } + try { + addresses.addAll(resolveAddress(url, host, port, clientDnsLookup)); + } catch (UnknownHostException e) { + // Silently ignore - this matches the original behavior } }); return addresses; @@ -111,25 +134,7 @@ public static List parseAndValidateAddresses(List url if (host == null || port == null) throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); - if (clientDnsLookup == ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY) { - InetAddress[] inetAddresses = InetAddress.getAllByName(host); - for (InetAddress inetAddress : inetAddresses) { - String resolvedCanonicalName = inetAddress.getCanonicalHostName(); - InetSocketAddress address = new InetSocketAddress(resolvedCanonicalName, port); - if (address.isUnresolved()) { - log.warn("Couldn't resolve server {} from {} as DNS resolution of the canonical hostname {} failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, resolvedCanonicalName, host); - } else { - addresses.add(address); - } - } - } else { - InetSocketAddress address = new InetSocketAddress(host, port); - if (address.isUnresolved()) { - log.warn("Couldn't resolve server {} from {} as DNS resolution failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, host); - } else { - addresses.add(address); - } - } + addresses.addAll(resolveAddress(url, host, port, clientDnsLookup)); } catch (IllegalArgumentException e) { throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); diff --git a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java index 617e75bd5e584..6a1763bbeab56 100644 --- a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java +++ b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManagerImpl.java @@ -18,8 +18,6 @@ package org.apache.kafka.server; import org.apache.kafka.clients.ApiVersions; -import org.apache.kafka.clients.ClientDnsLookup; -import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.ManualMetadataUpdater; import org.apache.kafka.clients.MetadataRecoveryStrategy; From 2422387b66ba2a685532a4ab4354d18019efb457 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sun, 8 Mar 2026 22:25:02 +0000 Subject: [PATCH 13/30] address comments --- .../org/apache/kafka/clients/ClientUtils.java | 8 +- .../org/apache/kafka/clients/KafkaClient.java | 1 + .../apache/kafka/clients/NetworkClient.java | 98 ++- .../internals/AsyncKafkaConsumer.java | 3 + .../kafka/clients/NetworkClientTest.java | 571 ++++++++++-------- 5 files changed, 409 insertions(+), 272 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index a7b0ace3a35f2..a1d120cfd6566 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -64,10 +64,10 @@ private ClientUtils() { * @throws UnknownHostException if DNS resolution fails */ private static List resolveAddress( - String url, - String host, - Integer port, - ClientDnsLookup clientDnsLookup) throws UnknownHostException { + String url, + String host, + Integer port, + ClientDnsLookup clientDnsLookup) throws UnknownHostException { List addresses = new ArrayList<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 405644024c9bc..5332fc4c25ae4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -212,4 +212,5 @@ ClientRequest newClientRequest(String nodeId, * was invoked for this client. */ boolean active(); + } diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 163e5a9fc6cd2..6a1ef36eb49c4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -639,11 +639,16 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long * metadata timeout * @param now The current time in milliseconds * @return The list of responses received + * + * @implNote During bootstrap, this method may perform blocking DNS lookups to resolve bootstrap server addresses. + * If DNS resolution is slow or fails, this method may block for longer than the specified timeout. + * The blocking behavior will continue until either: (1) DNS resolution succeeds, (2) the bootstrap + * timeout expires (throwing BootstrapResolutionException), or (3) the poll timeout is reached. */ @Override public List poll(long timeout, long now) { ensureActive(); - ensureBootstrapped(now); + ensureBootstrapped(timeout, now); if (!abortedSends.isEmpty()) { // If there are aborted sends because of unsupported version exceptions or disconnects, @@ -1187,7 +1192,7 @@ private boolean isTelemetryApi(ApiKeys apiKey) { } public static class BootstrapConfiguration { - public List bootstrapServers; + public final List bootstrapServers; public final ClientDnsLookup clientDnsLookup; public final long bootstrapResolveTimeoutMs; private boolean isBootstrapDisabled; @@ -1227,43 +1232,74 @@ private class BootstrapState { this.isDisabled = bootstrapConfiguration.isBootstrapDisabled; } - List tryResolveAddresses(final long currentTimeMs) { - timer.update(currentTimeMs); - List addresses = ClientUtils.validateAddresses(bootstrapServers, clientDnsLookup); - if (!addresses.isEmpty()) { - timer.reset(dnsResolutionTimeoutMs); - return addresses; - } - - if (timer.isExpired()) { - throw new BootstrapResolutionException("Timeout while attempting to resolve bootstrap " + - "servers. "); - } - return ClientUtils.validateAddresses(bootstrapServers, clientDnsLookup); - } - boolean isDisabled() { return isDisabled; } - - boolean isTimerExpired() { - return timer.isExpired(); - } } - void ensureBootstrapped(final long currentTimeMs) { + /** + * Ensures that the client has successfully resolved and bootstrapped with the configured bootstrap servers. + * This method will retry DNS resolution until one of the following conditions is met: + *

    + *
  • DNS resolution succeeds and bootstrap is complete (method returns normally)
  • + *
  • Bootstrap timeout expires (throws BootstrapResolutionException)
  • + *
  • Poll timeout is reached but bootstrap timeout hasn't expired (method returns to retry later)
  • + *
+ * + * @param pollTimeoutMs The poll timeout in milliseconds, controlling how long this method should block + * during a single poll invocation + * @param currentTimeMs The current time in milliseconds + * @throws BootstrapResolutionException if the bootstrap timeout expires + * before DNS resolution succeeds + * + * @implNote This method performs blocking DNS lookups via {@link InetAddress#getAllByName(String)}. + * Each DNS lookup may block for an indefinite amount of time depending on network conditions + * and DNS server responsiveness. As a result, this method may block for longer than pollTimeoutMs + * if DNS resolution is slow. The method will retry DNS resolution in a loop with brief sleeps + * (up to 100ms) between attempts until one of the exit conditions above is met. + */ + void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { if (bootstrapState.isDisabled() || metadataUpdater.isBootstrapped()) return; - List servers = bootstrapState.tryResolveAddresses(currentTimeMs); - if (!servers.isEmpty()) { - metadataUpdater.bootstrap(servers); - return; - } + long pollDeadlineMs = currentTimeMs + pollTimeoutMs; + + while (true) { + long now = time.milliseconds(); + bootstrapState.timer.update(now); + + List servers = ClientUtils.validateAddresses( + bootstrapState.bootstrapServers, bootstrapState.clientDnsLookup); + + if (!servers.isEmpty()) { + // Resolution succeeded + bootstrapState.timer.reset(bootstrapState.dnsResolutionTimeoutMs); + metadataUpdater.bootstrap(servers); + return; + } + + // Check which timeout expires first + boolean bootstrapExpired = bootstrapState.timer.isExpired(); + boolean pollExpired = now >= pollDeadlineMs; - if (bootstrapState.timer.isExpired()) { - throw new BootstrapResolutionException("Unable to Resolve Address within the configured period " + - bootstrapState.dnsResolutionTimeoutMs + "ms."); + if (bootstrapExpired) { + // Bootstrap timeout expired before poll timeout + throw new BootstrapResolutionException("Timeout while attempting to resolve bootstrap servers."); + } + + if (pollExpired) { + // Poll timeout reached but bootstrap timeout hasn't expired yet + return; + } + + // Sleep briefly before retrying to avoid tight loop + long remainingPollTimeMs = pollDeadlineMs - now; + long remainingBootstrapTimeMs = bootstrapState.timer.remainingMs(); + long sleepTimeMs = Math.min(Math.min(remainingPollTimeMs, remainingBootstrapTimeMs), 100); + + if (sleepTimeMs > 0) { + time.sleep(sleepTimeMs); + } } } @@ -1291,7 +1327,7 @@ class DefaultMetadataUpdater implements MetadataUpdater { @Override public List fetchNodes() { - ensureBootstrapped(time.milliseconds()); + ensureBootstrapped(Long.MAX_VALUE, time.milliseconds()); return metadata.fetch().nodes(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 86f0a0ce8f70b..5b84ea0e6a62c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -162,6 +162,7 @@ * {@link ConsumerNetworkThread network thread}. Visit * this document * for implementation detail. + * *

* * Note: this {@link Consumer} implementation is part of the revised consumer group protocol from KIP-848. @@ -1979,6 +1980,7 @@ private boolean isCommittedOffsetsManagementEnabled() { * This method signals the background thread to {@link CreateFetchRequestsEvent create fetch requests} for the * pre-fetch case, i.e. right before {@link #poll(Duration)} exits. In the pre-fetch case, the application thread * will not wait for confirmation of the request creation before continuing. + * *

* * At the point this method is called, {@link KafkaConsumer#poll(Duration)} has data ready to return to the user, @@ -2188,6 +2190,7 @@ private void subscribeInternal(Collection topics, Optional BOOTSTRAP_ADDRESSES = new ArrayList<>(List.of( "127.0.0.1:8000", "127.0.0.2:8000")); @@ -104,7 +105,7 @@ public class NetworkClientTest { 10 * 1000); protected final int defaultRequestTimeoutMs = 1000; - protected final MockSelector selector = new MockSelector(TIME); + protected final MockSelector selector = new MockSelector(time); protected final Node node = TestUtils.singletonCluster().nodes().iterator().next(); protected final long reconnectBackoffMsTest = 10 * 1000; protected final long reconnectBackoffMaxMsTest = 10 * 10000; @@ -137,7 +138,7 @@ private NetworkClient createNetworkClient(long reconnectBackoffMaxMs) { bootstrapMetadataUpdater(metadataUpdater); return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } @@ -145,7 +146,7 @@ private NetworkClient createNetworkClientWithMaxInFlightRequestsPerConnection( int maxInFlightRequestsPerConnection, long reconnectBackoffMaxMs) { return new NetworkClient(selector, metadataUpdater, "mock", maxInFlightRequestsPerConnection, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } @@ -154,14 +155,14 @@ private NetworkClient createNetworkClientWithMultipleNodes(long reconnectBackoff TestMetadataUpdater metadataUpdater = new TestMetadataUpdater(nodes); return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMs, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } private NetworkClient createNetworkClientWithStaticNodes() { return new NetworkClient(selector, metadataUpdater, "mock-static", Integer.MAX_VALUE, 0, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, - connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, true, new ApiVersions(), new LogContext(), + connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } @@ -171,7 +172,7 @@ private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadat } return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } @@ -180,7 +181,7 @@ private NetworkClient createNetworkClientWithNoVersionDiscovery() { return new NetworkClient(selector, metadataUpdater, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, - connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, bootstrapConfiguration); } @@ -197,7 +198,7 @@ public void setup() { @Test public void testSendToUnreadyNode() { MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.singletonList("test"), true); - long now = TIME.milliseconds(); + long now = time.milliseconds(); ClientRequest request = client.newClientRequest("5", builder, now, false); assertThrows(IllegalStateException.class, () -> client.send(request, now)); } @@ -220,22 +221,22 @@ public void testSimpleRequestResponseWithNoBrokerDiscovery() { @Test public void testDnsLookupFailure() { /* Fail cleanly when the node has a bad hostname */ - assertFalse(client.ready(new Node(1234, "badhost", 1234), TIME.milliseconds())); + assertFalse(client.ready(new Node(1234, "badhost", 1234), time.milliseconds())); } @Test public void testClose() { - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); awaitReady(client, node); - client.poll(1, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds()), "The client should be ready"); + client.poll(1, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); ProduceRequest.Builder builder = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection()) .setAcks((short) 1) .setTimeoutMs(1000)); - ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true); - client.send(request, TIME.milliseconds()); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); + client.send(request, time.milliseconds()); assertEquals(1, client.inFlightRequestCount(node.idString()), "There should be 1 in-flight request after send"); assertTrue(client.hasInFlightRequests(node.idString())); @@ -254,7 +255,7 @@ public void testUnsupportedVersionDuringInternalMetadataRequest() { // disabling auto topic creation for versions less than 4 is not supported MetadataRequest.Builder builder = new MetadataRequest.Builder(topics, false, (short) 3); - client.sendInternalMetadataRequest(builder, node.idString(), TIME.milliseconds()); + client.sendInternalMetadataRequest(builder, node.idString(), time.milliseconds()); assertEquals(UnsupportedVersionException.class, metadataUpdater.getAndClearFailure().getClass()); } @@ -272,42 +273,42 @@ public synchronized void rebootstrap() { NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP, bootstrapConfiguration); MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); metadata.bootstrap(Collections.singletonList(new InetSocketAddress("localhost", 9999))); metadata.requestUpdate(true); - client.poll(0, TIME.milliseconds()); - TIME.sleep(rebootstrapTriggerMs + 1); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); + time.sleep(rebootstrapTriggerMs + 1); + client.poll(0, time.milliseconds()); assertEquals(1, rebootstrapCount.get()); - TIME.sleep(1); - client.poll(0, TIME.milliseconds()); + time.sleep(1); + client.poll(0, time.milliseconds()); assertEquals(1, rebootstrapCount.get()); metadata.requestUpdate(true); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(1, rebootstrapCount.get()); - metadataUpdater.handleFailedRequest(TIME.milliseconds(), Optional.of(new KafkaException())); - client.poll(0, TIME.milliseconds()); + metadataUpdater.handleFailedRequest(time.milliseconds(), Optional.of(new KafkaException())); + client.poll(0, time.milliseconds()); assertEquals(1, rebootstrapCount.get()); - TIME.sleep(rebootstrapTriggerMs); - client.poll(0, TIME.milliseconds()); + time.sleep(rebootstrapTriggerMs); + client.poll(0, time.milliseconds()); assertEquals(2, rebootstrapCount.get()); metadata.requestUpdate(true); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(2, rebootstrapCount.get()); MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); - ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true); + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); MetadataResponse rebootstrapResponse = (MetadataResponse) builder.build().getErrorResponse(0, new RebootstrapRequiredException("rebootstrap")); - metadataUpdater.handleSuccessfulResponse(request.makeHeader(builder.latestAllowedVersion()), TIME.milliseconds(), rebootstrapResponse); + metadataUpdater.handleSuccessfulResponse(request.makeHeader(builder.latestAllowedVersion()), time.milliseconds(), rebootstrapResponse); assertEquals(2, rebootstrapCount.get()); - TIME.sleep(50); - client.poll(0, TIME.milliseconds()); + time.sleep(50); + client.poll(0, time.milliseconds()); assertEquals(3, rebootstrapCount.get()); } @@ -327,32 +328,32 @@ public synchronized void rebootstrap() { metadata.bootstrap(Collections.singletonList(new InetSocketAddress("localhost", 9999))); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, - defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, TIME, false, new ApiVersions(), new LogContext(), + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new ApiVersions(), new LogContext(), rebootstrapTriggerMs, MetadataRecoveryStrategy.REBOOTSTRAP, bootstrapConfiguration); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); List nodes = metadata.fetch().nodes(); nodes.forEach(node -> { - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); awaitReady(client, node); }); // Queue a request sendEmptyProduceRequest(client, nodes.get(0).idString()); - List responses = client.poll(0, TIME.milliseconds()); + List responses = client.poll(0, time.milliseconds()); assertEquals(0, responses.size()); assertEquals(1, client.inFlightRequestCount()); // Trigger rebootstrap metadata.requestUpdate(true); - TIME.sleep(refreshBackoffMs); - responses = client.poll(0, TIME.milliseconds()); + time.sleep(refreshBackoffMs); + responses = client.poll(0, time.milliseconds()); assertEquals(0, responses.size()); assertEquals(2, client.inFlightRequestCount()); - TIME.sleep(rebootstrapTriggerMs + 1); - responses = client.poll(0, TIME.milliseconds()); + time.sleep(rebootstrapTriggerMs + 1); + responses = client.poll(0, time.milliseconds()); // Verify that inflight produce request was aborted with disconnection assertEquals(1, responses.size()); @@ -372,15 +373,15 @@ private void checkSimpleRequestResponse(NetworkClient networkClient) { .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = networkClient.newClientRequest(node.idString(), builder, TIME.milliseconds(), + ClientRequest request = networkClient.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); - networkClient.send(request, TIME.milliseconds()); - networkClient.poll(1, TIME.milliseconds()); + networkClient.send(request, time.milliseconds()); + networkClient.poll(1, time.milliseconds()); assertEquals(1, networkClient.inFlightRequestCount()); ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData()); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, requestVersion, request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); - List responses = networkClient.poll(1, TIME.milliseconds()); + List responses = networkClient.poll(1, time.milliseconds()); assertEquals(1, responses.size()); assertTrue(handler.executed, "The handler should have executed."); assertTrue(handler.response.hasResponse(), "Should have a response body."); @@ -403,18 +404,18 @@ private void awaitReady(NetworkClient client, Node node) { setExpectedApiVersionsResponse(TestUtils.defaultApiVersionsResponse( ApiMessageType.ListenerType.BROKER)); } - while (!client.ready(node, TIME.milliseconds())) - client.poll(1, TIME.milliseconds()); + while (!client.ready(node, time.milliseconds())) + client.poll(1, time.milliseconds()); selector.clear(); } @Test public void testInvalidApiVersionsRequest() { // initiate the connection - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); // handle the connection, send the ApiVersionsRequest - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // check that the ApiVersionsRequest has been initiated assertTrue(client.hasInFlightRequests(node.idString())); @@ -428,22 +429,22 @@ public void testInvalidApiVersionsRequest() { )); // handle completed receives - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); // various assertions - assertFalse(client.isReady(node, TIME.milliseconds())); + assertFalse(client.isReady(node, time.milliseconds())); } @Test public void testApiVersionsRequest() { // initiate the connection - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); // handle the connection, send the ApiVersionsRequest - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // check that the ApiVersionsRequest has been initiated assertTrue(client.hasInFlightRequests(node.idString())); @@ -452,28 +453,28 @@ public void testApiVersionsRequest() { delayedApiVersionsResponse(0, ApiKeys.API_VERSIONS.latestVersion(), defaultApiVersionsResponse()); // handle completed receives - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); // various assertions - assertTrue(client.isReady(node, TIME.milliseconds())); + assertTrue(client.isReady(node, time.milliseconds())); } @Test public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { // initiate the connection - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); // handle the connection, initiate first ApiVersionsRequest - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); // completes initiated sends - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(1, selector.completedSends().size()); ByteBuffer buffer = selector.completedSendBuffers().get(0).buffer(); @@ -495,7 +496,7 @@ public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { )); // handle ApiVersionResponse, initiate second ApiVersionRequest - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); @@ -509,7 +510,7 @@ public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { selector.completedReceives().clear(); // completes initiated sends - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // ApiVersionsRequest has been sent assertEquals(1, selector.completedSends().size()); @@ -523,29 +524,29 @@ public void testUnsupportedApiVersionsRequestWithVersionProvidedByTheBroker() { delayedApiVersionsResponse(1, (short) 0, defaultApiVersionsResponse()); // handle completed receives - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); assertEquals(1, selector.completedReceives().size()); // the client is ready - assertTrue(client.isReady(node, TIME.milliseconds())); + assertTrue(client.isReady(node, time.milliseconds())); } @Test public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() { // initiate the connection - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); // handle the connection, initiate first ApiVersionsRequest - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); // completes initiated sends - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(1, selector.completedSends().size()); ByteBuffer buffer = selector.completedSendBuffers().get(0).buffer(); @@ -561,7 +562,7 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() )); // handle ApiVersionResponse, initiate second ApiVersionRequest - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // ApiVersionsRequest is in flight but not sent yet assertTrue(client.hasInFlightRequests(node.idString())); @@ -575,7 +576,7 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() selector.completedReceives().clear(); // completes initiated sends - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // ApiVersionsRequest has been sent assertEquals(1, selector.completedSends().size()); @@ -589,14 +590,14 @@ public void testUnsupportedApiVersionsRequestWithoutVersionProvidedByTheBroker() delayedApiVersionsResponse(1, (short) 0, defaultApiVersionsResponse()); // handle completed receives - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); assertEquals(1, selector.completedReceives().size()); // the client is ready - assertTrue(client.isReady(node, TIME.milliseconds())); + assertTrue(client.isReady(node, time.milliseconds())); } @Test @@ -626,14 +627,14 @@ public void testDefaultRequestTimeout() { private void testRequestTimeout(int requestTimeoutMs) { Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); NetworkClient client = createNetworkClientWithNoVersionDiscovery(metadata, true); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // Send first produce without any timeout. ClientResponse clientResponse = produce(client, requestTimeoutMs, false); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); assertEquals(node.idString(), clientResponse.destination()); assertFalse(clientResponse.wasDisconnected(), "Expected response to succeed and not disconnect"); assertFalse(clientResponse.wasTimedOut(), "Expected response to succeed and not time out"); @@ -654,20 +655,20 @@ private ClientResponse produce(NetworkClient client, int requestTimeoutMs, boole .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, requestTimeoutMs, handler); - client.send(request, TIME.milliseconds()); + client.send(request, time.milliseconds()); if (shouldEmulateTimeout) { // For a delay of slightly more than our timeout threshold to emulate the request timing out. - TIME.sleep(requestTimeoutMs + 1); + time.sleep(requestTimeoutMs + 1); } else { ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData()); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, PRODUCE.latestVersion(), request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); } - List responses = client.poll(0, TIME.milliseconds()); + List responses = client.poll(0, time.milliseconds()); assertEquals(1, responses.size()); return responses.get(0); } @@ -680,18 +681,18 @@ public void testConnectionSetupTimeout() { final Node node0 = cluster.nodeById(0); final Node node1 = cluster.nodeById(1); - client.ready(node0, TIME.milliseconds()); + client.ready(node0, time.milliseconds()); selector.serverConnectionBlocked(node0.idString()); - client.ready(node1, TIME.milliseconds()); + client.ready(node1, time.milliseconds()); selector.serverConnectionBlocked(node1.idString()); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertFalse(client.connectionFailed(node), "The connections should not fail before the socket connection setup timeout elapsed"); - TIME.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); - client.poll(0, TIME.milliseconds()); + time.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); + client.poll(0, time.milliseconds()); assertTrue(client.connectionFailed(node), "Expected the connections to fail due to the socket connection setup timeout"); } @@ -708,21 +709,21 @@ public void testConnectionTimeoutAfterThrottling() { .setAcks((short) 1) .setTimeoutMs(timeoutMs)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest r1 = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, + ClientRequest r1 = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(r1, TIME.milliseconds()); - client.poll(0, TIME.milliseconds()); + client.send(r1, time.milliseconds()); + client.poll(0, time.milliseconds()); // Throttle long enough to ensure other inFlight requests timeout. ProduceResponse pr = new ProduceResponse(new ProduceResponseData().setThrottleTimeMs(timeoutMs)); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(pr, requestVersion, r1.correlationId()); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); - ClientRequest r2 = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, + ClientRequest r2 = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(r2, TIME.milliseconds()); - TIME.sleep(timeoutMs); - client.poll(0, TIME.milliseconds()); + client.send(r2, time.milliseconds()); + time.sleep(timeoutMs); + client.poll(0, time.milliseconds()); assertEquals(1, client.inFlightRequestCount(node.idString())); assertFalse(client.connectionFailed(node), "Connection should not have failed due to the extra time spent throttling."); @@ -740,29 +741,29 @@ public void testConnectionThrottling() { .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(node.idString(), builder, TIME.milliseconds(), true, + ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(request, TIME.milliseconds()); - client.poll(1, TIME.milliseconds()); + client.send(request, time.milliseconds()); + client.poll(1, time.milliseconds()); int throttleTime = 100; ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData().setThrottleTimeMs(throttleTime)); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, requestVersion, request.correlationId()); selector.completeReceive(new NetworkReceive(node.idString(), buffer)); - client.poll(1, TIME.milliseconds()); + client.poll(1, time.milliseconds()); // The connection is not ready due to throttling. - assertFalse(client.ready(node, TIME.milliseconds())); - assertEquals(100, client.throttleDelayMs(node, TIME.milliseconds())); + assertFalse(client.ready(node, time.milliseconds())); + assertEquals(100, client.throttleDelayMs(node, time.milliseconds())); // After 50ms, the connection is not ready yet. - TIME.sleep(50); - assertFalse(client.ready(node, TIME.milliseconds())); - assertEquals(50, client.throttleDelayMs(node, TIME.milliseconds())); + time.sleep(50); + assertFalse(client.ready(node, time.milliseconds())); + assertEquals(50, client.throttleDelayMs(node, time.milliseconds())); // After another 50ms, the throttling is done and the connection becomes ready again. - TIME.sleep(50); - assertTrue(client.ready(node, TIME.milliseconds())); - assertEquals(0, client.throttleDelayMs(node, TIME.milliseconds())); + time.sleep(50); + assertTrue(client.ready(node, time.milliseconds())); + assertEquals(0, client.throttleDelayMs(node, time.milliseconds())); } private int sendEmptyProduceRequest() { @@ -775,9 +776,9 @@ private int sendEmptyProduceRequest(NetworkClient client, String nodeId) { .setAcks((short) 1) .setTimeoutMs(1000)); TestCallbackHandler handler = new TestCallbackHandler(); - ClientRequest request = client.newClientRequest(nodeId, builder, TIME.milliseconds(), true, + ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, defaultRequestTimeoutMs, handler); - client.send(request, TIME.milliseconds()); + client.send(request, time.milliseconds()); return request.correlationId(); } @@ -793,31 +794,31 @@ private void sendThrottledProduceResponse(int correlationId, int throttleMs, sho @Test public void testLeastLoadedNode() { - client.ready(node, TIME.milliseconds()); - assertFalse(client.isReady(node, TIME.milliseconds())); - LeastLoadedNode leastLoadedNode = client.leastLoadedNode(TIME.milliseconds()); + client.ready(node, time.milliseconds()); + assertFalse(client.isReady(node, time.milliseconds())); + LeastLoadedNode leastLoadedNode = client.leastLoadedNode(time.milliseconds()); assertEquals(node, leastLoadedNode.node()); assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); awaitReady(client, node); - client.poll(1, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds()), "The client should be ready"); + client.poll(1, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); // leastloadednode should be our single node - leastLoadedNode = client.leastLoadedNode(TIME.milliseconds()); + leastLoadedNode = client.leastLoadedNode(time.milliseconds()); assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); Node leastNode = leastLoadedNode.node(); assertEquals(leastNode.id(), node.id(), "There should be one leastloadednode"); // sleep for longer than reconnect backoff - TIME.sleep(reconnectBackoffMsTest); + time.sleep(reconnectBackoffMsTest); // CLOSE node selector.serverDisconnect(node.idString()); - client.poll(1, TIME.milliseconds()); - assertFalse(client.ready(node, TIME.milliseconds()), "After we forced the disconnection the client is no longer ready."); - leastLoadedNode = client.leastLoadedNode(TIME.milliseconds()); + client.poll(1, time.milliseconds()); + assertFalse(client.ready(node, time.milliseconds()), "After we forced the disconnection the client is no longer ready."); + leastLoadedNode = client.leastLoadedNode(time.milliseconds()); assertFalse(leastLoadedNode.hasNodeAvailableOrConnectionReady()); assertNull(leastLoadedNode.node(), "There should be NO leastloadednode"); } @@ -827,7 +828,7 @@ public void testHasNodeAvailableOrConnectionReady() { NetworkClient client = createNetworkClientWithMaxInFlightRequestsPerConnection(1, reconnectBackoffMaxMsTest); awaitReady(client, node); - long now = TIME.milliseconds(); + long now = time.milliseconds(); LeastLoadedNode leastLoadedNode = client.leastLoadedNode(now); assertEquals(node, leastLoadedNode.node()); assertTrue(leastLoadedNode.hasNodeAvailableOrConnectionReady()); @@ -849,13 +850,13 @@ public void testLeastLoadedNodeProvideDisconnectedNodesPrioritizedByLastConnecti Set providedNodeIds = new HashSet<>(); for (int i = 0; i < nodeNumber * 10; i++) { - Node node = client.leastLoadedNode(TIME.milliseconds()).node(); + Node node = client.leastLoadedNode(time.milliseconds()).node(); assertNotNull(node, "Should provide a node"); providedNodeIds.add(node); - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); client.disconnect(node.idString()); - TIME.sleep(connectionSetupTimeoutMsTest + 1); - client.poll(0, TIME.milliseconds()); + time.sleep(connectionSetupTimeoutMsTest + 1); + client.poll(0, time.milliseconds()); // Define a round as nodeNumber of nodes have been provided // In each round every node should be provided exactly once if ((i + 1) % nodeNumber == 0) { @@ -871,7 +872,7 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); Metadata metadata = new Metadata(refreshBackoffMs, refreshBackoffMs, 5000, new LogContext(), new ClusterResourceListeners()); - metadata.updateWithCurrentRequestVersion(metadataResponse, false, TIME.milliseconds()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds()); Cluster cluster = metadata.fetch(); Node node1 = cluster.nodes().get(0); @@ -882,18 +883,18 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { awaitReady(client, node1); metadata.requestUpdate(true); - TIME.sleep(refreshBackoffMs); + time.sleep(refreshBackoffMs); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); Optional nodeWithPendingMetadataOpt = cluster.nodes().stream() .filter(node -> client.hasInFlightRequests(node.idString())) .findFirst(); assertEquals(Optional.of(node1), nodeWithPendingMetadataOpt); - assertFalse(client.ready(node2, TIME.milliseconds())); + assertFalse(client.ready(node2, time.milliseconds())); selector.serverAuthenticationFailed(node2.idString()); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertNotNull(client.authenticationException(node2)); ByteBuffer requestBuffer = selector.completedSendBuffers().get(0).buffer(); @@ -904,30 +905,30 @@ public void testAuthenticationFailureWithInFlightMetadataRequest() { selector.delayedReceive(new DelayedReceive(node1.idString(), new NetworkReceive(node1.idString(), responseBuffer))); int initialUpdateVersion = metadata.updateVersion(); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(initialUpdateVersion + 1, metadata.updateVersion()); } @Test public void testLeastLoadedNodeConsidersThrottledConnections() { - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); awaitReady(client, node); - client.poll(1, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds()), "The client should be ready"); + client.poll(1, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds()), "The client should be ready"); int correlationId = sendEmptyProduceRequest(); - client.poll(1, TIME.milliseconds()); + client.poll(1, time.milliseconds()); sendThrottledProduceResponse(correlationId, 100, PRODUCE.latestVersion()); - client.poll(1, TIME.milliseconds()); + client.poll(1, time.milliseconds()); // leastloadednode should return null since the node is throttled - assertNull(client.leastLoadedNode(TIME.milliseconds()).node()); + assertNull(client.leastLoadedNode(time.milliseconds()).node()); } @Test public void testConnectionDelayWithNoExponentialBackoff() { - long now = TIME.milliseconds(); + long now = time.milliseconds(); long delay = clientWithNoExponentialBackoff.connectionDelay(node, now); assertEquals(0, delay); @@ -937,7 +938,7 @@ public void testConnectionDelayWithNoExponentialBackoff() { public void testConnectionDelayConnectedWithNoExponentialBackoff() { awaitReady(clientWithNoExponentialBackoff, node); - long now = TIME.milliseconds(); + long now = time.milliseconds(); long delay = clientWithNoExponentialBackoff.connectionDelay(node, now); assertEquals(Long.MAX_VALUE, delay); @@ -948,19 +949,19 @@ public void testConnectionDelayDisconnectedWithNoExponentialBackoff() { awaitReady(clientWithNoExponentialBackoff, node); selector.serverDisconnect(node.idString()); - clientWithNoExponentialBackoff.poll(defaultRequestTimeoutMs, TIME.milliseconds()); - long delay = clientWithNoExponentialBackoff.connectionDelay(node, TIME.milliseconds()); + clientWithNoExponentialBackoff.poll(defaultRequestTimeoutMs, time.milliseconds()); + long delay = clientWithNoExponentialBackoff.connectionDelay(node, time.milliseconds()); assertEquals(reconnectBackoffMsTest, delay); // Sleep until there is no connection delay - TIME.sleep(delay); - assertEquals(0, clientWithNoExponentialBackoff.connectionDelay(node, TIME.milliseconds())); + time.sleep(delay); + assertEquals(0, clientWithNoExponentialBackoff.connectionDelay(node, time.milliseconds())); // Start connecting and disconnect before the connection is established - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); selector.serverDisconnect(node.idString()); - client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); + client.poll(defaultRequestTimeoutMs, time.milliseconds()); // Second attempt should have the same behaviour as exponential backoff is disabled assertEquals(reconnectBackoffMsTest, delay); @@ -968,7 +969,7 @@ public void testConnectionDelayDisconnectedWithNoExponentialBackoff() { @Test public void testConnectionDelay() { - long now = TIME.milliseconds(); + long now = time.milliseconds(); long delay = client.connectionDelay(node, now); assertEquals(0, delay); @@ -978,7 +979,7 @@ public void testConnectionDelay() { public void testConnectionDelayConnected() { awaitReady(client, node); - long now = TIME.milliseconds(); + long now = time.milliseconds(); long delay = client.connectionDelay(node, now); assertEquals(Long.MAX_VALUE, delay); @@ -990,24 +991,24 @@ public void testConnectionDelayDisconnected() { // First disconnection selector.serverDisconnect(node.idString()); - client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); - long delay = client.connectionDelay(node, TIME.milliseconds()); + client.poll(defaultRequestTimeoutMs, time.milliseconds()); + long delay = client.connectionDelay(node, time.milliseconds()); long expectedDelay = reconnectBackoffMsTest; double jitter = 0.3; assertEquals(expectedDelay, delay, expectedDelay * jitter); // Sleep until there is no connection delay - TIME.sleep(delay); - assertEquals(0, client.connectionDelay(node, TIME.milliseconds())); + time.sleep(delay); + assertEquals(0, client.connectionDelay(node, time.milliseconds())); // Start connecting and disconnect before the connection is established - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); selector.serverDisconnect(node.idString()); - client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); + client.poll(defaultRequestTimeoutMs, time.milliseconds()); // Second attempt should take twice as long with twice the jitter expectedDelay = Math.round(delay * 2); - delay = client.connectionDelay(node, TIME.milliseconds()); + delay = client.connectionDelay(node, time.milliseconds()); jitter = 0.6; assertEquals(expectedDelay, delay, expectedDelay * jitter); } @@ -1019,7 +1020,7 @@ public void testDisconnectDuringUserMetadataRequest() { awaitReady(client, node); MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); - long now = TIME.milliseconds(); + long now = time.milliseconds(); ClientRequest request = client.newClientRequest(node.idString(), builder, now, true); client.send(request, now); client.poll(defaultRequestTimeoutMs, now); @@ -1028,7 +1029,7 @@ public void testDisconnectDuringUserMetadataRequest() { assertTrue(client.hasInFlightRequests()); selector.close(node.idString()); - List responses = client.poll(defaultRequestTimeoutMs, TIME.milliseconds()); + List responses = client.poll(defaultRequestTimeoutMs, time.milliseconds()); assertEquals(1, responses.size()); assertTrue(responses.iterator().next().wasDisconnected()); } @@ -1045,19 +1046,19 @@ public void testServerDisconnectAfterInternalApiVersionRequest() throws Exceptio selector.serverDisconnect(node.idString()); // The failed ApiVersion request should not be forwarded to upper layers - List responses = client.poll(0, TIME.milliseconds()); + List responses = client.poll(0, time.milliseconds()); assertFalse(client.hasInFlightRequests(node.idString())); assertTrue(responses.isEmpty()); long expectedBackoff = Math.round(Math.pow(reconnectBackoffExpBase, Math.min(i, reconnectBackoffMaxExp)) * reconnectBackoffMsTest); - long delay = client.connectionDelay(node, TIME.milliseconds()); + long delay = client.connectionDelay(node, time.milliseconds()); double reconnectBackoffJitter = ClusterConnectionStates.RECONNECT_BACKOFF_JITTER; assertEquals(expectedBackoff, delay, reconnectBackoffJitter * expectedBackoff); if (i == numIterations - 1) { break; } - TIME.sleep(delay + 1); + time.sleep(delay + 1); } } @@ -1068,7 +1069,7 @@ public void testClientDisconnectAfterInternalApiVersionRequest() throws Exceptio assertFalse(client.hasInFlightRequests(node.idString())); // The failed ApiVersion request should not be forwarded to upper layers - List responses = client.poll(0, TIME.milliseconds()); + List responses = client.poll(0, time.milliseconds()); assertTrue(responses.isEmpty()); } @@ -1076,11 +1077,11 @@ public void testClientDisconnectAfterInternalApiVersionRequest() throws Exceptio public void testDisconnectWithMultipleInFlights() { NetworkClient client = this.clientWithNoVersionDiscovery; awaitReady(client, node); - assertTrue(client.isReady(node, TIME.milliseconds()), + assertTrue(client.isReady(node, time.milliseconds()), "Expected NetworkClient to be ready to send to node " + node.idString()); MetadataRequest.Builder builder = new MetadataRequest.Builder(Collections.emptyList(), true); - long now = TIME.milliseconds(); + long now = time.milliseconds(); final List callbackResponses = new ArrayList<>(); RequestCompletionHandler callback = callbackResponses::add; @@ -1100,7 +1101,7 @@ public void testDisconnectWithMultipleInFlights() { client.disconnect(node.idString()); - List responses = client.poll(0, TIME.milliseconds()); + List responses = client.poll(0, time.milliseconds()); assertEquals(2, responses.size()); assertEquals(responses, callbackResponses); assertEquals(0, client.inFlightRequestCount()); @@ -1119,22 +1120,22 @@ public void testDisconnectWithMultipleInFlights() { @Test public void testCallDisconnect() { awaitReady(client, node); - assertTrue(client.isReady(node, TIME.milliseconds()), + assertTrue(client.isReady(node, time.milliseconds()), "Expected NetworkClient to be ready to send to node " + node.idString()); assertFalse(client.connectionFailed(node), "Did not expect connection to node " + node.idString() + " to be failed"); client.disconnect(node.idString()); - assertFalse(client.isReady(node, TIME.milliseconds()), + assertFalse(client.isReady(node, time.milliseconds()), "Expected node " + node.idString() + " to be disconnected."); assertTrue(client.connectionFailed(node), "Expected connection to node " + node.idString() + " to be failed after disconnect"); - assertFalse(client.canConnect(node, TIME.milliseconds())); + assertFalse(client.canConnect(node, time.milliseconds())); // ensure disconnect does not reset backoff period if already disconnected - TIME.sleep(reconnectBackoffMaxMsTest); - assertTrue(client.canConnect(node, TIME.milliseconds())); + time.sleep(reconnectBackoffMaxMsTest); + assertTrue(client.canConnect(node, time.milliseconds())); client.disconnect(node.idString()); - assertTrue(client.canConnect(node, TIME.milliseconds())); + assertTrue(client.canConnect(node, time.milliseconds())); } @Test @@ -1153,7 +1154,7 @@ public void testReconnectAfterAddressChange() { initialAddresses.toArray(new InetAddress[0]), newAddresses.toArray(new InetAddress[0])); AtomicInteger initialAddressConns = new AtomicInteger(); AtomicInteger newAddressConns = new AtomicInteger(); - MockSelector selector = new MockSelector(TIME, inetSocketAddress -> { + MockSelector selector = new MockSelector(time, inetSocketAddress -> { InetAddress inetAddress = inetSocketAddress.getAddress(); if (initialAddresses.contains(inetAddress)) { initialAddressConns.incrementAndGet(); @@ -1170,35 +1171,35 @@ public void testReconnectAfterAddressChange() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // Connect to one the initial addresses, then change the addresses and disconnect - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); // First poll should try to update the node but couldn't because node remains in connecting state // i.e. connection handling is completed after telemetry update. assertNull(client.telemetryConnectedNode()); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); mockHostResolver.changeAddresses(); selector.serverDisconnect(node.idString()); - client.poll(0, TIME.milliseconds()); - assertFalse(client.isReady(node, TIME.milliseconds())); + client.poll(0, time.milliseconds()); + assertFalse(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); - TIME.sleep(reconnectBackoffMaxMsTest); - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + time.sleep(reconnectBackoffMaxMsTest); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); // We should have tried to connect to one initial address and one new address, and resolved DNS twice @@ -1214,7 +1215,7 @@ public void testFailedConnectionToFirstAddress() { initialAddresses.toArray(new InetAddress[0]), newAddresses.toArray(new InetAddress[0])); AtomicInteger initialAddressConns = new AtomicInteger(); AtomicInteger newAddressConns = new AtomicInteger(); - MockSelector selector = new MockSelector(this.TIME, inetSocketAddress -> { + MockSelector selector = new MockSelector(this.time, inetSocketAddress -> { InetAddress inetAddress = inetSocketAddress.getAddress(); if (initialAddresses.contains(inetAddress)) { initialAddressConns.incrementAndGet(); @@ -1231,26 +1232,26 @@ public void testFailedConnectionToFirstAddress() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // First connection attempt should fail - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertFalse(client.isReady(node, TIME.milliseconds())); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertFalse(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); // Second connection attempt should succeed - TIME.sleep(reconnectBackoffMaxMsTest); - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + time.sleep(reconnectBackoffMaxMsTest); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); // Next client poll after handling connection setup should update telemetry node. - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); // We should have tried to connect to two of the initial addresses, none of the new address, and should @@ -1267,7 +1268,7 @@ public void testFailedConnectionToFirstAddressAfterReconnect() { initialAddresses.toArray(new InetAddress[0]), newAddresses.toArray(new InetAddress[0])); AtomicInteger initialAddressConns = new AtomicInteger(); AtomicInteger newAddressConns = new AtomicInteger(); - MockSelector selector = new MockSelector(TIME, inetSocketAddress -> { + MockSelector selector = new MockSelector(time, inetSocketAddress -> { InetAddress inetAddress = inetSocketAddress.getAddress(); if (initialAddresses.contains(inetAddress)) { initialAddressConns.incrementAndGet(); @@ -1284,43 +1285,43 @@ public void testFailedConnectionToFirstAddressAfterReconnect() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - TIME, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, + time, false, new ApiVersions(), null, new LogContext(), mockHostResolver, mockClientTelemetrySender, Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // Connect to one the initial addresses, then change the addresses and disconnect - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); // Next client poll after handling connection setup should update telemetry node. - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); mockHostResolver.changeAddresses(); selector.serverDisconnect(node.idString()); - client.poll(0, TIME.milliseconds()); - assertFalse(client.isReady(node, TIME.milliseconds())); + client.poll(0, time.milliseconds()); + assertFalse(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); // First connection attempt to new addresses should fail - TIME.sleep(reconnectBackoffMaxMsTest); - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertFalse(client.isReady(node, TIME.milliseconds())); + time.sleep(reconnectBackoffMaxMsTest); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertFalse(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); // Second connection attempt to new addresses should succeed - TIME.sleep(reconnectBackoffMaxMsTest); - client.ready(node, TIME.milliseconds()); - TIME.sleep(connectionSetupTimeoutMaxMsTest); - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + time.sleep(reconnectBackoffMaxMsTest); + client.ready(node, time.milliseconds()); + time.sleep(connectionSetupTimeoutMaxMsTest); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); assertNull(client.telemetryConnectedNode()); // Next client poll after handling connection setup should update telemetry node. - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); assertEquals(node, client.telemetryConnectedNode()); // We should have tried to connect to one of the initial addresses and two of the new addresses (the first one @@ -1336,32 +1337,32 @@ public void testCloseConnectingNode() { Cluster cluster = TestUtils.clusterWith(2); Node node0 = cluster.nodeById(0); Node node1 = cluster.nodeById(1); - client.ready(node0, TIME.milliseconds()); + client.ready(node0, time.milliseconds()); selector.serverConnectionBlocked(node0.idString()); - client.poll(1, TIME.milliseconds()); + client.poll(1, time.milliseconds()); client.close(node0.idString()); // Poll without any connections should return without exceptions - client.poll(0, TIME.milliseconds()); - assertFalse(NetworkClientUtils.isReady(client, node0, TIME.milliseconds())); - assertFalse(NetworkClientUtils.isReady(client, node1, TIME.milliseconds())); + client.poll(0, time.milliseconds()); + assertFalse(NetworkClientUtils.isReady(client, node0, time.milliseconds())); + assertFalse(NetworkClientUtils.isReady(client, node1, time.milliseconds())); // Connection to new node should work - client.ready(node1, TIME.milliseconds()); + client.ready(node1, time.milliseconds()); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(defaultApiVersionsResponse(), ApiKeys.API_VERSIONS.latestVersion(), 0); selector.delayedReceive(new DelayedReceive(node1.idString(), new NetworkReceive(node1.idString(), buffer))); - while (!client.ready(node1, TIME.milliseconds())) - client.poll(1, TIME.milliseconds()); - assertTrue(client.isReady(node1, TIME.milliseconds())); + while (!client.ready(node1, time.milliseconds())) + client.poll(1, time.milliseconds()); + assertTrue(client.isReady(node1, time.milliseconds())); selector.clear(); // New connection to node closed earlier should work - client.ready(node0, TIME.milliseconds()); + client.ready(node0, time.milliseconds()); buffer = RequestTestUtils.serializeResponseWithHeader(defaultApiVersionsResponse(), ApiKeys.API_VERSIONS.latestVersion(), 1); selector.delayedReceive(new DelayedReceive(node0.idString(), new NetworkReceive(node0.idString(), buffer))); - while (!client.ready(node0, TIME.milliseconds())) - client.poll(1, TIME.milliseconds()); - assertTrue(client.isReady(node0, TIME.milliseconds())); + while (!client.ready(node0, time.milliseconds())) + client.poll(1, time.milliseconds()); + assertTrue(client.isReady(node0, time.milliseconds())); } @Test @@ -1370,18 +1371,18 @@ public void testConnectionDoesNotRemainStuckInCheckingApiVersionsStateIfChannelN final Node node = cluster.nodeById(0); // Channel is ready by default so we mark it as not ready. - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); selector.channelNotReady(node.idString()); // Channel should not be ready. - client.poll(0, TIME.milliseconds()); - assertFalse(NetworkClientUtils.isReady(client, node, TIME.milliseconds())); + client.poll(0, time.milliseconds()); + assertFalse(NetworkClientUtils.isReady(client, node, time.milliseconds())); // Connection should time out if the channel does not become ready within // the connection setup timeout. This ensures that the client does not remain // stuck in the CHECKING_API_VERSIONS state. - TIME.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); - client.poll(0, TIME.milliseconds()); + time.sleep((long) (connectionSetupTimeoutMsTest * 1.2) + 1); + client.poll(0, time.milliseconds()); assertTrue(client.connectionFailed(node)); } @@ -1393,18 +1394,18 @@ public void testTelemetryRequest() { NetworkClient client = new NetworkClient(metadataUpdater, null, selector, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, reconnectBackoffMaxMsTest, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, - TIME, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender, + time, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver(), mockClientTelemetrySender, Long.MAX_VALUE, MetadataRecoveryStrategy.NONE, bootstrapConfiguration); // Send the ApiVersionsRequest - client.ready(node, TIME.milliseconds()); - client.poll(0, TIME.milliseconds()); + client.ready(node, time.milliseconds()); + client.poll(0, time.milliseconds()); assertNull(client.telemetryConnectedNode()); assertTrue(client.hasInFlightRequests(node.idString())); delayedApiVersionsResponse(0, ApiKeys.API_VERSIONS.latestVersion(), TestUtils.defaultApiVersionsResponse( ApiMessageType.ListenerType.BROKER)); // handle ApiVersionsResponse - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); // the ApiVersionsRequest is gone assertFalse(client.hasInFlightRequests(node.idString())); selector.clear(); @@ -1418,8 +1419,8 @@ TIME, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver() selector.completeReceive(new NetworkReceive(node.idString(), buffer)); // Initiate poll to send GetTelemetrySubscriptions request - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); assertEquals(node, client.telemetryConnectedNode()); verify(mockClientTelemetrySender, times(1)).handleResponse(any(GetTelemetrySubscriptionsResponse.class)); selector.clear(); @@ -1433,8 +1434,8 @@ TIME, true, new ApiVersions(), null, new LogContext(), new DefaultHostResolver() selector.completeReceive(new NetworkReceive(node.idString(), pushBuffer)); // Initiate poll to send PushTelemetry request - client.poll(0, TIME.milliseconds()); - assertTrue(client.isReady(node, TIME.milliseconds())); + client.poll(0, time.milliseconds()); + assertTrue(client.isReady(node, time.milliseconds())); assertEquals(node, client.telemetryConnectedNode()); verify(mockClientTelemetrySender, times(1)).handleResponse(any(PushTelemetryResponse.class)); verify(mockClientTelemetrySender, times(4)).timeToNextUpdate(anyLong()); @@ -1447,12 +1448,12 @@ private RequestHeader parseHeader(ByteBuffer buffer) { } private void awaitInFlightApiVersionRequest() throws Exception { - client.ready(node, TIME.milliseconds()); + client.ready(node, time.milliseconds()); TestUtils.waitForCondition(() -> { - client.poll(0, TIME.milliseconds()); + client.poll(0, time.milliseconds()); return client.hasInFlightRequests(node.idString()); }, 1000, ""); - assertFalse(client.isReady(node, TIME.milliseconds())); + assertFalse(client.isReady(node, time.milliseconds())); } private ApiVersionsResponse defaultApiVersionsResponse() { @@ -1521,4 +1522,100 @@ private void bootstrapMetadataUpdater(final MetadataUpdater metadataUpdater) { metadataUpdater.bootstrap(serverAddresses); System.out.println("Bootstraping metadata------:" + metadataUpdater.isBootstrapped()); } + + @Test + public void testEnsureBootstrappedSuccess() { + Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); + NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + BOOTSTRAP_ADDRESSES, + ClientDnsLookup.USE_ALL_DNS_IPS, + 5000 + ); + NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, + reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, + time, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, config); + + // First poll should succeed in bootstrapping + client.poll(1000, time.milliseconds()); + + // Verify bootstrap succeeded + MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); + assertTrue(metadataUpdater.isBootstrapped()); + } + + @Test + public void testEnsureBootstrappedTimeoutThrowsException() { + Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); + // Use invalid addresses that cannot be resolved + List invalidAddresses = List.of("invalid.host.that.does.not.exist:9092"); + NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + invalidAddresses, + ClientDnsLookup.USE_ALL_DNS_IPS, + 100 // Short timeout + ); + NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, + reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, + time, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, config); + + // Advance time past bootstrap timeout + time.sleep(150); + + // Should throw BootstrapResolutionException + assertThrows(BootstrapResolutionException.class, () -> client.poll(1000, time.milliseconds())); + } + + @Test + public void testEnsureBootstrappedPollTimeoutReturnsWithoutError() { + Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); + // Use invalid addresses that cannot be resolved + List invalidAddresses = List.of("invalid.host.that.does.not.exist:9092"); + NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + invalidAddresses, + ClientDnsLookup.USE_ALL_DNS_IPS, + 5000 // Long bootstrap timeout + ); + NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, + reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, + time, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, config); + + // Directly call ensureBootstrapped with short poll timeout (100ms) + // Should return without error even though bootstrap hasn't succeeded + client.ensureBootstrapped(100, time.milliseconds()); + + // Verify bootstrap has not succeeded yet + MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); + assertFalse(metadataUpdater.isBootstrapped()); + } + + @Test + public void testEnsureBootstrappedRetryUntilSuccess() { + Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); + NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + BOOTSTRAP_ADDRESSES, + ClientDnsLookup.USE_ALL_DNS_IPS, + 5000 + ); + NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, + reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, + defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, + time, false, new ApiVersions(), new LogContext(), + MetadataRecoveryStrategy.NONE, config); + + // First poll should succeed in bootstrapping + client.poll(1000, time.milliseconds()); + + // Verify bootstrap succeeded + MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); + assertTrue(metadataUpdater.isBootstrapped()); + + // Subsequent polls should not fail even if already bootstrapped + client.poll(1000, time.milliseconds()); + assertTrue(metadataUpdater.isBootstrapped()); + } } From 8d36da7d890021ca5c10419ecee1b85a5332ba2b Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Mon, 9 Mar 2026 10:11:37 +0000 Subject: [PATCH 14/30] fix failed tests --- .../org/apache/kafka/clients/ClientUtils.java | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index a1d120cfd6566..493e0e5aef04a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -275,14 +275,31 @@ public static NetworkClient createNetworkClient(AbstractConfig config, List bootstrapServers = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); ClientDnsLookup dnsLookup = ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)); - // Only validate if bootstrap servers are provided (non-empty list) - // This allows configurations that don't use bootstrap (e.g., broker-to-broker) to skip validation + // Try to get bootstrap.controllers if it exists (only defined in AdminClientConfig) + List bootstrapControllers = null; + try { + bootstrapControllers = config.getList("bootstrap.controllers"); + } catch (ConfigException e) { + // bootstrap.controllers is not defined in this config (e.g., ProducerConfig, ConsumerConfig) + // This is expected and not an error + } + + // Determine which bootstrap addresses to use + List bootstrapAddresses; if (bootstrapServers != null && !bootstrapServers.isEmpty()) { + bootstrapAddresses = bootstrapServers; + // Only validate if bootstrap servers are provided (non-empty list) + // This allows configurations that don't use bootstrap (e.g., broker-to-broker) to skip validation parseAndValidateAddresses(bootstrapServers, dnsLookup); + } else if (bootstrapControllers != null && !bootstrapControllers.isEmpty()) { + bootstrapAddresses = bootstrapControllers; + parseAndValidateAddresses(bootstrapControllers, dnsLookup); + } else { + bootstrapAddresses = List.of(); } bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - bootstrapServers, + bootstrapAddresses, dnsLookup, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS ); From ef3bb8f7071ad8c71222c300015f851dd4c35962 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 17 Mar 2026 17:50:10 +0000 Subject: [PATCH 15/30] address comments --- .../main/java/org/apache/kafka/clients/ClientUtils.java | 5 +++++ .../main/java/org/apache/kafka/clients/NetworkClient.java | 7 ++++++- .../org/apache/kafka/tools/ReplicaVerificationTool.java | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 493e0e5aef04a..728abff58af8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -105,6 +105,11 @@ public static List validateAddresses(List urls, Clien final String host = getHost(url); final Integer port = getPort(url); + if (host == null || port == null) { + log.warn("Skipping invalid bootstrap URL: {}", url); + return; + } + try { addresses.addAll(resolveAddress(url, host, port, clientDnsLookup)); } catch (UnknownHostException e) { diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 6a1ef36eb49c4..b4915c50dc975 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1262,7 +1262,12 @@ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { if (bootstrapState.isDisabled() || metadataUpdater.isBootstrapped()) return; - long pollDeadlineMs = currentTimeMs + pollTimeoutMs; + // Handle potential overflow when adding timeout to current time + long pollDeadlineMs; + if (currentTimeMs > Long.MAX_VALUE - pollTimeoutMs) + pollDeadlineMs = Long.MAX_VALUE; + else + pollDeadlineMs = currentTimeMs + pollTimeoutMs; while (true) { long now = time.milliseconds(); diff --git a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java index 8e937e4b9e697..dcd9ccd913dfa 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java @@ -676,7 +676,7 @@ private static class ReplicaFetcherBlockingSend { NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( consumerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(consumerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - consumerConfig.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG)); + consumerConfig.getLong(ConsumerConfig.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG)); this.networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), From 7458b72be6da2604a3a42cd9fa0110d3095485b9 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 18 Mar 2026 11:17:41 +0000 Subject: [PATCH 16/30] address comments --- .../org/apache/kafka/clients/ClientUtils.java | 31 ++++++------------- .../org/apache/kafka/clients/KafkaClient.java | 2 +- .../apache/kafka/clients/NetworkClient.java | 17 ++++++++-- .../kafka/clients/admin/KafkaAdminClient.java | 7 +++++ .../consumer/internals/ConsumerUtils.java | 1 + .../internals/NetworkClientDelegate.java | 1 + .../kafka/clients/producer/KafkaProducer.java | 1 + .../BootstrapResolutionException.java | 2 +- .../kafka/clients/NetworkClientTest.java | 2 +- 9 files changed, 37 insertions(+), 27 deletions(-) rename clients/src/main/java/org/apache/kafka/common/{network => errors}/BootstrapResolutionException.java (97%) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 728abff58af8a..b0729631f31fc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -96,7 +96,7 @@ private static List resolveAddress( return addresses; } - public static List validateAddresses(List urls, ClientDnsLookup clientDnsLookup) { + public static List parseAddresses(List urls, ClientDnsLookup clientDnsLookup) { List addresses = new ArrayList<>(); if (urls == null) { return addresses; @@ -199,6 +199,7 @@ static List filterPreferredAddresses(InetAddress[] allAddresses) { } public static NetworkClient createNetworkClient(AbstractConfig config, + List bootstrapServers, Metrics metrics, String metricsGroupPrefix, LogContext logContext, @@ -209,6 +210,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, Sensor throttleTimeSensor, ClientTelemetrySender clientTelemetrySender) { return createNetworkClient(config, + bootstrapServers, config.getString(CommonClientConfigs.CLIENT_ID_CONFIG), metrics, metricsGroupPrefix, @@ -225,6 +227,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, } public static NetworkClient createNetworkClient(AbstractConfig config, + List bootstrapServers, String clientId, Metrics metrics, String metricsGroupPrefix, @@ -236,6 +239,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, MetadataUpdater metadataUpdater, HostResolver hostResolver) { return createNetworkClient(config, + bootstrapServers, clientId, metrics, metricsGroupPrefix, @@ -252,6 +256,7 @@ public static NetworkClient createNetworkClient(AbstractConfig config, } public static NetworkClient createNetworkClient(AbstractConfig config, + List bootstrapServers, String clientId, Metrics metrics, String metricsGroupPrefix, @@ -277,34 +282,16 @@ public static NetworkClient createNetworkClient(AbstractConfig config, metricsGroupPrefix, channelBuilder, logContext); - List bootstrapServers = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); ClientDnsLookup dnsLookup = ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)); - // Try to get bootstrap.controllers if it exists (only defined in AdminClientConfig) - List bootstrapControllers = null; - try { - bootstrapControllers = config.getList("bootstrap.controllers"); - } catch (ConfigException e) { - // bootstrap.controllers is not defined in this config (e.g., ProducerConfig, ConsumerConfig) - // This is expected and not an error - } - - // Determine which bootstrap addresses to use - List bootstrapAddresses; + // Validate bootstrap servers if provided (non-empty list) + // This allows configurations that don't use bootstrap (e.g., broker-to-broker) to skip validation if (bootstrapServers != null && !bootstrapServers.isEmpty()) { - bootstrapAddresses = bootstrapServers; - // Only validate if bootstrap servers are provided (non-empty list) - // This allows configurations that don't use bootstrap (e.g., broker-to-broker) to skip validation parseAndValidateAddresses(bootstrapServers, dnsLookup); - } else if (bootstrapControllers != null && !bootstrapControllers.isEmpty()) { - bootstrapAddresses = bootstrapControllers; - parseAndValidateAddresses(bootstrapControllers, dnsLookup); - } else { - bootstrapAddresses = List.of(); } bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( - bootstrapAddresses, + bootstrapServers != null ? bootstrapServers : List.of(), dnsLookup, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS ); diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 5332fc4c25ae4..46b6498606412 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -212,5 +212,5 @@ ClientRequest newClientRequest(String nodeId, * was invoked for this client. */ boolean active(); - + } diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index b4915c50dc975..5640bb763023d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -21,11 +21,11 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.BootstrapResolutionException; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.network.BootstrapResolutionException; import org.apache.kafka.common.network.ChannelState; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkSend; @@ -1223,6 +1223,7 @@ private class BootstrapState { private final ClientDnsLookup clientDnsLookup; private final long dnsResolutionTimeoutMs; private final boolean isDisabled; + private boolean timerStarted; BootstrapState(BootstrapConfiguration bootstrapConfiguration) { this.dnsResolutionTimeoutMs = bootstrapConfiguration.bootstrapResolveTimeoutMs; @@ -1230,11 +1231,20 @@ private class BootstrapState { this.bootstrapServers = bootstrapConfiguration.bootstrapServers; this.clientDnsLookup = bootstrapConfiguration.clientDnsLookup; this.isDisabled = bootstrapConfiguration.isBootstrapDisabled; + this.timerStarted = false; } boolean isDisabled() { return isDisabled; } + + void ensureTimerStarted(long currentTimeMs) { + if (!timerStarted) { + timer.update(currentTimeMs); + timer.reset(dnsResolutionTimeoutMs); + timerStarted = true; + } + } } /** @@ -1262,6 +1272,9 @@ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { if (bootstrapState.isDisabled() || metadataUpdater.isBootstrapped()) return; + // Start the bootstrap timer on first call to ensure it starts counting from the first poll + bootstrapState.ensureTimerStarted(currentTimeMs); + // Handle potential overflow when adding timeout to current time long pollDeadlineMs; if (currentTimeMs > Long.MAX_VALUE - pollTimeoutMs) @@ -1273,7 +1286,7 @@ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { long now = time.milliseconds(); bootstrapState.timer.update(now); - List servers = ClientUtils.validateAddresses( + List servers = ClientUtils.parseAddresses( bootstrapState.bootstrapServers, bootstrapState.clientDnsLookup); if (!servers.isEmpty()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 85f1e87459f01..7bf4586dec422 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -550,7 +550,14 @@ static KafkaAdminClient createInternal( MetricsContext metricsContext = new KafkaMetricsContext(JMX_PREFIX, config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)); metrics = new Metrics(metricConfig, reporters, time, metricsContext); + + // Use the appropriate bootstrap configuration determined by AdminBootstrapAddresses + List bootstrapAddressesToUse = adminAddresses.usingBootstrapControllers() + ? config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) + : config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); + networkClient = ClientUtils.createNetworkClient(config, + bootstrapAddressesToUse, clientId, metrics, "admin-client", diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java index 2f715e206cc07..10be5854d021c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java @@ -89,6 +89,7 @@ public static ConsumerNetworkClient createConsumerNetworkClient(ConsumerConfig c long retryBackoffMs, ClientTelemetrySender clientTelemetrySender) { NetworkClient netClient = ClientUtils.createNetworkClient(config, + config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG), metrics, CONSUMER_METRIC_GROUP_PREFIX, logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 42c1c73acb565..0b433f33ba088 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -476,6 +476,7 @@ public static Supplier supplier(final Time time, @Override protected NetworkClientDelegate create() { KafkaClient client = ClientUtils.createNetworkClient(config, + config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG), metrics, CONSUMER_METRIC_GROUP_PREFIX, logContext, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 49fb8ee228be2..0625c4e9b056c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -525,6 +525,7 @@ Sender newSender(LogContext logContext, KafkaClient kafkaClient, ProducerMetadat ProducerMetrics metricsRegistry = new ProducerMetrics(this.metrics); Sensor throttleTimeSensor = Sender.throttleTimeSensor(metricsRegistry.senderMetrics); KafkaClient client = kafkaClient != null ? kafkaClient : ClientUtils.createNetworkClient(producerConfig, + producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG), this.metrics, "producer", logContext, diff --git a/clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java b/clients/src/main/java/org/apache/kafka/common/errors/BootstrapResolutionException.java similarity index 97% rename from clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java rename to clients/src/main/java/org/apache/kafka/common/errors/BootstrapResolutionException.java index db92127a0553a..899562af01f92 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/BootstrapResolutionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/BootstrapResolutionException.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.network; +package org.apache.kafka.common.errors; import org.apache.kafka.common.KafkaException; diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 48d63e8524c8a..a737f89771565 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.BootstrapResolutionException; import org.apache.kafka.common.errors.RebootstrapRequiredException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.ClusterResourceListeners; @@ -33,7 +34,6 @@ import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.message.PushTelemetryRequestData; import org.apache.kafka.common.message.PushTelemetryResponseData; -import org.apache.kafka.common.network.BootstrapResolutionException; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; From 993b42b56cf126690a6ebb762f56fe69f3d2f79f Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 18 Mar 2026 14:32:27 +0000 Subject: [PATCH 17/30] address comments --- .../org/apache/kafka/clients/ClientUtils.java | 3 +- .../kafka/clients/ManualMetadataUpdater.java | 7 +- .../apache/kafka/clients/MetadataUpdater.java | 4 +- .../apache/kafka/clients/NetworkClient.java | 76 +++++++------------ .../admin/internals/AdminMetadataManager.java | 6 +- .../kafka/clients/NetworkClientTest.java | 18 +++-- .../distributed/WorkerGroupMember.java | 3 +- .../kafka/tools/BrokerApiVersionsCommand.java | 3 +- .../kafka/tools/ReplicaVerificationTool.java | 3 +- .../workload/ConnectionStressWorker.java | 3 +- 10 files changed, 62 insertions(+), 64 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index b0729631f31fc..e25a6e42b074f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -293,7 +293,8 @@ public static NetworkClient createNetworkClient(AbstractConfig config, bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( bootstrapServers != null ? bootstrapServers : List.of(), dnsLookup, - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG) ); return new NetworkClient(metadataUpdater, metadata, diff --git a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java index ed909850ecf03..7217bdc63e530 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/ManualMetadataUpdater.java @@ -84,12 +84,15 @@ public void handleSuccessfulResponse(RequestHeader requestHeader, long now, Meta @Override public boolean isBootstrapped() { - return false; + // ManualMetadataUpdater is designed for cases where nodes are manually set, + // so we consider it bootstrapped if nodes have been provided + return !nodes.isEmpty(); } @Override public void bootstrap(List addresses) { - + // ManualMetadataUpdater doesn't use NetworkClient's bootstrap mechanism + // Nodes should be set manually via constructor or setNodes() } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java index add46cb7fcbca..7f7f1999e1ef0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java @@ -106,7 +106,9 @@ default void rebootstrap(long now) {} /** * Returns true if the metadata has been bootstrapped. */ - boolean isBootstrapped(); + default boolean isBootstrapped() { + return false; + } /** * Bootstrap the metadata cache with the given addresses. diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 5640bb763023d..567b857405925 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -139,7 +139,11 @@ private enum State { private final AtomicReference state; - private final BootstrapState bootstrapState; + private final BootstrapConfiguration bootstrapConfiguration; + + private final Timer bootstrapTimer; + + private boolean bootstrapTimerStarted; private final TelemetrySender telemetrySender; @@ -359,7 +363,9 @@ public NetworkClient(MetadataUpdater metadataUpdater, this.telemetrySender = (clientTelemetrySender != null) ? new TelemetrySender(clientTelemetrySender) : null; this.rebootstrapTriggerMs = rebootstrapTriggerMs; this.metadataRecoveryStrategy = metadataRecoveryStrategy; - this.bootstrapState = new BootstrapState(bootstrapConfiguration); + this.bootstrapConfiguration = bootstrapConfiguration; + this.bootstrapTimer = time.timer(bootstrapConfiguration.bootstrapResolveTimeoutMs); + this.bootstrapTimerStarted = false; } /** @@ -1195,19 +1201,22 @@ public static class BootstrapConfiguration { public final List bootstrapServers; public final ClientDnsLookup clientDnsLookup; public final long bootstrapResolveTimeoutMs; + public final long retryBackoffMs; private boolean isBootstrapDisabled; public BootstrapConfiguration(final List bootstrapServers, final ClientDnsLookup clientDnsLookup, - final long bootstrapResolveTimeoutMs) { + final long bootstrapResolveTimeoutMs, + final long retryBackoffMs) { this.bootstrapServers = bootstrapServers; this.clientDnsLookup = clientDnsLookup; this.bootstrapResolveTimeoutMs = bootstrapResolveTimeoutMs; + this.retryBackoffMs = retryBackoffMs; this.isBootstrapDisabled = false; } public static BootstrapConfiguration disabled() { - BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration(List.of(), null, 0); + BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration(List.of(), null, 0, 0); bootstrapConfiguration.disableBootstrap(); return bootstrapConfiguration; } @@ -1217,36 +1226,6 @@ public void disableBootstrap() { } } - private class BootstrapState { - private final Timer timer; - private final List bootstrapServers; - private final ClientDnsLookup clientDnsLookup; - private final long dnsResolutionTimeoutMs; - private final boolean isDisabled; - private boolean timerStarted; - - BootstrapState(BootstrapConfiguration bootstrapConfiguration) { - this.dnsResolutionTimeoutMs = bootstrapConfiguration.bootstrapResolveTimeoutMs; - this.timer = time.timer(bootstrapConfiguration.bootstrapResolveTimeoutMs); - this.bootstrapServers = bootstrapConfiguration.bootstrapServers; - this.clientDnsLookup = bootstrapConfiguration.clientDnsLookup; - this.isDisabled = bootstrapConfiguration.isBootstrapDisabled; - this.timerStarted = false; - } - - boolean isDisabled() { - return isDisabled; - } - - void ensureTimerStarted(long currentTimeMs) { - if (!timerStarted) { - timer.update(currentTimeMs); - timer.reset(dnsResolutionTimeoutMs); - timerStarted = true; - } - } - } - /** * Ensures that the client has successfully resolved and bootstrapped with the configured bootstrap servers. * This method will retry DNS resolution until one of the following conditions is met: @@ -1269,11 +1248,15 @@ void ensureTimerStarted(long currentTimeMs) { * (up to 100ms) between attempts until one of the exit conditions above is met. */ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { - if (bootstrapState.isDisabled() || metadataUpdater.isBootstrapped()) + if (bootstrapConfiguration.isBootstrapDisabled || metadataUpdater.isBootstrapped()) return; // Start the bootstrap timer on first call to ensure it starts counting from the first poll - bootstrapState.ensureTimerStarted(currentTimeMs); + if (!bootstrapTimerStarted) { + bootstrapTimer.update(currentTimeMs); + bootstrapTimer.reset(bootstrapConfiguration.bootstrapResolveTimeoutMs); + bootstrapTimerStarted = true; + } // Handle potential overflow when adding timeout to current time long pollDeadlineMs; @@ -1284,36 +1267,33 @@ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { while (true) { long now = time.milliseconds(); - bootstrapState.timer.update(now); + bootstrapTimer.update(now); List servers = ClientUtils.parseAddresses( - bootstrapState.bootstrapServers, bootstrapState.clientDnsLookup); + bootstrapConfiguration.bootstrapServers, bootstrapConfiguration.clientDnsLookup); if (!servers.isEmpty()) { // Resolution succeeded - bootstrapState.timer.reset(bootstrapState.dnsResolutionTimeoutMs); + bootstrapTimer.reset(bootstrapConfiguration.bootstrapResolveTimeoutMs); metadataUpdater.bootstrap(servers); return; } - // Check which timeout expires first - boolean bootstrapExpired = bootstrapState.timer.isExpired(); - boolean pollExpired = now >= pollDeadlineMs; - - if (bootstrapExpired) { + if (bootstrapTimer.isExpired()) { // Bootstrap timeout expired before poll timeout throw new BootstrapResolutionException("Timeout while attempting to resolve bootstrap servers."); } - if (pollExpired) { + if (now >= pollDeadlineMs) { // Poll timeout reached but bootstrap timeout hasn't expired yet return; } - // Sleep briefly before retrying to avoid tight loop + // Sleep before retrying to avoid tight loop and reduce load on DNS server + // Use the standard retry backoff to prevent overloading DNS with requests long remainingPollTimeMs = pollDeadlineMs - now; - long remainingBootstrapTimeMs = bootstrapState.timer.remainingMs(); - long sleepTimeMs = Math.min(Math.min(remainingPollTimeMs, remainingBootstrapTimeMs), 100); + long remainingBootstrapTimeMs = bootstrapTimer.remainingMs(); + long sleepTimeMs = Math.min(Math.min(remainingPollTimeMs, remainingBootstrapTimeMs), bootstrapConfiguration.retryBackoffMs); if (sleepTimeMs > 0) { time.sleep(sleepTimeMs); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java index 43242e7fcdbf7..f0de44a17b782 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java @@ -151,12 +151,14 @@ public void rebootstrap(long now) { @Override public boolean isBootstrapped() { - return false; + // AdminClient bootstraps during construction via metadataManager.update(Cluster.bootstrap(...)) + // so we check if the bootstrap cluster has been set + return bootstrapCluster != null; } @Override public void bootstrap(List addresses) { - // do nothing + // AdminClient handles bootstrap during construction, so this method is not used } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index a737f89771565..20c42cbabd325 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -102,7 +102,8 @@ public class NetworkClientTest { new NetworkClient.BootstrapConfiguration( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, - 10 * 1000); + 10 * 1000, + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); protected final int defaultRequestTimeoutMs = 1000; protected final MockSelector selector = new MockSelector(time); @@ -191,7 +192,8 @@ public void setup() { bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS ); } @@ -1529,7 +1531,8 @@ public void testEnsureBootstrappedSuccess() { NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, - 5000 + 5000, + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS ); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, @@ -1553,7 +1556,8 @@ public void testEnsureBootstrappedTimeoutThrowsException() { NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( invalidAddresses, ClientDnsLookup.USE_ALL_DNS_IPS, - 100 // Short timeout + 100, // Short timeout + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS ); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, @@ -1576,7 +1580,8 @@ public void testEnsureBootstrappedPollTimeoutReturnsWithoutError() { NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( invalidAddresses, ClientDnsLookup.USE_ALL_DNS_IPS, - 5000 // Long bootstrap timeout + 5000, // Long bootstrap timeout + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS ); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, @@ -1599,7 +1604,8 @@ public void testEnsureBootstrappedRetryUntilSuccess() { NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, - 5000 + 5000, + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS ); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index b6e3f41dacd31..814aee099c079 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -110,7 +110,8 @@ public WorkerGroupMember(DistributedConfig config, NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS); + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); NetworkClient netClient = new NetworkClient( new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, channelBuilder, logContext), metadata, diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index 214469f161cd0..bc80d7cbe3f5e 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -185,7 +185,8 @@ static AdminClient create(AbstractConfig config) { NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG) ); NetworkClient networkClient = new NetworkClient( selector, diff --git a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java index dcd9ccd913dfa..543bd3f34b8ad 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java @@ -676,7 +676,8 @@ private static class ReplicaFetcherBlockingSend { NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( consumerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(consumerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - consumerConfig.getLong(ConsumerConfig.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG)); + consumerConfig.getLong(ConsumerConfig.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), + consumerConfig.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); this.networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index cbefc8c86c1e1..ff915bcb838d9 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -169,7 +169,8 @@ public boolean tryConnect() { NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( conf.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(conf.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + conf.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG) ); try (NetworkClient client = new NetworkClient(selector, updater, From 9b534822852282e4cfaa547c6a8132adbae7ab7f Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 18 Mar 2026 16:48:18 +0000 Subject: [PATCH 18/30] address comments --- .../org/apache/kafka/clients/ClientUtils.java | 16 +++------- .../apache/kafka/clients/MetadataUpdater.java | 4 +-- .../apache/kafka/clients/NetworkClient.java | 15 +++------ .../clients/admin/AdminClientConfig.java | 12 +++++++ .../clients/producer/ProducerConfig.java | 9 ++++++ .../kafka/clients/NetworkClientTest.java | 31 ++++++++++++------- .../distributed/WorkerGroupMember.java | 2 +- .../kafka/tools/BrokerApiVersionsCommand.java | 4 +-- .../kafka/tools/ReplicaVerificationTool.java | 2 +- .../workload/ConnectionStressWorker.java | 4 +-- 10 files changed, 57 insertions(+), 42 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index e25a6e42b074f..3195a237bdfed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -272,7 +272,6 @@ public static NetworkClient createNetworkClient(AbstractConfig config, ClientTelemetrySender clientTelemetrySender) { ChannelBuilder channelBuilder = null; Selector selector = null; - NetworkClient.BootstrapConfiguration bootstrapConfiguration; try { channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); @@ -284,18 +283,13 @@ public static NetworkClient createNetworkClient(AbstractConfig config, logContext); ClientDnsLookup dnsLookup = ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)); - // Validate bootstrap servers if provided (non-empty list) - // This allows configurations that don't use bootstrap (e.g., broker-to-broker) to skip validation - if (bootstrapServers != null && !bootstrapServers.isEmpty()) { - parseAndValidateAddresses(bootstrapServers, dnsLookup); - } - - bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( bootstrapServers != null ? bootstrapServers : List.of(), dnsLookup, - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG) - ); + config.getLong(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), + false); + return new NetworkClient(metadataUpdater, metadata, selector, diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java index 7f7f1999e1ef0..add46cb7fcbca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java @@ -106,9 +106,7 @@ default void rebootstrap(long now) {} /** * Returns true if the metadata has been bootstrapped. */ - default boolean isBootstrapped() { - return false; - } + boolean isBootstrapped(); /** * Bootstrap the metadata cache with the given addresses. diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 567b857405925..074cb0ba3085d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1202,27 +1202,22 @@ public static class BootstrapConfiguration { public final ClientDnsLookup clientDnsLookup; public final long bootstrapResolveTimeoutMs; public final long retryBackoffMs; - private boolean isBootstrapDisabled; + private final boolean isBootstrapDisabled; public BootstrapConfiguration(final List bootstrapServers, final ClientDnsLookup clientDnsLookup, final long bootstrapResolveTimeoutMs, - final long retryBackoffMs) { + final long retryBackoffMs, + final boolean isBootstrapDisabled) { this.bootstrapServers = bootstrapServers; this.clientDnsLookup = clientDnsLookup; this.bootstrapResolveTimeoutMs = bootstrapResolveTimeoutMs; this.retryBackoffMs = retryBackoffMs; - this.isBootstrapDisabled = false; + this.isBootstrapDisabled = isBootstrapDisabled; } public static BootstrapConfiguration disabled() { - BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration(List.of(), null, 0, 0); - bootstrapConfiguration.disableBootstrap(); - return bootstrapConfiguration; - } - - public void disableBootstrap() { - this.isBootstrapDisabled = true; + return new BootstrapConfiguration(List.of(), null, 0, 0, true); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java index 471d3916cfb55..5ed8ceef038a2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java @@ -87,6 +87,12 @@ public class AdminClientConfig extends AbstractConfig { public static final String RETRY_BACKOFF_MAX_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MAX_MS_CONFIG; private static final String RETRY_BACKOFF_MAX_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MAX_MS_DOC; + /** + * bootstrap.resolve.timeout.ms + */ + public static final String BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG = CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG; + private static final String BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC = CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC; + /** * enable.metrics.push */ @@ -193,6 +199,12 @@ public class AdminClientConfig extends AbstractConfig { atLeast(0L), Importance.LOW, RETRY_BACKOFF_MAX_MS_DOC) + .define(BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, + Type.LONG, + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + atLeast(0L), + Importance.MEDIUM, + BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, Type.BOOLEAN, false, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index f9a99c5ef08e6..68d53269b1c47 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -224,6 +224,9 @@ public class ProducerConfig extends AbstractConfig { /** retry.backoff.max.ms */ public static final String RETRY_BACKOFF_MAX_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MAX_MS_CONFIG; + /** bootstrap.resolve.timeout.ms */ + public static final String BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG = CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG; + /** * enable.metrics.push */ @@ -427,6 +430,12 @@ public class ProducerConfig extends AbstractConfig { atLeast(0L), Importance.LOW, CommonClientConfigs.RETRY_BACKOFF_MAX_MS_DOC) + .define(BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, + Type.LONG, + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + atLeast(0L), + Importance.MEDIUM, + CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, Type.BOOLEAN, true, diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 20c42cbabd325..542186bdc0bd4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -103,7 +103,8 @@ public class NetworkClientTest { BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, 10 * 1000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + false); protected final int defaultRequestTimeoutMs = 1000; protected final MockSelector selector = new MockSelector(time); @@ -169,7 +170,13 @@ connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new A private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata, boolean disableBootstrap) { if (disableBootstrap) { - bootstrapConfiguration.disableBootstrap(); + bootstrapConfiguration = + new NetworkClient.BootstrapConfiguration( + BOOTSTRAP_ADDRESSES, + ClientDnsLookup.USE_ALL_DNS_IPS, + 10 * 1000, + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + true); } return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, @@ -193,8 +200,8 @@ public void setup() { BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS - ); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + false); } @Test @@ -1532,8 +1539,8 @@ public void testEnsureBootstrappedSuccess() { BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, 5000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS - ); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + false); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, @@ -1557,8 +1564,8 @@ public void testEnsureBootstrappedTimeoutThrowsException() { invalidAddresses, ClientDnsLookup.USE_ALL_DNS_IPS, 100, // Short timeout - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS - ); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + false); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, @@ -1581,8 +1588,8 @@ public void testEnsureBootstrappedPollTimeoutReturnsWithoutError() { invalidAddresses, ClientDnsLookup.USE_ALL_DNS_IPS, 5000, // Long bootstrap timeout - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS - ); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + false); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, @@ -1605,8 +1612,8 @@ public void testEnsureBootstrappedRetryUntilSuccess() { BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, 5000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS - ); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, + false); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 814aee099c079..f110c2d14875e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -111,7 +111,7 @@ public WorkerGroupMember(DistributedConfig config, config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), false); NetworkClient netClient = new NetworkClient( new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, channelBuilder, logContext), metadata, diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index bc80d7cbe3f5e..7100019f08a1e 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -186,8 +186,8 @@ static AdminClient create(AbstractConfig config) { config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG) - ); + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), + false); NetworkClient networkClient = new NetworkClient( selector, metadata, diff --git a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java index 543bd3f34b8ad..4971f2d41dddb 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java @@ -677,7 +677,7 @@ private static class ReplicaFetcherBlockingSend { consumerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(consumerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), consumerConfig.getLong(ConsumerConfig.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), - consumerConfig.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); + consumerConfig.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), false); this.networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index ff915bcb838d9..77e0273bc54e4 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -170,8 +170,8 @@ public boolean tryConnect() { conf.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(conf.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - conf.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG) - ); + conf.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), + false); try (NetworkClient client = new NetworkClient(selector, updater, "ConnectionStressWorker", From 6f414e981f1c265a10a0f9c14585b948c616d328 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 18 Mar 2026 20:07:51 +0000 Subject: [PATCH 19/30] fix failed tests --- .../kafka/clients/consumer/KafkaConsumerTest.java | 4 +++- .../integration/ConnectWorkerIntegrationTest.java | 10 ++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 9d1011601ab05..ee3d45f649f4b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -536,8 +536,10 @@ public void testConstructorClose(GroupProtocol groupProtocol) { Properties props = new Properties(); props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testConstructorClose"); - props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "invalid-23-8409-adsfsdj"); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + // Use an invalid interceptor class to trigger constructor failure + props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "invalid.interceptor.class"); final int oldInitCount = MockMetricsReporter.INIT_COUNT.get(); final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index d46d76c3606ef..a80d495c2c016 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -72,7 +72,6 @@ import jakarta.ws.rs.core.Response; import static jakarta.ws.rs.core.Response.Status.INTERNAL_SERVER_ERROR; -import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG; import static org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CONFIG; import static org.apache.kafka.common.config.TopicConfig.DELETE_RETENTION_MS_CONFIG; @@ -80,7 +79,6 @@ import static org.apache.kafka.connect.integration.BlockingConnectorTest.TASK_STOP; import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; -import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX; import static org.apache.kafka.connect.runtime.ConnectorConfig.HEADER_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.PREDICATES_CONFIG; @@ -211,9 +209,9 @@ public void testRestartFailedTask() throws Exception { // setup up props for the source connector Map props = defaultSourceConnectorProps(TOPIC_NAME); - // Properties for the source connector. The task should fail at startup due to the bad broker address. + // Properties for the source connector. The task should fail at startup due to injected error. props.put(TASKS_MAX_CONFIG, Objects.toString(numTasks)); - props.put(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG, "nobrokerrunningatthisaddress"); + props.put("task-" + CONNECTOR_NAME + "-0.start.inject.error", "true"); // Try to start the connector and its single task. connect.configureConnector(CONNECTOR_NAME, props); @@ -221,8 +219,8 @@ public void testRestartFailedTask() throws Exception { connect.assertions().assertConnectorIsRunningAndTasksHaveFailed(CONNECTOR_NAME, numTasks, "Connector tasks did not fail in time"); - // Reconfigure the connector without the bad broker address. - props.remove(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG); + // Reconfigure the connector without the injected error. + props.remove("task-" + CONNECTOR_NAME + "-0.start.inject.error"); connect.configureConnector(CONNECTOR_NAME, props); // Restart the failed task From 2550e57fbc20058c1cf19829408b9ea8a7d7b153 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 18 Mar 2026 22:16:33 +0000 Subject: [PATCH 20/30] address comments --- .../org/apache/kafka/clients/ClientUtils.java | 5 +-- .../apache/kafka/clients/NetworkClient.java | 17 ++++++--- .../kafka/clients/admin/KafkaAdminClient.java | 23 ++++++++--- .../internals/AdminBootstrapAddresses.java | 10 ++--- .../admin/internals/AdminMetadataManager.java | 2 +- .../kafka/clients/NetworkClientTest.java | 38 +++++++------------ .../clients/admin/KafkaAdminClientTest.java | 5 +-- .../distributed/WorkerGroupMember.java | 6 +-- .../kafka/tools/BrokerApiVersionsCommand.java | 7 ++-- .../kafka/tools/ReplicaVerificationTool.java | 4 +- .../workload/ConnectionStressWorker.java | 7 ++-- 11 files changed, 61 insertions(+), 63 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 3195a237bdfed..f12cf674fdb2c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -283,12 +283,11 @@ public static NetworkClient createNetworkClient(AbstractConfig config, logContext); ClientDnsLookup dnsLookup = ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)); - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration bootstrapConfiguration = NetworkClient.BootstrapConfiguration.enabled( bootstrapServers != null ? bootstrapServers : List.of(), dnsLookup, config.getLong(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), - config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), - false); + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); return new NetworkClient(metadataUpdater, metadata, diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 074cb0ba3085d..be874aba3788d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1204,11 +1204,11 @@ public static class BootstrapConfiguration { public final long retryBackoffMs; private final boolean isBootstrapDisabled; - public BootstrapConfiguration(final List bootstrapServers, - final ClientDnsLookup clientDnsLookup, - final long bootstrapResolveTimeoutMs, - final long retryBackoffMs, - final boolean isBootstrapDisabled) { + private BootstrapConfiguration(final List bootstrapServers, + final ClientDnsLookup clientDnsLookup, + final long bootstrapResolveTimeoutMs, + final long retryBackoffMs, + final boolean isBootstrapDisabled) { this.bootstrapServers = bootstrapServers; this.clientDnsLookup = clientDnsLookup; this.bootstrapResolveTimeoutMs = bootstrapResolveTimeoutMs; @@ -1216,6 +1216,13 @@ public BootstrapConfiguration(final List bootstrapServers, this.isBootstrapDisabled = isBootstrapDisabled; } + public static BootstrapConfiguration enabled(final List bootstrapServers, + final ClientDnsLookup clientDnsLookup, + final long bootstrapResolveTimeoutMs, + final long retryBackoffMs) { + return new BootstrapConfiguration(bootstrapServers, clientDnsLookup, bootstrapResolveTimeoutMs, retryBackoffMs, false); + } + public static BootstrapConfiguration disabled() { return new BootstrapConfiguration(List.of(), null, 0, 0, true); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 7bf4586dec422..6c26cf87efb80 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -267,6 +267,7 @@ import org.slf4j.Logger; +import java.net.InetSocketAddress; import java.security.InvalidKeyException; import java.security.NoSuchAlgorithmException; import java.time.Duration; @@ -538,7 +539,22 @@ static KafkaAdminClient createInternal( config.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), config.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), adminAddresses.usingBootstrapControllers()); - metadataManager.update(Cluster.bootstrap(adminAddresses.addresses()), time.milliseconds()); + + // Get the appropriate bootstrap configuration + List bootstrapAddressesToUse = adminAddresses.usingBootstrapControllers() + ? config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) + : config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); + + // Create unresolved addresses for bootstrap cluster (defer DNS resolution to NetworkClient.poll()) + List unresolvedAddresses = new ArrayList<>(); + for (String address : bootstrapAddressesToUse) { + String host = Utils.getHost(address); + Integer port = Utils.getPort(address); + if (host != null && port != null) { + unresolvedAddresses.add(InetSocketAddress.createUnresolved(host, port)); + } + } + metadataManager.update(Cluster.bootstrap(unresolvedAddresses), time.milliseconds()); List reporters = CommonClientConfigs.metricsReporters(clientId, config); clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); clientTelemetryReporter.ifPresent(reporters::add); @@ -551,11 +567,6 @@ static KafkaAdminClient createInternal( config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)); metrics = new Metrics(metricConfig, reporters, time, metricsContext); - // Use the appropriate bootstrap configuration determined by AdminBootstrapAddresses - List bootstrapAddressesToUse = adminAddresses.usingBootstrapControllers() - ? config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) - : config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); - networkClient = ClientUtils.createNetworkClient(config, bootstrapAddressesToUse, clientId, diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java index 77a491a320180..113048fed52ad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.admin.internals; -import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.common.config.AbstractConfig; @@ -56,20 +55,19 @@ public static AdminBootstrapAddresses fromConfig(AbstractConfig config) { if (controllerServers == null) { controllerServers = Collections.emptyList(); } - String clientDnsLookupConfig = config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG); if (bootstrapServers.isEmpty()) { if (controllerServers.isEmpty()) { throw new ConfigException("You must set either " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " or " + AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); } else { - return new AdminBootstrapAddresses(true, - ClientUtils.parseAndValidateAddresses(controllerServers, clientDnsLookupConfig)); + // Don't perform DNS resolution here - defer to NetworkClient.poll() + return new AdminBootstrapAddresses(true, List.of()); } } else { if (controllerServers.isEmpty()) { - return new AdminBootstrapAddresses(false, - ClientUtils.parseAndValidateAddresses(bootstrapServers, clientDnsLookupConfig)); + // Don't perform DNS resolution here - defer to NetworkClient.poll() + return new AdminBootstrapAddresses(false, List.of()); } else { throw new ConfigException("You cannot set both " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " and " + diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java index f0de44a17b782..ed65b6aebcaf8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java @@ -158,7 +158,7 @@ public boolean isBootstrapped() { @Override public void bootstrap(List addresses) { - // AdminClient handles bootstrap during construction, so this method is not used + // Bootstrap is handled by NetworkClient.ensureBootstrapped() during poll } @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 542186bdc0bd4..57765ef5c1ff8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -99,12 +99,11 @@ public class NetworkClientTest { private static List initialAddresses; private static List newAddresses; private static NetworkClient.BootstrapConfiguration bootstrapConfiguration = - new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration.enabled( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, 10 * 1000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - false); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); protected final int defaultRequestTimeoutMs = 1000; protected final MockSelector selector = new MockSelector(time); @@ -170,13 +169,7 @@ connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, true, new A private NetworkClient createNetworkClientWithNoVersionDiscovery(Metadata metadata, boolean disableBootstrap) { if (disableBootstrap) { - bootstrapConfiguration = - new NetworkClient.BootstrapConfiguration( - BOOTSTRAP_ADDRESSES, - ClientDnsLookup.USE_ALL_DNS_IPS, - 10 * 1000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - true); + bootstrapConfiguration = NetworkClient.BootstrapConfiguration.disabled(); } return new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, @@ -196,12 +189,11 @@ connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, time, false, new @BeforeEach public void setup() { selector.reset(); - bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + bootstrapConfiguration = NetworkClient.BootstrapConfiguration.enabled( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - false); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); } @Test @@ -1535,12 +1527,11 @@ private void bootstrapMetadataUpdater(final MetadataUpdater metadataUpdater) { @Test public void testEnsureBootstrappedSuccess() { Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); - NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration config = NetworkClient.BootstrapConfiguration.enabled( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, 5000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - false); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, @@ -1560,12 +1551,11 @@ public void testEnsureBootstrappedTimeoutThrowsException() { Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); // Use invalid addresses that cannot be resolved List invalidAddresses = List.of("invalid.host.that.does.not.exist:9092"); - NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration config = NetworkClient.BootstrapConfiguration.enabled( invalidAddresses, ClientDnsLookup.USE_ALL_DNS_IPS, 100, // Short timeout - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - false); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, @@ -1584,12 +1574,11 @@ public void testEnsureBootstrappedPollTimeoutReturnsWithoutError() { Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); // Use invalid addresses that cannot be resolved List invalidAddresses = List.of("invalid.host.that.does.not.exist:9092"); - NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration config = NetworkClient.BootstrapConfiguration.enabled( invalidAddresses, ClientDnsLookup.USE_ALL_DNS_IPS, 5000, // Long bootstrap timeout - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - false); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, @@ -1608,12 +1597,11 @@ time, false, new ApiVersions(), new LogContext(), @Test public void testEnsureBootstrappedRetryUntilSuccess() { Metadata metadata = new Metadata(50, 50, 5000, new LogContext(), new ClusterResourceListeners()); - NetworkClient.BootstrapConfiguration config = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration config = NetworkClient.BootstrapConfiguration.enabled( BOOTSTRAP_ADDRESSES, ClientDnsLookup.USE_ALL_DNS_IPS, 5000, - CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS, - false); + CommonClientConfigs.DEFAULT_RETRY_BACKOFF_MS); NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, reconnectBackoffMsTest, 0, 64 * 1024, 64 * 1024, defaultRequestTimeoutMs, connectionSetupTimeoutMsTest, connectionSetupTimeoutMaxMsTest, diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 1bb7e2e171e24..d4d859192678b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -16,9 +16,7 @@ */ package org.apache.kafka.clients.admin; -import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientRequest; -import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.MetadataRecoveryStrategy; import org.apache.kafka.clients.MockClient; @@ -529,8 +527,7 @@ private static Cluster mockCluster(int numNodes, int controllerIndex) { } private static Cluster mockBootstrapCluster() { - return Cluster.bootstrap(ClientUtils.parseAndValidateAddresses( - singletonList("localhost:8121"), ClientDnsLookup.USE_ALL_DNS_IPS)); + return Cluster.bootstrap(singletonList(InetSocketAddress.createUnresolved("localhost", 8121))); } private static AdminClientUnitTestEnv mockClientEnv(String... configVals) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index f110c2d14875e..e37b34bf12b64 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -107,11 +107,11 @@ public WorkerGroupMember(DistributedConfig config, metadata.bootstrap(addresses); String metricGrpPrefix = "connect"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config, time, logContext); - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration bootstrapConfiguration = NetworkClient.BootstrapConfiguration.enabled( config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), false); + config.getLong(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); NetworkClient netClient = new NetworkClient( new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, channelBuilder, logContext), metadata, diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index 7100019f08a1e..4166dbff12079 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -182,12 +182,11 @@ static AdminClient create(AbstractConfig config) { "admin", ClientUtils.createChannelBuilder(config, time, logContext), logContext); - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration bootstrapConfiguration = NetworkClient.BootstrapConfiguration.enabled( config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), - false); + config.getLong(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), + config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); NetworkClient networkClient = new NetworkClient( selector, metadata, diff --git a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java index 4971f2d41dddb..451582a0852f6 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java @@ -673,11 +673,11 @@ private static class ReplicaFetcherBlockingSend { channelBuilder, logContext ); - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration bootstrapConfiguration = NetworkClient.BootstrapConfiguration.enabled( consumerConfig.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(consumerConfig.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), consumerConfig.getLong(ConsumerConfig.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), - consumerConfig.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), false); + consumerConfig.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); this.networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index 77e0273bc54e4..e0392e76c68ef 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -166,12 +166,11 @@ public boolean tryConnect() { try (Metrics metrics = new Metrics()) { try (Selector selector = new Selector(conf.getLong(AdminClientConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, Time.SYSTEM, "", channelBuilder, logContext)) { - NetworkClient.BootstrapConfiguration bootstrapConfiguration = new NetworkClient.BootstrapConfiguration( + NetworkClient.BootstrapConfiguration bootstrapConfiguration = NetworkClient.BootstrapConfiguration.enabled( conf.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG), ClientDnsLookup.forConfig(conf.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG)), - CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, - conf.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG), - false); + conf.getLong(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG), + conf.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG)); try (NetworkClient client = new NetworkClient(selector, updater, "ConnectionStressWorker", From 4566ef138368707a112a2d429accbbe7baf4ef99 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Thu, 19 Mar 2026 07:19:07 +0000 Subject: [PATCH 21/30] fix failed tests --- .../admin/internals/AdminBootstrapAddressesTest.java | 9 ++------- .../connect/runtime/distributed/DistributedConfig.java | 6 ++++++ .../org/apache/kafka/tools/BrokerApiVersionsCommand.java | 1 + 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java index 0581d672fb8a0..16b87553766eb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java @@ -23,8 +23,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.net.InetSocketAddress; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -69,10 +67,7 @@ public void testFromConfig(boolean usingBootstrapControllers) { AdminClientConfig config = new AdminClientConfig(map); AdminBootstrapAddresses addresses = AdminBootstrapAddresses.fromConfig(config); assertEquals(usingBootstrapControllers, addresses.usingBootstrapControllers()); - assertEquals(Arrays.asList( - new InetSocketAddress("localhost", 9092), - new InetSocketAddress("localhost", 9093), - new InetSocketAddress("localhost", 9094)), - addresses.addresses()); + assertEquals(0, addresses.addresses().size(), + "DNS resolution is deferred to NetworkClient.poll(), so addresses list should be empty"); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index ccf33926bf93f..066d98c1af55f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -394,6 +394,12 @@ private static ConfigDef config(Crypto crypto) { atLeast(0L), ConfigDef.Importance.LOW, CommonClientConfigs.RETRY_BACKOFF_MAX_MS_DOC) + .define(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, + ConfigDef.Type.LONG, + CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, + atLeast(0L), + ConfigDef.Importance.MEDIUM, + CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC) .define(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG, ConfigDef.Type.INT, Math.toIntExact(TimeUnit.SECONDS.toMillis(40)), diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index 4166dbff12079..651d77394f718 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -151,6 +151,7 @@ protected static class AdminClient implements AutoCloseable { .define(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, ConfigDef.Type.LONG, CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS, ConfigDef.Importance.MEDIUM, CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC) .define(CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, ConfigDef.Type.LONG, CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, ConfigDef.Importance.MEDIUM, CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC) .define(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG, ConfigDef.Type.LONG, DEFAULT_RETRY_BACKOFF_MS, ConfigDef.Importance.MEDIUM, CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, ConfigDef.Type.LONG, CommonClientConfigs.DEFAULT_BOOTSTRAP_RESOLVE_TIMEOUT_MS, ConfigDef.Importance.MEDIUM, CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_DOC) .withClientSslSupport() .withClientSaslSupport(); From e1b31b2e4bd74a0e167c716638bbda839e1ebbb9 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Thu, 19 Mar 2026 16:06:35 +0000 Subject: [PATCH 22/30] address comments --- .../org/apache/kafka/clients/NetworkClient.java | 5 +++++ .../kafka/clients/admin/KafkaAdminClient.java | 13 ++----------- .../admin/internals/AdminMetadataManager.java | 5 ++--- 3 files changed, 9 insertions(+), 14 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index be874aba3788d..1fcce97824614 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1268,6 +1268,11 @@ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { pollDeadlineMs = currentTimeMs + pollTimeoutMs; while (true) { + // Check if client is still active (user may have called close()) + if (!active()) { + return; + } + long now = time.milliseconds(); bootstrapTimer.update(now); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 6c26cf87efb80..d543ffb9a860b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -267,7 +267,6 @@ import org.slf4j.Logger; -import java.net.InetSocketAddress; import java.security.InvalidKeyException; import java.security.NoSuchAlgorithmException; import java.time.Duration; @@ -545,16 +544,8 @@ static KafkaAdminClient createInternal( ? config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) : config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); - // Create unresolved addresses for bootstrap cluster (defer DNS resolution to NetworkClient.poll()) - List unresolvedAddresses = new ArrayList<>(); - for (String address : bootstrapAddressesToUse) { - String host = Utils.getHost(address); - Integer port = Utils.getPort(address); - if (host != null && port != null) { - unresolvedAddresses.add(InetSocketAddress.createUnresolved(host, port)); - } - } - metadataManager.update(Cluster.bootstrap(unresolvedAddresses), time.milliseconds()); + // Don't create bootstrap cluster here - let NetworkClient.ensureBootstrapped() handle it + // during the first poll after DNS resolution succeeds List reporters = CommonClientConfigs.metricsReporters(clientId, config); clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); clientTelemetryReporter.ifPresent(reporters::add); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java index ed65b6aebcaf8..764aa440f81c6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java @@ -151,14 +151,13 @@ public void rebootstrap(long now) { @Override public boolean isBootstrapped() { - // AdminClient bootstraps during construction via metadataManager.update(Cluster.bootstrap(...)) - // so we check if the bootstrap cluster has been set return bootstrapCluster != null; } @Override public void bootstrap(List addresses) { - // Bootstrap is handled by NetworkClient.ensureBootstrapped() during poll + // Called by NetworkClient.ensureBootstrapped() after DNS resolution succeeds + AdminMetadataManager.this.update(Cluster.bootstrap(addresses), 0); } @Override From bfdd24cce52bc3618a80fce0485e74c028822978 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Thu, 19 Mar 2026 16:19:49 +0000 Subject: [PATCH 23/30] address comments --- .../main/java/org/apache/kafka/clients/NetworkClient.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 1fcce97824614..52974782416db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1268,8 +1268,9 @@ void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { pollDeadlineMs = currentTimeMs + pollTimeoutMs; while (true) { - // Check if client is still active (user may have called close()) - if (!active()) { + // Check if thread has been interrupted + if (Thread.interrupted()) { + log.debug("ensureBootstrapped interrupted, returning early"); return; } From e3159c8c5b1a1355c1c90bcc9f1fe799e1f3c9b8 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 25 Mar 2026 13:18:52 -0500 Subject: [PATCH 24/30] address comments --- .../apache/kafka/clients/NetworkClient.java | 106 +++++------------- .../kafka/clients/admin/KafkaAdminClient.java | 44 +++++++- .../internals/AdminBootstrapAddresses.java | 106 ------------------ .../kafka/clients/NetworkClientTest.java | 6 +- .../AdminBootstrapAddressesTest.java | 73 ------------ 5 files changed, 71 insertions(+), 264 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 52974782416db..4395941dee61d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -143,8 +143,6 @@ private enum State { private final Timer bootstrapTimer; - private boolean bootstrapTimerStarted; - private final TelemetrySender telemetrySender; public NetworkClient(Selectable selector, @@ -365,7 +363,6 @@ public NetworkClient(MetadataUpdater metadataUpdater, this.metadataRecoveryStrategy = metadataRecoveryStrategy; this.bootstrapConfiguration = bootstrapConfiguration; this.bootstrapTimer = time.timer(bootstrapConfiguration.bootstrapResolveTimeoutMs); - this.bootstrapTimerStarted = false; } /** @@ -645,16 +642,11 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long * metadata timeout * @param now The current time in milliseconds * @return The list of responses received - * - * @implNote During bootstrap, this method may perform blocking DNS lookups to resolve bootstrap server addresses. - * If DNS resolution is slow or fails, this method may block for longer than the specified timeout. - * The blocking behavior will continue until either: (1) DNS resolution succeeds, (2) the bootstrap - * timeout expires (throwing BootstrapResolutionException), or (3) the poll timeout is reached. */ @Override public List poll(long timeout, long now) { ensureActive(); - ensureBootstrapped(timeout, now); + ensureBootstrapped(now); if (!abortedSends.isEmpty()) { // If there are aborted sends because of unsupported version exceptions or disconnects, @@ -1229,84 +1221,42 @@ public static BootstrapConfiguration disabled() { } /** - * Ensures that the client has successfully resolved and bootstrapped with the configured bootstrap servers. - * This method will retry DNS resolution until one of the following conditions is met: - *

    - *
  • DNS resolution succeeds and bootstrap is complete (method returns normally)
  • - *
  • Bootstrap timeout expires (throws BootstrapResolutionException)
  • - *
  • Poll timeout is reached but bootstrap timeout hasn't expired (method returns to retry later)
  • - *
+ * Attempts to resolve bootstrap server addresses via DNS and create an initial bootstrap cluster. + * This method is called from {@link #poll(long, long)} and uses a non-blocking approach. * - * @param pollTimeoutMs The poll timeout in milliseconds, controlling how long this method should block - * during a single poll invocation - * @param currentTimeMs The current time in milliseconds - * @throws BootstrapResolutionException if the bootstrap timeout expires - * before DNS resolution succeeds + *

On each invocation, this method will attempt DNS resolution once. If resolution fails, + * it returns immediately and will be retried on the next poll invocation. This ensures the + * event loop remains responsive and doesn't block on DNS lookups. * - * @implNote This method performs blocking DNS lookups via {@link InetAddress#getAllByName(String)}. - * Each DNS lookup may block for an indefinite amount of time depending on network conditions - * and DNS server responsiveness. As a result, this method may block for longer than pollTimeoutMs - * if DNS resolution is slow. The method will retry DNS resolution in a loop with brief sleeps - * (up to 100ms) between attempts until one of the exit conditions above is met. + * @param currentTimeMs The current time in milliseconds + * @throws BootstrapResolutionException if the bootstrap timeout expires before DNS resolution succeeds */ - void ensureBootstrapped(final long pollTimeoutMs, final long currentTimeMs) { + void ensureBootstrapped(final long currentTimeMs) { if (bootstrapConfiguration.isBootstrapDisabled || metadataUpdater.isBootstrapped()) return; - // Start the bootstrap timer on first call to ensure it starts counting from the first poll - if (!bootstrapTimerStarted) { - bootstrapTimer.update(currentTimeMs); - bootstrapTimer.reset(bootstrapConfiguration.bootstrapResolveTimeoutMs); - bootstrapTimerStarted = true; - } - - // Handle potential overflow when adding timeout to current time - long pollDeadlineMs; - if (currentTimeMs > Long.MAX_VALUE - pollTimeoutMs) - pollDeadlineMs = Long.MAX_VALUE; - else - pollDeadlineMs = currentTimeMs + pollTimeoutMs; - - while (true) { - // Check if thread has been interrupted - if (Thread.interrupted()) { - log.debug("ensureBootstrapped interrupted, returning early"); - return; - } - - long now = time.milliseconds(); - bootstrapTimer.update(now); - - List servers = ClientUtils.parseAddresses( - bootstrapConfiguration.bootstrapServers, bootstrapConfiguration.clientDnsLookup); - - if (!servers.isEmpty()) { - // Resolution succeeded - bootstrapTimer.reset(bootstrapConfiguration.bootstrapResolveTimeoutMs); - metadataUpdater.bootstrap(servers); - return; - } - - if (bootstrapTimer.isExpired()) { - // Bootstrap timeout expired before poll timeout - throw new BootstrapResolutionException("Timeout while attempting to resolve bootstrap servers."); - } + // Timer is already initialized in constructor, just update it with current time + bootstrapTimer.update(currentTimeMs); - if (now >= pollDeadlineMs) { - // Poll timeout reached but bootstrap timeout hasn't expired yet - return; - } + // Attempt DNS resolution (single attempt per poll, typically fast) + List servers = ClientUtils.parseAddresses(bootstrapConfiguration.bootstrapServers, bootstrapConfiguration.clientDnsLookup); - // Sleep before retrying to avoid tight loop and reduce load on DNS server - // Use the standard retry backoff to prevent overloading DNS with requests - long remainingPollTimeMs = pollDeadlineMs - now; - long remainingBootstrapTimeMs = bootstrapTimer.remainingMs(); - long sleepTimeMs = Math.min(Math.min(remainingPollTimeMs, remainingBootstrapTimeMs), bootstrapConfiguration.retryBackoffMs); + if (!servers.isEmpty()) { + log.debug("Bootstrap DNS resolution succeeded, {} servers resolved", servers.size()); + metadataUpdater.bootstrap(servers); + return; + } - if (sleepTimeMs > 0) { - time.sleep(sleepTimeMs); - } + // DNS resolution failed + if (bootstrapTimer.isExpired()) { + throw new BootstrapResolutionException("Failed to resolve bootstrap servers after " + + bootstrapConfiguration.bootstrapResolveTimeoutMs + "ms. " + + "Please check your bootstrap.servers configuration and DNS settings."); } + + // DNS failed but timeout not reached, log and will retry on next poll + log.warn("Failed to resolve bootstrap servers, will retry on next poll. Remaining time: {}ms", + bootstrapTimer.remainingMs()); } class DefaultMetadataUpdater implements MetadataUpdater { @@ -1333,7 +1283,7 @@ class DefaultMetadataUpdater implements MetadataUpdater { @Override public List fetchNodes() { - ensureBootstrapped(Long.MAX_VALUE, time.milliseconds()); + ensureBootstrapped(time.milliseconds()); return metadata.fetch().nodes(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 1c8f6060b5523..88a84a54edae6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -40,7 +40,6 @@ import org.apache.kafka.clients.admin.internals.AdminApiFuture; import org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture; import org.apache.kafka.clients.admin.internals.AdminApiHandler; -import org.apache.kafka.clients.admin.internals.AdminBootstrapAddresses; import org.apache.kafka.clients.admin.internals.AdminFetchMetricsManager; import org.apache.kafka.clients.admin.internals.AdminMetadataManager; import org.apache.kafka.clients.admin.internals.AllBrokersStrategy; @@ -513,6 +512,43 @@ static String prettyPrintException(Throwable throwable) { return throwable.getClass().getSimpleName(); } + /** + * Determines which bootstrap configuration to use based on the provided config. + * Validates that exactly one of bootstrap.servers or bootstrap.controllers is configured. + * + * @param config The admin client configuration + * @return true if using bootstrap.controllers, false if using bootstrap.servers + * @throws ConfigException if both or neither bootstrap configurations are set + */ + static boolean determineBootstrapType(AdminClientConfig config) { + List bootstrapServers = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); + if (bootstrapServers == null) { + bootstrapServers = Collections.emptyList(); + } + List controllerServers = config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); + if (controllerServers == null) { + controllerServers = Collections.emptyList(); + } + + if (bootstrapServers.isEmpty()) { + if (controllerServers.isEmpty()) { + throw new ConfigException("You must set either " + + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " or " + + AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); + } else { + return true; // Using bootstrap.controllers + } + } else { + if (controllerServers.isEmpty()) { + return false; // Using bootstrap.servers + } else { + throw new ConfigException("You cannot set both " + + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " and " + + AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); + } + } + } + static KafkaAdminClient createInternal(AdminClientConfig config, TimeoutProcessorFactory timeoutProcessorFactory) { return createInternal(config, timeoutProcessorFactory, null); } @@ -533,14 +569,14 @@ static KafkaAdminClient createInternal( try { // Since we only request node information, it's safe to pass true for allowAutoTopicCreation (and it // simplifies communication with older brokers) - AdminBootstrapAddresses adminAddresses = AdminBootstrapAddresses.fromConfig(config); + boolean usingBootstrapControllers = determineBootstrapType(config); AdminMetadataManager metadataManager = new AdminMetadataManager(logContext, config.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), config.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), - adminAddresses.usingBootstrapControllers()); + usingBootstrapControllers); // Get the appropriate bootstrap configuration - List bootstrapAddressesToUse = adminAddresses.usingBootstrapControllers() + List bootstrapAddressesToUse = usingBootstrapControllers ? config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) : config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java deleted file mode 100644 index 113048fed52ad..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddresses.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.kafka.clients.admin.internals; - -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.common.config.AbstractConfig; -import org.apache.kafka.common.config.ConfigException; - -import java.net.InetSocketAddress; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -public final class AdminBootstrapAddresses { - private final boolean usingBootstrapControllers; - private final List addresses; - - AdminBootstrapAddresses( - boolean usingBootstrapControllers, - List addresses - ) { - this.usingBootstrapControllers = usingBootstrapControllers; - this.addresses = addresses; - } - - public boolean usingBootstrapControllers() { - return usingBootstrapControllers; - } - - public List addresses() { - return addresses; - } - - public static AdminBootstrapAddresses fromConfig(AbstractConfig config) { - List bootstrapServers = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); - if (bootstrapServers == null) { - bootstrapServers = Collections.emptyList(); - } - List controllerServers = config.getList(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); - if (controllerServers == null) { - controllerServers = Collections.emptyList(); - } - if (bootstrapServers.isEmpty()) { - if (controllerServers.isEmpty()) { - throw new ConfigException("You must set either " + - CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " or " + - AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); - } else { - // Don't perform DNS resolution here - defer to NetworkClient.poll() - return new AdminBootstrapAddresses(true, List.of()); - } - } else { - if (controllerServers.isEmpty()) { - // Don't perform DNS resolution here - defer to NetworkClient.poll() - return new AdminBootstrapAddresses(false, List.of()); - } else { - throw new ConfigException("You cannot set both " + - CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + " and " + - AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG); - } - } - } - - @Override - public int hashCode() { - return Objects.hash(usingBootstrapControllers, addresses); - } - - @Override - public boolean equals(Object o) { - if (o == null || (!o.getClass().equals(AdminBootstrapAddresses.class))) return false; - AdminBootstrapAddresses other = (AdminBootstrapAddresses) o; - return usingBootstrapControllers == other.usingBootstrapControllers && - addresses.equals(other.addresses); - } - - @Override - public String toString() { - StringBuilder bld = new StringBuilder(); - bld.append("AdminBootstrapAddresses"); - bld.append("(usingBootstrapControllers=").append(usingBootstrapControllers); - bld.append(", addresses=["); - String prefix = ""; - for (InetSocketAddress address : addresses) { - bld.append(prefix).append(address); - prefix = ", "; - } - bld.append("])"); - return bld.toString(); - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 57765ef5c1ff8..11c51fd0f87b1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -1585,9 +1585,9 @@ public void testEnsureBootstrappedPollTimeoutReturnsWithoutError() { time, false, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, config); - // Directly call ensureBootstrapped with short poll timeout (100ms) - // Should return without error even though bootstrap hasn't succeeded - client.ensureBootstrapped(100, time.milliseconds()); + // Directly call ensureBootstrapped + // Should return without error even though bootstrap hasn't succeeded (will retry on next poll) + client.ensureBootstrapped(time.milliseconds()); // Verify bootstrap has not succeeded yet MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java deleted file mode 100644 index 16b87553766eb..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.kafka.clients.admin.internals; - -import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.common.config.ConfigException; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -import java.util.HashMap; -import java.util.Map; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class AdminBootstrapAddressesTest { - - @Test - public void testNoBootstrapSet() { - Map map = Map.of( - AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "", - AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, "" - ); - AdminClientConfig config = new AdminClientConfig(map); - assertEquals("You must set either bootstrap.servers or bootstrap.controllers", - assertThrows(ConfigException.class, () -> AdminBootstrapAddresses.fromConfig(config)). - getMessage()); - } - - @Test - public void testTwoBootstrapsSet() { - Map map = new HashMap<>(); - map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, "localhost:9092"); - map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - AdminClientConfig config = new AdminClientConfig(map); - assertEquals("You cannot set both bootstrap.servers and bootstrap.controllers", - assertThrows(ConfigException.class, () -> AdminBootstrapAddresses.fromConfig(config)). - getMessage()); - } - - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testFromConfig(boolean usingBootstrapControllers) { - Map map = new HashMap<>(); - String connectString = "localhost:9092,localhost:9093,localhost:9094"; - if (usingBootstrapControllers) { - map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, connectString); - } else { - map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, connectString); - } - AdminClientConfig config = new AdminClientConfig(map); - AdminBootstrapAddresses addresses = AdminBootstrapAddresses.fromConfig(config); - assertEquals(usingBootstrapControllers, addresses.usingBootstrapControllers()); - assertEquals(0, addresses.addresses().size(), - "DNS resolution is deferred to NetworkClient.poll(), so addresses list should be empty"); - } -} From 4e745ab2caed7026506d2da4a793295180d321d1 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 25 Mar 2026 13:55:06 -0500 Subject: [PATCH 25/30] address comments --- .../org/apache/kafka/clients/Metadata.java | 8 ++---- .../kafka/clients/MetadataSnapshot.java | 17 ++++------- .../java/org/apache/kafka/common/Cluster.java | 28 ------------------- .../internals/RecordAccumulatorTest.java | 1 - 4 files changed, 8 insertions(+), 46 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index ddc45a53e0b43..2686f042c2ba8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -545,13 +545,9 @@ else if (metadata.error() == Errors.TOPIC_AUTHORIZATION_FAILED) return this.metadataSnapshot.mergeWith(metadataResponse.clusterId(), nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, (topic, isInternal) -> !topics.contains(topic) && retainTopic(topic, isInternal, nowMs)); - else { - // Preserve the bootstrap flag from the current snapshot when creating a new one - boolean isBootstrapConfigured = this.metadataSnapshot.cluster().isBootstrapConfigured(); + else return new MetadataSnapshot(metadataResponse.clusterId(), nodes, partitions, - unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, - isBootstrapConfigured, null); - } + unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java index 551b8b1391ebf..aa0caaef239c5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java @@ -54,7 +54,6 @@ public class MetadataSnapshot { private final Map metadataByPartition; private final Map topicIds; private final Map topicNames; - private final boolean isBootstrapConfigured; private Cluster clusterInstance; public MetadataSnapshot(String clusterId, @@ -65,7 +64,7 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds) { - this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, false, null); + this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, null); } // Visible for testing @@ -77,7 +76,6 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds, - boolean isBootstrapConfigured, Cluster clusterInstance) { this.clusterId = clusterId; this.nodes = Collections.unmodifiableMap(nodes); @@ -89,7 +87,6 @@ public MetadataSnapshot(String clusterId, this.topicNames = Collections.unmodifiableMap( topicIds.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)) ); - this.isBootstrapConfigured = isBootstrapConfigured; Map tmpMetadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { @@ -202,9 +199,8 @@ MetadataSnapshot mergeWith(String newClusterId, Set newInvalidTopics = fillSet(addInvalidTopics, invalidTopics, shouldRetainTopic); Set newInternalTopics = fillSet(addInternalTopics, internalTopics, shouldRetainTopic); - // Preserve the bootstrap flag from the current snapshot during merge return new MetadataSnapshot(newClusterId, newNodes, newMetadataByPartition.values(), newUnauthorizedTopics, - newInvalidTopics, newInternalTopics, newController, newTopicIds, this.isBootstrapConfigured, null); + newInvalidTopics, newInternalTopics, newController, newTopicIds); } /** @@ -231,9 +227,8 @@ private void computeClusterView() { .stream() .map(metadata -> MetadataResponse.toPartitionInfo(metadata, nodes)) .collect(Collectors.toList()); - // Use the factory method that preserves the bootstrap state - this.clusterInstance = Cluster.withBootstrapFlag(clusterId, nodes.values(), partitionInfos, - unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, isBootstrapConfigured); + this.clusterInstance = new Cluster(clusterId, nodes.values(), partitionInfos, + unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); } static MetadataSnapshot bootstrap(List addresses) { @@ -245,12 +240,12 @@ static MetadataSnapshot bootstrap(List addresses) { } return new MetadataSnapshot(null, nodes, Collections.emptyList(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), - null, Collections.emptyMap(), true, Cluster.bootstrap(addresses)); + null, Collections.emptyMap(), Cluster.bootstrap(addresses)); } static MetadataSnapshot empty() { return new MetadataSnapshot(null, Collections.emptyMap(), Collections.emptyList(), - Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), false, Cluster.empty()); + Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), Cluster.empty()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 9c7acc7791931..93f2f4225bc74 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -217,34 +217,6 @@ public static Cluster bootstrap(List addresses) { Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap()); } - /** - * Create a cluster instance preserving the bootstrap state. - * This is used internally by MetadataSnapshot to maintain bootstrap flag across metadata updates. - * - * @param clusterId the cluster id - * @param nodes the nodes in the cluster - * @param partitions information about partitions - * @param unauthorizedTopics unauthorized topics - * @param invalidTopics invalid topics - * @param internalTopics internal topics - * @param controller the controller node - * @param topicIds topic IDs - * @param isBootstrapConfigured whether the cluster is bootstrapped - * @return a new Cluster instance with the specified bootstrap state - */ - public static Cluster withBootstrapFlag(String clusterId, - Collection nodes, - Collection partitions, - Set unauthorizedTopics, - Set invalidTopics, - Set internalTopics, - Node controller, - Map topicIds, - boolean isBootstrapConfigured) { - return new Cluster(clusterId, isBootstrapConfigured, nodes, partitions, - unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); - } - /** * Return a copy of this cluster combined with `partitions`. */ diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index bdd15fcb6710b..78beb2557ebf4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -161,7 +161,6 @@ public void testDrainBatches() throws Exception { Collections.emptySet(), null, Collections.emptyMap(), - false, cluster); long batchSize = value.length + DefaultRecordBatch.RECORD_BATCH_OVERHEAD; RecordAccumulator accum = createTestRecordAccumulator((int) batchSize, Integer.MAX_VALUE, Compression.NONE, 10); From 0621ede183f0156ae17616808a36f6d15c5931ec Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 27 Mar 2026 10:02:29 +0000 Subject: [PATCH 26/30] address comments --- .../org/apache/kafka/clients/Metadata.java | 8 ++++-- .../kafka/clients/MetadataSnapshot.java | 17 +++++++---- .../java/org/apache/kafka/common/Cluster.java | 28 +++++++++++++++++++ .../internals/RecordAccumulatorTest.java | 1 + 4 files changed, 46 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 2686f042c2ba8..ddc45a53e0b43 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -545,9 +545,13 @@ else if (metadata.error() == Errors.TOPIC_AUTHORIZATION_FAILED) return this.metadataSnapshot.mergeWith(metadataResponse.clusterId(), nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, (topic, isInternal) -> !topics.contains(topic) && retainTopic(topic, isInternal, nowMs)); - else + else { + // Preserve the bootstrap flag from the current snapshot when creating a new one + boolean isBootstrapConfigured = this.metadataSnapshot.cluster().isBootstrapConfigured(); return new MetadataSnapshot(metadataResponse.clusterId(), nodes, partitions, - unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds); + unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, + isBootstrapConfigured, null); + } } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java index aa0caaef239c5..551b8b1391ebf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java @@ -54,6 +54,7 @@ public class MetadataSnapshot { private final Map metadataByPartition; private final Map topicIds; private final Map topicNames; + private final boolean isBootstrapConfigured; private Cluster clusterInstance; public MetadataSnapshot(String clusterId, @@ -64,7 +65,7 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds) { - this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, null); + this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, false, null); } // Visible for testing @@ -76,6 +77,7 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds, + boolean isBootstrapConfigured, Cluster clusterInstance) { this.clusterId = clusterId; this.nodes = Collections.unmodifiableMap(nodes); @@ -87,6 +89,7 @@ public MetadataSnapshot(String clusterId, this.topicNames = Collections.unmodifiableMap( topicIds.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)) ); + this.isBootstrapConfigured = isBootstrapConfigured; Map tmpMetadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { @@ -199,8 +202,9 @@ MetadataSnapshot mergeWith(String newClusterId, Set newInvalidTopics = fillSet(addInvalidTopics, invalidTopics, shouldRetainTopic); Set newInternalTopics = fillSet(addInternalTopics, internalTopics, shouldRetainTopic); + // Preserve the bootstrap flag from the current snapshot during merge return new MetadataSnapshot(newClusterId, newNodes, newMetadataByPartition.values(), newUnauthorizedTopics, - newInvalidTopics, newInternalTopics, newController, newTopicIds); + newInvalidTopics, newInternalTopics, newController, newTopicIds, this.isBootstrapConfigured, null); } /** @@ -227,8 +231,9 @@ private void computeClusterView() { .stream() .map(metadata -> MetadataResponse.toPartitionInfo(metadata, nodes)) .collect(Collectors.toList()); - this.clusterInstance = new Cluster(clusterId, nodes.values(), partitionInfos, - unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); + // Use the factory method that preserves the bootstrap state + this.clusterInstance = Cluster.withBootstrapFlag(clusterId, nodes.values(), partitionInfos, + unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, isBootstrapConfigured); } static MetadataSnapshot bootstrap(List addresses) { @@ -240,12 +245,12 @@ static MetadataSnapshot bootstrap(List addresses) { } return new MetadataSnapshot(null, nodes, Collections.emptyList(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), - null, Collections.emptyMap(), Cluster.bootstrap(addresses)); + null, Collections.emptyMap(), true, Cluster.bootstrap(addresses)); } static MetadataSnapshot empty() { return new MetadataSnapshot(null, Collections.emptyMap(), Collections.emptyList(), - Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), Cluster.empty()); + Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), false, Cluster.empty()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 93f2f4225bc74..9c7acc7791931 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -217,6 +217,34 @@ public static Cluster bootstrap(List addresses) { Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap()); } + /** + * Create a cluster instance preserving the bootstrap state. + * This is used internally by MetadataSnapshot to maintain bootstrap flag across metadata updates. + * + * @param clusterId the cluster id + * @param nodes the nodes in the cluster + * @param partitions information about partitions + * @param unauthorizedTopics unauthorized topics + * @param invalidTopics invalid topics + * @param internalTopics internal topics + * @param controller the controller node + * @param topicIds topic IDs + * @param isBootstrapConfigured whether the cluster is bootstrapped + * @return a new Cluster instance with the specified bootstrap state + */ + public static Cluster withBootstrapFlag(String clusterId, + Collection nodes, + Collection partitions, + Set unauthorizedTopics, + Set invalidTopics, + Set internalTopics, + Node controller, + Map topicIds, + boolean isBootstrapConfigured) { + return new Cluster(clusterId, isBootstrapConfigured, nodes, partitions, + unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); + } + /** * Return a copy of this cluster combined with `partitions`. */ diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 78beb2557ebf4..bdd15fcb6710b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -161,6 +161,7 @@ public void testDrainBatches() throws Exception { Collections.emptySet(), null, Collections.emptyMap(), + false, cluster); long batchSize = value.length + DefaultRecordBatch.RECORD_BATCH_OVERHEAD; RecordAccumulator accum = createTestRecordAccumulator((int) batchSize, Integer.MAX_VALUE, Compression.NONE, 10); From ff5492e89dc7c7541336ecfecb28b891c9215014 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 27 Mar 2026 15:37:15 +0000 Subject: [PATCH 27/30] add interrupt mechanism --- .../main/java/org/apache/kafka/clients/NetworkClient.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 4395941dee61d..d59a842bf4e1a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.BootstrapResolutionException; import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion; import org.apache.kafka.common.metrics.Sensor; @@ -1235,6 +1236,10 @@ void ensureBootstrapped(final long currentTimeMs) { if (bootstrapConfiguration.isBootstrapDisabled || metadataUpdater.isBootstrapped()) return; + if (Thread.interrupted()) { + throw new InterruptException(new InterruptedException()); + } + // Timer is already initialized in constructor, just update it with current time bootstrapTimer.update(currentTimeMs); From c0ce72aea82130fd834a6906e7d4ac09b233b40a Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 27 Mar 2026 16:26:41 +0000 Subject: [PATCH 28/30] address comments --- .../org/apache/kafka/clients/Metadata.java | 8 ++---- .../kafka/clients/MetadataSnapshot.java | 17 ++++------- .../apache/kafka/clients/NetworkClient.java | 4 ++- .../java/org/apache/kafka/common/Cluster.java | 28 ------------------- .../internals/RecordAccumulatorTest.java | 1 - 5 files changed, 11 insertions(+), 47 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index ddc45a53e0b43..2686f042c2ba8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -545,13 +545,9 @@ else if (metadata.error() == Errors.TOPIC_AUTHORIZATION_FAILED) return this.metadataSnapshot.mergeWith(metadataResponse.clusterId(), nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, (topic, isInternal) -> !topics.contains(topic) && retainTopic(topic, isInternal, nowMs)); - else { - // Preserve the bootstrap flag from the current snapshot when creating a new one - boolean isBootstrapConfigured = this.metadataSnapshot.cluster().isBootstrapConfigured(); + else return new MetadataSnapshot(metadataResponse.clusterId(), nodes, partitions, - unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds, - isBootstrapConfigured, null); - } + unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(), topicIds); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java index 551b8b1391ebf..aa0caaef239c5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java +++ b/clients/src/main/java/org/apache/kafka/clients/MetadataSnapshot.java @@ -54,7 +54,6 @@ public class MetadataSnapshot { private final Map metadataByPartition; private final Map topicIds; private final Map topicNames; - private final boolean isBootstrapConfigured; private Cluster clusterInstance; public MetadataSnapshot(String clusterId, @@ -65,7 +64,7 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds) { - this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, false, null); + this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, null); } // Visible for testing @@ -77,7 +76,6 @@ public MetadataSnapshot(String clusterId, Set internalTopics, Node controller, Map topicIds, - boolean isBootstrapConfigured, Cluster clusterInstance) { this.clusterId = clusterId; this.nodes = Collections.unmodifiableMap(nodes); @@ -89,7 +87,6 @@ public MetadataSnapshot(String clusterId, this.topicNames = Collections.unmodifiableMap( topicIds.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)) ); - this.isBootstrapConfigured = isBootstrapConfigured; Map tmpMetadataByPartition = new HashMap<>(partitions.size()); for (PartitionMetadata p : partitions) { @@ -202,9 +199,8 @@ MetadataSnapshot mergeWith(String newClusterId, Set newInvalidTopics = fillSet(addInvalidTopics, invalidTopics, shouldRetainTopic); Set newInternalTopics = fillSet(addInternalTopics, internalTopics, shouldRetainTopic); - // Preserve the bootstrap flag from the current snapshot during merge return new MetadataSnapshot(newClusterId, newNodes, newMetadataByPartition.values(), newUnauthorizedTopics, - newInvalidTopics, newInternalTopics, newController, newTopicIds, this.isBootstrapConfigured, null); + newInvalidTopics, newInternalTopics, newController, newTopicIds); } /** @@ -231,9 +227,8 @@ private void computeClusterView() { .stream() .map(metadata -> MetadataResponse.toPartitionInfo(metadata, nodes)) .collect(Collectors.toList()); - // Use the factory method that preserves the bootstrap state - this.clusterInstance = Cluster.withBootstrapFlag(clusterId, nodes.values(), partitionInfos, - unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds, isBootstrapConfigured); + this.clusterInstance = new Cluster(clusterId, nodes.values(), partitionInfos, + unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); } static MetadataSnapshot bootstrap(List addresses) { @@ -245,12 +240,12 @@ static MetadataSnapshot bootstrap(List addresses) { } return new MetadataSnapshot(null, nodes, Collections.emptyList(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), - null, Collections.emptyMap(), true, Cluster.bootstrap(addresses)); + null, Collections.emptyMap(), Cluster.bootstrap(addresses)); } static MetadataSnapshot empty() { return new MetadataSnapshot(null, Collections.emptyMap(), Collections.emptyList(), - Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), false, Cluster.empty()); + Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap(), Cluster.empty()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index d59a842bf4e1a..00f32a6ffa6f7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -1417,7 +1417,9 @@ public void rebootstrap(long now) { @Override public boolean isBootstrapped() { - return metadata.fetch().isBootstrapConfigured(); + // Check if we have nodes (either from bootstrap or real metadata) + // If we have nodes, we can skip DNS resolution + return !metadata.fetch().nodes().isEmpty(); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 9c7acc7791931..93f2f4225bc74 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -217,34 +217,6 @@ public static Cluster bootstrap(List addresses) { Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Collections.emptyMap()); } - /** - * Create a cluster instance preserving the bootstrap state. - * This is used internally by MetadataSnapshot to maintain bootstrap flag across metadata updates. - * - * @param clusterId the cluster id - * @param nodes the nodes in the cluster - * @param partitions information about partitions - * @param unauthorizedTopics unauthorized topics - * @param invalidTopics invalid topics - * @param internalTopics internal topics - * @param controller the controller node - * @param topicIds topic IDs - * @param isBootstrapConfigured whether the cluster is bootstrapped - * @return a new Cluster instance with the specified bootstrap state - */ - public static Cluster withBootstrapFlag(String clusterId, - Collection nodes, - Collection partitions, - Set unauthorizedTopics, - Set invalidTopics, - Set internalTopics, - Node controller, - Map topicIds, - boolean isBootstrapConfigured) { - return new Cluster(clusterId, isBootstrapConfigured, nodes, partitions, - unauthorizedTopics, invalidTopics, internalTopics, controller, topicIds); - } - /** * Return a copy of this cluster combined with `partitions`. */ diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index bdd15fcb6710b..78beb2557ebf4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -161,7 +161,6 @@ public void testDrainBatches() throws Exception { Collections.emptySet(), null, Collections.emptyMap(), - false, cluster); long batchSize = value.length + DefaultRecordBatch.RECORD_BATCH_OVERHEAD; RecordAccumulator accum = createTestRecordAccumulator((int) batchSize, Integer.MAX_VALUE, Compression.NONE, 10); From 752d79c1d0d99b8020562adcf84988ba1eaddd7e Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Thu, 2 Apr 2026 15:54:46 +0100 Subject: [PATCH 29/30] address comments --- .../internals/NetworkClientDelegate.java | 8 +++- .../clients/consumer/KafkaConsumerTest.java | 40 +++++++++++++++++++ 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java index 0b433f33ba088..fba2808075d24 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java @@ -30,6 +30,7 @@ import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.BootstrapResolutionException; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.Metrics; @@ -157,7 +158,12 @@ public void poll(final long timeoutMs, final long currentTimeMs) { * @param onClose True when the network thread is closing. */ public void poll(final long timeoutMs, final long currentTimeMs, boolean onClose) { - trySend(currentTimeMs); + try { + trySend(currentTimeMs); + } catch (BootstrapResolutionException e) { + // Bootstrap DNS resolution timeout - propagate to app thread + backgroundEventHandler.add(new ErrorEvent(e)); + } long pollTimeoutMs = timeoutMs; if (!unsentRequests.isEmpty()) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index d17a3d5f4caf9..39938516bbb9e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -42,6 +42,7 @@ import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.BootstrapResolutionException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.InvalidGroupIdException; @@ -4134,6 +4135,45 @@ public void testConstructorFailsOnNetworkClientConstructorFailure(GroupProtocol assertEquals("No LoginModule found for org.example.InvalidLoginModule", cause.getMessage()); } + @Test + public void testAsyncConsumerBootstrapResolutionExceptionPropagatedToPoll() throws InterruptedException { + // Use an invalid hostname that will fail DNS resolution + String invalidHost = "invalid-host-that-does-not-exist-12345.example.com:9092"; + + Map configs = Map.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName(), + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName(), + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, invalidHost, + // Set a short bootstrap timeout so the test doesn't take too long + CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, "3000", + ConsumerConfig.GROUP_PROTOCOL_CONFIG, "consumer", + ConsumerConfig.GROUP_ID_CONFIG, "test-group" + ); + + try (KafkaConsumer consumer = new KafkaConsumer<>(configs)) { + // Subscribe to a topic to trigger metadata fetch + consumer.subscribe(Set.of("test-topic")); + + // Poll continuously until we get the BootstrapResolutionException + // The exception should be thrown after bootstrap.resolve.timeout.ms expires + BootstrapResolutionException exception = + assertThrows(BootstrapResolutionException.class, () -> { + long startTime = System.currentTimeMillis(); + long maxWaitTime = 15000; // 15 seconds max to prevent test hanging + + while (System.currentTimeMillis() - startTime < maxWaitTime) { + consumer.poll(Duration.ofMillis(100)); + } + fail("Expected BootstrapResolutionException to be thrown within " + maxWaitTime + "ms"); + }); + + // Verify the exception message contains information about DNS resolution failure + assertTrue(exception.getMessage().contains("Failed to resolve bootstrap servers") || + exception.getMessage().contains("DNS resolution"), + "Exception message should mention DNS resolution failure: " + exception.getMessage()); + } + } + private MetricName expectedMetricName(String clientId, String config, Class clazz) { Map expectedTags = new LinkedHashMap<>(); expectedTags.put("client-id", clientId); From f78ddc397a5c2c5831a224a7a6fedee56294af38 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 8 Apr 2026 15:15:41 +0100 Subject: [PATCH 30/30] address comments --- .../apache/kafka/clients/NetworkClient.java | 24 +++++++++++++++---- .../kafka/clients/NetworkClientTest.java | 11 +++++---- .../clients/consumer/KafkaConsumerTest.java | 7 +++--- 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 00f32a6ffa6f7..108b22238f085 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -777,8 +777,9 @@ public void close() { @Override public LeastLoadedNode leastLoadedNode(long now) { List nodes = this.metadataUpdater.fetchNodes(); - if (nodes.isEmpty()) - throw new IllegalStateException("There are no nodes in the Kafka cluster"); + if (nodes.isEmpty()) { + return handleEmptyNodeList(); + } int inflight = Integer.MAX_VALUE; Node foundConnecting = null; @@ -840,6 +841,19 @@ public LeastLoadedNode leastLoadedNode(long now) { } } + /** + * Handle the case when there are no nodes available. + * During bootstrap phase, return null node to allow DNS resolution to continue. + * After bootstrap, throw IllegalStateException. + */ + private LeastLoadedNode handleEmptyNodeList() { + if (!metadataUpdater.isBootstrapped()) { + log.debug("No nodes available yet, still in bootstrap phase"); + return new LeastLoadedNode(null, false); + } + throw new IllegalStateException("There are no nodes in the Kafka cluster"); + } + public static AbstractResponse parseResponse(ByteBuffer responseBuffer, RequestHeader requestHeader) { try { return AbstractResponse.parseResponse(responseBuffer, requestHeader); @@ -1320,7 +1334,10 @@ public long maybeUpdate(long now) { LeastLoadedNode leastLoadedNode = leastLoadedNode(now); // Rebootstrap if needed and configured. + // Only rebootstrap if we've already completed initial bootstrap - otherwise we're still + // in the initial DNS resolution phase and should let ensureBootstrapped() handle it. if (metadataRecoveryStrategy == MetadataRecoveryStrategy.REBOOTSTRAP + && isBootstrapped() && !leastLoadedNode.hasNodeAvailableOrConnectionReady()) { rebootstrap(now); @@ -1417,8 +1434,7 @@ public void rebootstrap(long now) { @Override public boolean isBootstrapped() { - // Check if we have nodes (either from bootstrap or real metadata) - // If we have nodes, we can skip DNS resolution + // We are bootstrapped if we have any nodes available (either from DNS resolution or metadata response) return !metadata.fetch().nodes().isEmpty(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index f49a5ba7c236a..82909e45342f5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -1562,10 +1562,13 @@ public void testEnsureBootstrappedTimeoutThrowsException() { time, false, new ApiVersions(), new LogContext(), MetadataRecoveryStrategy.NONE, config); + // First poll to initialize bootstrap timer + client.poll(10, time.milliseconds()); + // Advance time past bootstrap timeout time.sleep(150); - // Should throw BootstrapResolutionException + // Should throw BootstrapResolutionException on next poll assertThrows(BootstrapResolutionException.class, () -> client.poll(1000, time.milliseconds())); } @@ -1587,11 +1590,11 @@ time, false, new ApiVersions(), new LogContext(), // Directly call ensureBootstrapped // Should return without error even though bootstrap hasn't succeeded (will retry on next poll) + // DNS resolution will fail but timeout hasn't been reached yet client.ensureBootstrapped(time.milliseconds()); - // Verify bootstrap has not succeeded yet - MetadataUpdater metadataUpdater = TestUtils.fieldValue(client, NetworkClient.class, "metadataUpdater"); - assertFalse(metadataUpdater.isBootstrapped()); + // Verify that no exception was thrown and metadata is still empty + assertEquals(0, metadata.fetch().nodes().size(), "Metadata should have no nodes after failed DNS resolution"); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index be6aeafc67b9f..f3885f020cd4a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -4197,8 +4197,9 @@ public void testConstructorFailsOnNetworkClientConstructorFailure(GroupProtocol assertEquals("No LoginModule found for org.example.InvalidLoginModule", cause.getMessage()); } - @Test - public void testAsyncConsumerBootstrapResolutionExceptionPropagatedToPoll() throws InterruptedException { + @ParameterizedTest + @EnumSource(value = GroupProtocol.class) + public void testConsumerBootstrapResolutionExceptionPropagatedToPoll(GroupProtocol protocol) throws InterruptedException { // Use an invalid hostname that will fail DNS resolution String invalidHost = "invalid-host-that-does-not-exist-12345.example.com:9092"; @@ -4208,7 +4209,7 @@ public void testAsyncConsumerBootstrapResolutionExceptionPropagatedToPoll() thro CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, invalidHost, // Set a short bootstrap timeout so the test doesn't take too long CommonClientConfigs.BOOTSTRAP_RESOLVE_TIMEOUT_MS_CONFIG, "3000", - ConsumerConfig.GROUP_PROTOCOL_CONFIG, "consumer", + ConsumerConfig.GROUP_PROTOCOL_CONFIG, protocol.name(), ConsumerConfig.GROUP_ID_CONFIG, "test-group" );