showuon commented on PR #18165: URL: https://github.com/apache/kafka/pull/18165#issuecomment-2611707118
> @showuon @mimaison Thanks for the patch. I'd like to better understand the impact. My understanding is that the controller is not cleared on NOT_CONTROLLER or NOT_LEADER_OR_FOLLOWER errors and hence the admin client can no longer communicate to the active controller because it keeps sending requests to the old one. Is my understanding correct? @dajac , sorry that I didn't make it clear in the description. The root cause is not on the error should be `NOT_CONTROLLER` or `NOT_LEADER_OR_FOLLOWER`, it is that the admin client didn't handle the `NOT_CONTROLLER` or `NOT_LEADER_OR_FOLLOWER` error when talking directly to the controller, which causes the admin client can only get the exception. Take `describeMetadataQuorum` for example, we'll use `LeastLoadedBrokerOrActiveKController` to get the active controller via describeCluster/Metadata API, then send `describeMetadataQuorum` to the active controller. You can see, it's possible that the active controller changed between right after `describeCluster/Metadata` call and before `describeMetadataQuorum` call. And you can see how we handle the `describeMetadataQuorum` response [here](https://github.com/apache/kafka/blob/ad79b4afa779a2a577358787096b4fe016e2d67e/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L4654). We don't handle `NOT_CONTROLLER` nor `NOT_LEADER_OR_FOLLOWER` errors. The error response will look like this: ``` DescribeQuorumResponseData(errorCode=0, errorMessage='', topics=[TopicData(topicName='__cluster_metadata', partitions=[PartitionData(partitionIndex=0, errorCode=6, errorMessage='For requests intended only for the leader, this error indicates that the broker is not the current leader. For requests intended for any replica, this error indicates that the broker is not a replica of the topic partition.', leaderId=0, leaderEpoch=0, highWatermark=0, currentVoters=[], observers=[])])], nodes=[]) ``` Comparably, when we invoke [createTopic](https://github.com/apache/kafka/blob/ad79b4afa779a2a577358787096b4fe016e2d67e/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L1848), deleteTopic, alterPartitionReassignment, ... we'll invoke `handleNotControllerError` to handle controller change, because we know when brokers receive these calls, they will forward to the active controller, but when there's controller leadership change, we need to re-fetch the metadata and retry. Please note that when talking directly to the controller, we might get `NOT_LEADER_OR_FOLLOWER`( ex: [here](https://github.com/apache/kafka/blob/ad79b4afa779a2a577358787096b4fe016e2d67e/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java#L1848)) because in the controller quorum's perspective, this controller is not a leader. That's why I added this in `handleNotControllerError`: `metadataManager.usingBootstrapControllers() && response.errorCounts().containsKey(Errors.NOT_LEADER_OR_FOLLOWER))` I think this is just a miss when we were implementing [KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum](https://cwiki.apache.org/confluence/display/KAFKA/KIP-919%3A+Allow+AdminClient+to+Talk+Directly+with+the+KRaft+Controller+Quorum+and+add+Controller+Registration). And that's why in this PR, I handled `NOT_CONTROLLER` and `NOT_LEADER_OR_FOLLOWER` not just for `describeMetadataQuorum`, but also for the requests that talk to controller directly, and the request must need the active controller to handle (ex: controller will modify the metadata log, requesting raft update,...). The APIs are: 1. createAcls 2. deleteAcls 3. alterConfigs 4. describeMetadataQuorum 5. addRaftVoter 6. removeRaftVoter Hope that's clear. I've also updated the PR description. Thanks. -- 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