Skip to content

KAFKA-18230: handle not controller or not leader error in admin client#18165

Merged
mimaison merged 2 commits intoapache:trunkfrom
showuon:KAFKA-18230
Feb 4, 2025
Merged

KAFKA-18230: handle not controller or not leader error in admin client#18165
mimaison merged 2 commits intoapache:trunkfrom
showuon:KAFKA-18230

Conversation

@showuon
Copy link
Copy Markdown
Member

@showuon showuon commented Dec 13, 2024

When admin client starts up, it'll get the metadata of the cluster. And when the admin client sends request directly to the controller (via --bootstrap-controller), it'll send the request to the active controller. But if there is a leadership change in the controller after the metadata request and before the target request sent, the request will fail immediately with NOT_CONTROLLER error or NOT_LEADER_OR_FOLLOWER error. It's because the requests that need metadata log change must need to do on the active controller. Instead of failing immediately, the admin client should catch the error and retry the metadata update to send the request again. Note, in some application, the admin client could exist for a long time to send multiple requests when needed, this case could happen more often.

Take describeMetadataQuorum for example, we'll use LeastLoadedBrokerOrActiveKController(here) 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 right after describeCluster/Metadata call and before describeMetadataQuorum call, and even worse if the application creates an long running adminClient to handle any calls in the lifecycle. And you can see how we handle the describeMetadataQuorum response here. 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=[])

Since the NOT_LEADER_OR_FOLLOWER and NOT_CONTROLLER are both retriable errors, when receiving them, the admin client will keep retrying until request time out or metadata expired in config metadata.max.age.ms (default is 5 mins).

Comparably, when we invoke createTopic, 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) 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. 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

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@github-actions github-actions bot added triage PRs from the community core Kafka Broker clients labels Dec 13, 2024
@mumrah
Copy link
Copy Markdown
Member

mumrah commented Dec 13, 2024

@showuon #17881 adds a "triage" label to PRs from non-committers. Turns out this also affect committers if their membership visibility in the ASF GitHub org is not public. I added instructions for setting your membership visibility to public https://github.com/apache/kafka/blob/trunk/.github/workflows/README.md#pr-triage

@github-actions

This comment was marked as outdated.

@mumrah mumrah removed triage PRs from the community needs-attention labels Dec 23, 2024
@showuon
Copy link
Copy Markdown
Member Author

showuon commented Jan 16, 2025

@chia7712 @cmccabe , could you help take a look? Thanks.

@mimaison
Copy link
Copy Markdown
Member

@dajac @jolshan Can you take a look?
I think it's a good candidate for 4.0 too, as this really impacts Admin clients when the controller quorum rolls.

@dajac
Copy link
Copy Markdown
Member

dajac commented Jan 23, 2025

@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?

@showuon
Copy link
Copy Markdown
Member Author

showuon commented Jan 24, 2025

@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. Yes, you're right, the root cause 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(here) 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 right after describeCluster/Metadata call and before describeMetadataQuorum call, and even worse if the application creates an long running adminClient to handle any calls in the lifecycle. And you can see how we handle the describeMetadataQuorum response here. 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=[])

Since the NOT_LEADER_OR_FOLLOWER and NOT_CONTROLLER are both retriable errors, when receiving them, the admin client will keep retrying until request time out or metadata expired in config metadata.max.age.ms (default is 5 mins).

Comparably, when we invoke createTopic, 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) 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. 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 supports to 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.

@dajac
Copy link
Copy Markdown
Member

dajac commented Jan 30, 2025

@showuon Thanks for the explanation. I am OK with getting this one into 4.0. However, I don't have time for reviewing it. @mimaison Could you review it?

@mimaison
Copy link
Copy Markdown
Member

I'll try to take a look today. @AndrewJSchofield if you have time, can you take a look too? Thanks

Copy link
Copy Markdown
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@jolshan
Copy link
Copy Markdown
Member

jolshan commented Jan 31, 2025

Sorry, for some reason I only get tags when someone approves/merges the PR. Thanks for taking a look @mimaison!

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@showuon thanks for this patch, and I have only some small comment. otherwise, LGTM

// 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);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we pass NOT_LEADER_OR_FOLLOWER instead of NOT_CONTROLLER when it encounters the error NOT_LEADER_OR_FOLLOWER?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree. Updated.

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);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we can replace usingBootstrapController by config.containsKey(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) to streamline it

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good suggestion. Updated.

@mimaison mimaison merged commit 612e129 into apache:trunk Feb 4, 2025
mimaison pushed a commit that referenced this pull request Feb 4, 2025
#18165)


Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
@mimaison
Copy link
Copy Markdown
Member

mimaison commented Feb 4, 2025

Applied to 4.0 too: 8026d6b

pdruley pushed a commit to pdruley/kafka that referenced this pull request Feb 12, 2025
apache#18165)


Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
manoj-mathivanan pushed a commit to manoj-mathivanan/kafka that referenced this pull request Feb 19, 2025
apache#18165)


Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

clients core Kafka Broker

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants