chia7712 commented on code in PR #21080:
URL: https://github.com/apache/kafka/pull/21080#discussion_r2954580716


##########
clients/src/main/java/org/apache/kafka/clients/ClientUtils.java:
##########
@@ -228,6 +282,20 @@ public static NetworkClient 
createNetworkClient(AbstractConfig config,
                     metricsGroupPrefix,
                     channelBuilder,
                     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

Review Comment:
   Out of curiosity, why to resolve the dns for now? IIRC, KIP-909 tries to 
handle the resolution error in the poll loop, right?



##########
clients/src/main/java/org/apache/kafka/clients/NetworkClient.java:
##########
@@ -1171,6 +1197,110 @@ private boolean isTelemetryApi(ApiKeys apiKey) {
         return apiKey == ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS || apiKey == 
ApiKeys.PUSH_TELEMETRY;
     }
 
+    public static class BootstrapConfiguration {
+        public final List<String> bootstrapServers;
+        public final ClientDnsLookup clientDnsLookup;
+        public final long bootstrapResolveTimeoutMs;
+        public final long retryBackoffMs;
+        private boolean isBootstrapDisabled;

Review Comment:
   `isBootstrapDisabled` could be a final field, since it is never modified 
after initialization, right?



##########
clients/src/main/java/org/apache/kafka/clients/ClientUtils.java:
##########
@@ -228,6 +282,20 @@ public static NetworkClient 
createNetworkClient(AbstractConfig config,
                     metricsGroupPrefix,
                     channelBuilder,
                     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(

Review Comment:
   line#275 could be merged into this one.



##########
clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java:
##########
@@ -102,6 +103,19 @@ default boolean needsRebootstrap(long now, long 
rebootstrapTriggerMs) {
      */
     default void rebootstrap(long now) {}
 
+    /**
+     * Returns true if the metadata has been bootstrapped.
+     */
+    default boolean isBootstrapped() {

Review Comment:
   It seems all subclasses have implemented this method. We could remove the 
default implementation for now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to