-
Notifications
You must be signed in to change notification settings - Fork 3.2k
Fix reboostrapping with no bootstrap servers #5067
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Fix reboostrapping with no bootstrap servers #5067
Conversation
🎉 All Contributor License Agreements have been signed. Ready to merge. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pull Request Overview
This PR fixes the re-bootstrap issue when no bootstrap servers are configured by ensuring that brokers added manually via rd_kafka_brokers_add are handled properly. The changes include:
- Adding a new test (0152-rebootstrap.c) to validate the re-bootstrap sequence without bootstrap servers.
- Renaming and updating the topic info destroy function from rd_kafka_topic_info_destroy to rd_kafka_topic_info_destroy_free in both header and source files.
- Adding a brokerlist null-check in the rebootstrap callback in src/rdkafka.c.
Reviewed Changes
Copilot reviewed 7 out of 9 changed files in this pull request and generated 1 comment.
Show a summary per file
File | Description |
---|---|
tests/test.c | Registers the new rebootstrap test. |
tests/0152-rebootstrap.c | Implements the test verifying re-bootstrap behavior with manual brokers. |
src/rdkafka_topic.h | Renames the destroy function to rd_kafka_topic_info_destroy_free. |
src/rdkafka_topic.c | Updates the destroy function implementation to use the new free function. |
src/rdkafka_cgrp.c | Updates all list callbacks that free topic info with the new function. |
src/rdkafka.c | Adds a conditional check on brokerlist before invoking the re-bootstrap logic. |
CHANGELOG.md | Documents the fix and details regarding the re-bootstrap change. |
Files not reviewed (2)
- tests/CMakeLists.txt: Language not supported
- win32/tests/tests.vcxproj: Language not supported
Comments suppressed due to low confidence (2)
src/rdkafka_topic.h:293
- The function has been renamed to rd_kafka_topic_info_destroy_free to better reflect its behavior. Please ensure the corresponding documentation and comments are updated to indicate the new semantics and that all references use the new name.
-void rd_kafka_topic_info_destroy(rd_kafka_topic_info_t *ti);
tests/0152-rebootstrap.c:48
- [nitpick] The test currently relies on a sleep call to allow time for ALL_BROKERS_DOWN to trigger. Consider adding explicit assertions or verifications to ensure the re-bootstrap logic behaves as expected rather than relying solely on timing.
rd_sleep(1);
if (rk->rk_conf.brokerlist) { | ||
rd_kafka_brokers_add0( | ||
rk, | ||
rk->rk_conf.brokerlist, rd_true | ||
/* resolve canonical bootstrap server | ||
* list names if requested*/); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
[nitpick] The added null-check for brokerlist improves safety; however, consider clarifying the expected behavior when the brokerlist is an empty string. A comment or additional guard could help maintain clarity.
Copilot uses AI. Check for mistakes.
|
||
int main_0152_rebootstrap_local(int argc, char **argv) { | ||
|
||
do_test_rebootstrap_local_no_bootstrap_servers(RD_KAFKA_PRODUCER); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should add for normal case as well as for empty string case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I added the rebootstrap cases tests for 899 and 1102 to the KIP-1102 PR
https://github.com/confluentinc/librdkafka/pull/4981/files#diff-30f310cb830cfab99eff2f2a0fd76f3ebf3bbae73cf0c6fcbdfec5b7d2ca8511R851
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was talking about empty bootstrap brokers string case. Lets add it in that PR.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok
Update the PR and we can merge. |
when re-bootstrapping
…er to incorrect function type
given no `boostrap.servers` is present and brokers were added through `rd_kafka_brokers_add` Closes #5057
7646f48
to
88e978b
Compare
|
||
int main_0152_rebootstrap_local(int argc, char **argv) { | ||
|
||
do_test_rebootstrap_local_no_bootstrap_servers(RD_KAFKA_PRODUCER); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was talking about empty bootstrap brokers string case. Lets add it in that PR.
LGTM!. Thanks. |
when brokers were added only through
rd_kafka_brokers_add
.