chia7712 commented on code in PR #18165: URL: https://github.com/apache/kafka/pull/18165#discussion_r1937733425
########## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ########## @@ -4081,7 +4084,9 @@ void handleFailure(Throwable throwable) { } private void handleNotControllerError(AbstractResponse response) throws ApiException { - if (response.errorCounts().containsKey(Errors.NOT_CONTROLLER)) { + // When sending requests directly to the follower controller, it might return NOT_LEADER_OR_FOLLOWER error. + if (response.errorCounts().containsKey(Errors.NOT_CONTROLLER) || + metadataManager.usingBootstrapControllers() && response.errorCounts().containsKey(Errors.NOT_LEADER_OR_FOLLOWER)) { handleNotControllerError(Errors.NOT_CONTROLLER); Review Comment: Should we pass `NOT_LEADER_OR_FOLLOWER` instead of `NOT_CONTROLLER` when it encounters the error `NOT_LEADER_OR_FOLLOWER`? ########## clients/src/test/java/org/apache/kafka/clients/admin/AdminClientUnitTestEnv.java: ########## @@ -71,9 +71,13 @@ public AdminClientUnitTestEnv(Time time, Cluster cluster, Map<String, Object> co this.cluster = cluster; AdminClientConfig adminClientConfig = new AdminClientConfig(config); + boolean usingBootstrapController = false; + if (config.containsKey(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG)) { + usingBootstrapController = true; + } AdminMetadataManager metadataManager = new AdminMetadataManager(new LogContext(), adminClientConfig.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG), - adminClientConfig.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), false); + adminClientConfig.getLong(AdminClientConfig.METADATA_MAX_AGE_CONFIG), usingBootstrapController); Review Comment: we can replace `usingBootstrapController` by `config.containsKey(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG)` to streamline it -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org