-
Notifications
You must be signed in to change notification settings - Fork 3.2k
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 to metadata cache expiration on full metadata refresh #4677
Fix to metadata cache expiration on full metadata refresh #4677
Conversation
fa6ea32
to
6f3c1b5
Compare
4d7eab9
to
8ea3b9e
Compare
b27ccc6
to
d4d40c1
Compare
d4d40c1
to
d9540da
Compare
rd_kafka_t *rk, | ||
const rd_kafka_metadata_topic_t *mdt, | ||
const rd_kafka_metadata_topic_internal_t *mdit, | ||
rd_bool_t propagate, | ||
rd_bool_t include_metadata, | ||
rd_kafka_metadata_broker_internal_t *brokers, | ||
size_t broker_cnt); | ||
size_t broker_cnt, | ||
rd_bool_t only_existing); |
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.
Better name maybe update_only_existing
and let's explain what it means in the doc.
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.
If you search for only_
the code convention in this repo is to avoid repeating the action in variable names, for example in rd_kafka_buf_write_topic_partitions
we have only_invalid_offsets
not write_only_invalid_offsets
.
Okey for the doc
src/rdkafka_metadata_cache.c
Outdated
rkmce = rd_kafka_metadata_cache_find(rk, mdt->topic, 0); | ||
if (!rkmce) | ||
return 0; | ||
} else if (unlikely(!mdt->topic)) { |
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.
This is not checking only_existing
part.
on full metadata refresh Metadata cache was cleared on full metadata refresh, leading to unnecessary refreshes and occasional `UNKNOWN_TOPIC_OR_PART` errors. Solved by updating cache for existing or hinted entries instead of clearing them. Happening since 2.1.0
d9540da
to
a159193
Compare
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.
LGTM!. Just a couple of minor comments.
@@ -35,15 +35,6 @@ def read_scenario_conf(scenario): | |||
return parser.load(f) | |||
|
|||
|
|||
# FIXME: merge in trivup | |||
class KafkaBrokerApp(KafkaBrokerAppOrig): |
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.
This is expected right?
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.
Yeah, it was added back during a rebase
int main_0146_metadata_mock(int argc, char **argv) { | ||
TEST_SKIP_MOCK_CLUSTER(0); | ||
|
||
do_test_metadata_persists_in_cache("range"); |
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.
why not round robin 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.
The difference can be because of taking the eager path instead of the incremental one, but roundrobin also takes the eager path, as "range", so it's not important for this test.
Metadata cache was cleared on full metadata refresh, leading to unnecessary refreshes and occasional `UNKNOWN_TOPIC_OR_PART` errors. Solved by updating cache for existing or hinted entries instead of clearing them. Happening since 2.1.0
Metadata cache was cleared on full metadata refresh, leading to unnecessary refreshes and occasional `UNKNOWN_TOPIC_OR_PART` errors. Solved by updating cache for existing or hinted entries instead of clearing them. Happening since 2.1.0
Metadata cache was cleared on full metadata refresh, leading to unnecessary refreshes and occasional `UNKNOWN_TOPIC_OR_PART` errors. Solved by updating cache for existing or hinted entries instead of clearing them. Happening since 2.1.0
Metadata cache was cleared on full metadata
refresh, leading to unnecessary refreshes and
occasional
UNKNOWN_TOPIC_OR_PART
errors.Solved by updating cache for existing or
hinted entries instead of clearing them.
Happening since 2.1.0