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

hang in destructor of RdKafka::KafkaConsumer #3954

Open
4 of 7 tasks
rickif opened this issue Aug 26, 2022 · 8 comments
Open
4 of 7 tasks

hang in destructor of RdKafka::KafkaConsumer #3954

rickif opened this issue Aug 26, 2022 · 8 comments

Comments

@rickif
Copy link

rickif commented Aug 26, 2022

Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ

Do NOT create issues for questions, use the discussion forum: https://github.com/edenhill/librdkafka/discussions

Description

I got a process hang with the librdkafka v1.8.2.
I make some investigation about this.

  1. The direct reson is the process is stuck in thread join of the 139923336308480 (0x7f4270c1a700 in hex) destructor of RdKafka::KafkaConsumer
    image
  2. The thread 0x7f4270c1a700 is joinning another thread 0x7f426ec16700
    image
  3. The thread 0x7f426ec16700 is in the infinite loop of https://github.com/edenhill/librdkafka/blob/2d78e928d8c0d798f341b1843c97eb6dcdecefc3/src/rdkafka_broker.c#L5266
    image
    The rkb->rkb_refcnt is 3 so the check at https://github.com/edenhill/librdkafka/blob/2d78e928d8c0d798f341b1843c97eb6dcdecefc3/src/rdkafka_broker.c#L5266
    does not work at all.
    image

The complete pstack ouput of the process is attached as attachment.

How to reproduce

<your steps how to reproduce goes here, or remove section if not relevant>
It's an accident hang and I have not found the way to reproduce it stably.

IMPORTANT: Always try to reproduce the issue on the latest released version (see https://github.com/edenhill/librdkafka/releases), if it can't be reproduced on the latest version the issue has been fixed.

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

  • librdkafka version (release number or git tag): v1.8.2
  • Apache Kafka version: 2.3.0
  • librdkafka client configuration: enable.partition.eof=true, enable.auto.offset.store=false, statistics.interval.ms=0, auto.offset.reset=error, api.version.request=true
  • Operating system: CentOS Linux release 7.9.2009
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
    pstack.txt
@rickif
Copy link
Author

rickif commented Aug 26, 2022

https://github.com/edenhill/librdkafka/blob/2d78e928d8c0d798f341b1843c97eb6dcdecefc3/src/rdkafka_broker.c#L5393
How about adding this check to the loop condition to solve this problem?

@ruicao93
Copy link

@edenhill Is this a known issue?

@Quuxplusone
Copy link
Contributor

FWIW, I work on a codebase where we don't use RdKafka::KafkaConsumer but we do wrap librdkafka in our own C++ wrapper; and we very reproducibly see a hang in our destructor with exactly these symptoms, whenever the destructor is called, if we have a consumer group and if the broker is unreachable (e.g. the Kafka server is stopped first, then after that our consumer is destroyed).
Our symptoms are exactly the same: rd_kafka_destroy_app waiting on rd_kafka_destroy_internal waiting on rd_kafka_broker_thread_main, which never exits because rkb->rkb_refcnt == 3. However, I have been unable to make much progress debugging this, because "leaked a refcount somewhere" is such a vague root cause. We have the exact same symptom but that doesn't mean that we necessarily have the same bug at all. This happens for us in 1.8.2, and I have verified that upgrading to 1.9.2 doesn't fix it (for us).

@edenhill
Copy link
Contributor

edenhill commented Oct 3, 2022

Make sure all outstanding objects are destroyed prior to calling close.

See https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#termination

@rickif
Copy link
Author

rickif commented Jan 5, 2023

Make sure all outstanding objects are destroyed prior to calling close.

See https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#termination

Actually, what we use is the C++ wrapper RdKafka::KafkaConsumer. @edenhill

@robinfehr
Copy link

@rickif Our issue last time we faced similar issues on the node-rdkafka end (which also uses the C++ wrapper) was that we didn't process the last incoming rebalance anymore. Not sure this helps - but helped us.

@ryzhyk
Copy link

ryzhyk commented Sep 1, 2023

@rickif , have you been able to solve this issue? I ran into it using rdkafka bindings for Rust, except for me this is triggered by deleting the topic that the consumer is subscribed to right before deleting the consumer. The stack trace looks exactly the same. There are no live Kafka objects in the code except the consumer itself.

@ryzhyk
Copy link

ryzhyk commented Sep 1, 2023

Upon further investigation, this only happens when I either set enable.auto.offset.store to true or invoke rd_kafka_offset_store (with enable.auto.offset.store set to false).

ryzhyk pushed a commit to feldera/feldera that referenced this issue Sep 1, 2023
Change Kafka consumer configuration to _not_ automatically committ offsets of consumed
messages to the broker, meaning that next time the connector is instantiated it will
start reading from the offset specified in `auto.offset.reset` and not from the last
committed offset.  The previous behavioe caused `rdkafka` to hang in some circumstances
(confluentinc/librdkafka#3954).  Besides, the new behavior
is probably more correct given that circuit state currently does not survive across
pipeline restarts, so it makes sense to start feeding messages from the start rather
than from the last offset consumed by the previous instance of the pipeline, whose state
is lost.  Once we add fault tolerance, we will likely use explicit commits,
which also do not require these options.

Signed-off-by: Leonid Ryzhyk <leonid@feldera.com>
ryzhyk pushed a commit to feldera/feldera that referenced this issue Sep 1, 2023
Change Kafka consumer configuration to _not_ automatically committ offsets of consumed
messages to the broker, meaning that next time the connector is instantiated it will
start reading from the offset specified in `auto.offset.reset` and not from the last
committed offset.  The previous behavioe caused `rdkafka` to hang in some circumstances
(confluentinc/librdkafka#3954).  Besides, the new behavior
is probably more correct given that circuit state currently does not survive across
pipeline restarts, so it makes sense to start feeding messages from the start rather
than from the last offset consumed by the previous instance of the pipeline, whose state
is lost.  Once we add fault tolerance, we will likely use explicit commits,
which also do not require these options.

Signed-off-by: Leonid Ryzhyk <leonid@feldera.com>
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

No branches or pull requests

6 participants