KAFKA-18230: handle not controller or not leader error in admin client#18165
KAFKA-18230: handle not controller or not leader error in admin client#18165mimaison merged 2 commits intoapache:trunkfrom
Conversation
|
@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 |
This comment was marked as outdated.
This comment was marked as outdated.
|
@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 Take Since the Comparably, when we invoke createTopic, deleteTopic, alterPartitionReassignment, ... we'll invoke Please note that when talking directly to the controller, we might get 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
Hope that's clear. I've also updated the PR description. Thanks. |
|
I'll try to take a look today. @AndrewJSchofield if you have time, can you take a look too? Thanks |
|
Sorry, for some reason I only get tags when someone approves/merges the PR. Thanks for taking a look @mimaison! |
| // 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); |
There was a problem hiding this comment.
Should we pass NOT_LEADER_OR_FOLLOWER instead of NOT_CONTROLLER when it encounters the error NOT_LEADER_OR_FOLLOWER?
| 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); |
There was a problem hiding this comment.
we can replace usingBootstrapController by config.containsKey(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG) to streamline it
There was a problem hiding this comment.
Good suggestion. Updated.
#18165) Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
|
Applied to 4.0 too: 8026d6b |
apache#18165) Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
apache#18165) Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
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
describeMetadataQuorumfor example, we'll useLeastLoadedBrokerOrActiveKController(here) to get the active controller via describeCluster/Metadata API, then senddescribeMetadataQuorumto the active controller. You can see, it's possible that the active controller changed right afterdescribeCluster/Metadatacall and beforedescribeMetadataQuorumcall, 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 thedescribeMetadataQuorumresponse here. We don't handleNOT_CONTROLLERnorNOT_LEADER_OR_FOLLOWERerrors. The error response will look like this: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
handleNotControllerErrorto 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 inhandleNotControllerError: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_CONTROLLERandNOT_LEADER_OR_FOLLOWERnot just fordescribeMetadataQuorum, 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:Committer Checklist (excluded from commit message)