Skip to content
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 remove fetch queue messages that blocked the destroy of rdkafka instances #4724

Merged
merged 2 commits into from
Oct 30, 2024

Conversation

emasab
Copy link
Contributor

@emasab emasab commented May 21, 2024

Circular dependencies from a partition fetch queue message to the same partition blocked the destroy of an instance, that happened in case the partition was removed from the cluster while it was being consumed. Solved by purging internal partition queue, after being stopped and removed, to allow reference count to reach zero and trigger a destroy

@emasab
Copy link
Contributor Author

emasab commented May 21, 2024

How to reproduce: happening sporadically with test 0113 subtest n_wildcard. Run it it with TEST_DEBUG=all and until-fail.sh to see the refcnt not reaching zero.

@zuellig
Copy link

zuellig commented Aug 26, 2024

FWIW I've confirmed that this branch also fixes a non-constant but somewhat frequent issue I've been observing. I reproduced it by stressing the client node's swap while also stressing the broker node's cpu. It took a few restart cycles but within several hours the deadlocked destroy call occurred.

@antaljanosbenjamin
Copy link

Is there any expected date on this to be merged?

@confluent-cla-assistant
Copy link

🎉 All Contributor License Agreements have been signed. Ready to merge.
Please push an empty commit if you would like to re-run the checks to verify CLA status for all contributors.

@antaljanosbenjamin
Copy link

All Contributor License Agreements have been signed. Ready to merge.

One step closer! 🎉

@@ -3407,6 +3407,8 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) {
: (topic_err
? topic_err
: RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION));

rd_kafka_toppar_purge_internal_fetch_queue_maybe(rktp);
Copy link
Member

Choose a reason for hiding this comment

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

Shall we do this only for the Consumer?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, better to avoid it for the producer given rktp_fetchq is allocated there too but the queue isn't forwarded we avoid looping through this empty queue.

…a instances

Circular dependencies from a partition fetch queue message to  the same partition blocked the destroy of an instance, that happened in case the partition was removed from the cluster while it was being consumed. Solved by purging internal partition queue, after being stopped and removed, to allow reference count to reach zero and trigger a destroy
on removing the partition only for the consumer
@airlock-confluentinc airlock-confluentinc bot force-pushed the dev_fix_op_toppar_blocking_destroy branch from 9b9d001 to 7ce0d72 Compare October 29, 2024 14:49
@emasab
Copy link
Contributor Author

emasab commented Oct 29, 2024

Addressed comment, updated CHANGELOG and rebased

@emasab emasab requested a review from pranavrth October 29, 2024 14:50
@ydsun90
Copy link

ydsun90 commented Oct 30, 2024

Hi, is there an estimated time to have this fix in a release? I have got the same issue when closing kafka consumer #4885.

Copy link
Member

@pranavrth pranavrth left a comment

Choose a reason for hiding this comment

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

LGTM!.

@emasab emasab merged commit 9199335 into master Oct 30, 2024
2 checks passed
@emasab emasab deleted the dev_fix_op_toppar_blocking_destroy branch October 30, 2024 12:44
@ydsun90
Copy link

ydsun90 commented Nov 5, 2024

Hi, is there an estimated time to have this fix in a release? I have got the same issue when closing kafka consumer #4885.

Any plan to release the fix in a new version?

@ydsun90
Copy link

ydsun90 commented Nov 13, 2024

Hi, is there an estimated time to have this fix in a release? I have got the same issue when closing kafka consumer #4885.

Any plan to release the fix in a new version?

@pranavrth @emasab Hi, is there any expected time to have this fix in a new release?

airlock-confluentinc bot pushed a commit that referenced this pull request Nov 29, 2024
…a instances (#4724)

Circular dependencies from a partition fetch queue message to  the same partition blocked the destroy of an instance, that happened in case the partition was removed from the cluster while it was being consumed. Solved by purging internal partition queue, after being stopped and removed, to allow reference count to reach zero and trigger a destroy.

Purging internal fetch queue on removing the partition only for the consumer.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants