Skip to content

rd_kafka_consumer_close gets stuck when polling events from consumer queue #2077

@mfontanini

Description

Description

Using rd_kafka_queue_poll on a consumer handle's consumer queue (the one returned by rd_kafka_queue_get_consumer) seems to lead to the application hanging when invoking rd_kafka_consumer_close on the handle.

As far as I can tell, I'm not doing anything funky, at least given what the documentation for each of these calls states.

How to reproduce

I can't post rdkafka code that reproduces this but using cppkafka:

// Create a consumer and subscribe to the topic
Configuration config = {
    { "metadata.broker.list", "127.0.0.1:6667" },
    { "enable.auto.commit", false },
    { "group.id", "foo" }
}
Consumer consumer(config);
consumer.subscribe({ "some_topic" });

// This calls rd_kafka_queue_get_consumer
Queue queue = consumer.get_consumer_queue();
// This calls rd_kafka_queue_poll
const Event event = queue.next_event();

// At this point, the queue handle will be destroyed via rd_kafka_queue_destroy
// After that, the Consumer's destructor is called, which calls rd_kafka_consumer_close
// before destroying the handle

The call to rd_kafka_consumer_close gets stuck forever (I waited 2 minutes and it was still blocked). It seems like this is purposely spinning forever right here. This doesn't seem to be caused by the queue handle being destroyed. Both destroying it and not destroying it cause the same behavior.

Stack trace:

#1  0x00007ffff787ecd9 in cnd_wait (cond=<optimized out>, mtx=<optimized out>) at tinycthread.c:449
#2  0x00007ffff787f105 in cnd_timedwait_abs (cnd=cnd@entry=0xa4f418, mtx=mtx@entry=0xa4f3f0, 
    tspec=tspec@entry=0x7fffffffd280) at tinycthread_extra.c:96
#3  0x00007ffff784a481 in rd_kafka_q_pop_serve (rkq=rkq@entry=0xa4f3f0, timeout_ms=timeout_ms@entry=-1, 
    version=version@entry=0, cb_type=cb_type@entry=RD_KAFKA_Q_CB_RETURN, callback=callback@entry=0x0, 
    opaque=opaque@entry=0x0) at rdkafka_queue.c:373
#4  0x00007ffff784a550 in rd_kafka_q_pop (rkq=rkq@entry=0xa4f3f0, timeout_ms=timeout_ms@entry=-1, 
    version=version@entry=0) at rdkafka_queue.c:399
#5  0x00007ffff7823186 in rd_kafka_consumer_close (rk=0xa4c440) at rdkafka.c:2384
#6  0x00007ffff7baae4a in cppkafka::Consumer::close() ()
   from /home/matias/Projects/cppkafka/build/lib/libcppkafka.so.0.2
#7  0x00007ffff7ba9bf6 in cppkafka::Consumer::~Consumer() ()

It looks like the consumer is stuck waiting for a rebalance. Full logs are here. I removed all the repeated lines at the end, which were all the same as this one:

%7|1540656060.028|CGRPTERM|rdkafka#consumer-1| [thrd:main]: Group "5bd48b909dd088cd52ced29d": waiting for rebalance_cb, 0 toppar(s), 0 unassignment(s), 0 commit(s) (state up, join-state wait-assign-rebalance_cb) before terminating

Checklist

Please provide the following information:

  • librdkafka version (release number or git tag): v0.11.6
  • Apache Kafka version: Both 0.10.2 and 2.12 show the same behavior.
  • librdkafka client configuration: Posted above, nothing special here.
  • Operating system: Linux Mint 18.3
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue

Metadata

Metadata

Assignees

No one assigned

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions