You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Consider the following sequence of events for a Kafka consumer (not using consumer groups):
Assign consumer partition 0.
Create partition 1.
Assign consumer partition 1.
Refresh metadata.
The metadata refresh will incorrectly generate an error that partition 0 is unknown. The issue seems to be a glitch into how the desired partition list (rkt->rkt_deps) is maintained.
This reproduces on the latest librdkafka etc. I'm going to submit a PR with a test case and a fix momentarily, but wanted an issue number to put in the test.
The text was updated successfully, but these errors were encountered:
rd_kafka_cgrp_assign calls rd_kafka_toppar_desired_add0 rather than its
wrapper rd_kafka_toppar_desired_add. The "add" wrapper preserves the
invariant that a known topic should never get added to the desired
partitions queue, while the "add0" function does not.
Maintaining this invariant is important for
rd_kafka_topic_partition_cnt_update, which assumes that a toppar is in
either the list of known partitions or the list of desired partitions,
but not both. Violating this invariant results in the situation
described in confluentinc#2915, where updating assignments can trigger incorrect
"unknown partition" errors.
This patch rearranges rd_kafka_toppar_desired_add/add0 so that add0, in
addition to add, will avoid adding known partitions to the desired
partition list. The enclosed test correctly fails if run against the
current master (for the reasons described above).
Fixconfluentinc#2915.
rd_kafka_cgrp_assign calls rd_kafka_toppar_desired_add0 rather than its
wrapper rd_kafka_toppar_desired_add. The "add" wrapper preserves the
invariant that a known topic should never get added to the desired
partitions queue, while the "add0" function does not.
Maintaining this invariant is important for
rd_kafka_topic_partition_cnt_update, which assumes that a toppar is in
either the list of known partitions or the list of desired partitions,
but not both. Violating this invariant results in the situation
described in #2915, where updating assignments can trigger incorrect
"unknown partition" errors.
This patch rearranges rd_kafka_toppar_desired_add/add0 so that add0, in
addition to add, will avoid adding known partitions to the desired
partition list. The enclosed test correctly fails if run against the
current master (for the reasons described above).
Fix#2915.
Consider the following sequence of events for a Kafka consumer (not using consumer groups):
The metadata refresh will incorrectly generate an error that partition 0 is unknown. The issue seems to be a glitch into how the desired partition list (
rkt->rkt_deps
) is maintained.This reproduces on the latest librdkafka etc. I'm going to submit a PR with a test case and a fix momentarily, but wanted an issue number to put in the test.
The text was updated successfully, but these errors were encountered: