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

Recursive locking on periodic refresh of leader-less partition #1311

Closed
9 tasks
lathakris opened this issue Jul 7, 2017 · 22 comments
Closed
9 tasks

Recursive locking on periodic refresh of leader-less partition #1311

lathakris opened this issue Jul 7, 2017 · 22 comments
Labels

Comments

@lathakris
Copy link

Description

The program asserted in the kafka library. here is the trace. Seems like a kafka produce also was happening during this time. 0.9.5 is the client library version used. Thank you.

[New LWP 330]
[Thread debugging using libthread_db enabled]
Using host libthread_db library "/lib/libthread_db.so.1".
Core was generated by /opt/shieldx/cc/bin/cc -t FE -m 19 -i 188.1.1.7/24 -k 169.254.0.4 -a 169.254.0.'. Program terminated with signal SIGABRT, Aborted. #0 0x00007f97c52c5c37 in __GI_raise (sig=sig@entry=6) at ../nptl/sysdeps/unix/sysv/linux/raise.c:56 56 ../nptl/sysdeps/unix/sysv/linux/raise.c: No such file or directory. (gdb) bt #0 0x00007f97c52c5c37 in __GI_raise (sig=sig@entry=6) at ../nptl/sysdeps/unix/sysv/linux/raise.c:56 #1 0x00007f97c52c9028 in __GI_abort () at abort.c:89 #2 0x00007f97c52bebf6 in __assert_fail_base (fmt=0x7f97c5413018 "%s%s%s:%u: %s%sAssertion %s' failed.\n%n", assertion=assertion@entry=0x7f97c6d96e4d "r == 0",
file=file@entry=0x7f97c6d96e3f "tinycthread.c", line=line@entry=1011, function=function@entry=0x7f97c6d96e80 <PRETTY_FUNCTION.6405> "rwlock_wrlock")
at assert.c:92
#3 0x00007f97c52beca2 in __GI___assert_fail (assertion=assertion@entry=0x7f97c6d96e4d "r == 0", file=file@entry=0x7f97c6d96e3f "tinycthread.c",
line=line@entry=1011, function=function@entry=0x7f97c6d96e80 <PRETTY_FUNCTION.6405> "rwlock_wrlock") at assert.c:101
#4 0x00007f97c6d6d4e6 in rwlock_wrlock (rwl=rwl@entry=0x10c6df0) at tinycthread.c:1011
#5 0x00007f97c6d7182b in rd_kafka_metadata_refresh_topics (rk=rk@entry=0x10c6a60, rkb=rkb@entry=0x0, topics=topics@entry=0x7f97c3881b10, force=force@entry=1,
reason=reason@entry=0x7f97c6d97612 "partition leader query") at rdkafka_metadata.c:723
#6 0x00007f97c6d71b88 in rd_kafka_metadata_leader_query_tmr_cb (rkts=0x10c7038, arg=) at rdkafka_metadata.c:952
#7 0x00007f97c6d2c488 in rd_kafka_timers_run (rkts=rkts@entry=0x10c7038, timeout_us=timeout_us@entry=0) at rdkafka_timer.c:251
#8 0x00007f97c6d0d927 in rd_kafka_thread_main (arg=arg@entry=0x10c6a60) at rdkafka.c:1170
#9 0x00007f97c6d6cee7 in _thrd_wrapper_function (aArg=) at tinycthread.c:624
#10 0x00007f97c62e3184 in start_thread (arg=0x7f97c3883700) at pthread_create.c:312
#11 0x00007f97c538cffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111
(gdb)

How to reproduce

Checklist

Please provide the following information:

  • librdkafka version (release number or git tag):
  • Apache Kafka version:
  • librdkafka client configuration:
  • Operating system:
  • Using the legacy Consumer
  • Using the high-level KafkaConsumer
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@lathakris
Copy link
Author

I have one thread doing the consumer in a queue and a couple of producer threads.

@edenhill
Copy link
Contributor

edenhill commented Jul 7, 2017

can you provide a thread appl all bt to list all threads and their backtraces?

@lathakris
Copy link
Author

I am so sorry someone accidentally deleted the core file!!

@lathakris
Copy link
Author

lathakris commented Jul 11, 2017

(gdb) bt
#0  0x00007f959f54ec37 in __GI_raise (sig=sig@entry=6) at ../nptl/sysdeps/unix/sysv/linux/raise.c:56
#1  0x00007f959f552028 in __GI_abort () at abort.c:89
#2  0x00007f959f547bf6 in __assert_fail_base (fmt=0x7f959f69c018 "%s%s%s:%u: %s%sAssertion `%s' failed.\n%n",
    assertion=assertion@entry=0x7f95a101fe4d "r == 0", file=file@entry=0x7f95a101fe3f "tinycthread.c",
    line=line@entry=1011, function=function@entry=0x7f95a101fe80 <__PRETTY_FUNCTION__.6405> "rwlock_wrlock") at assert.c:92
#3  0x00007f959f547ca2 in __GI___assert_fail (assertion=assertion@entry=0x7f95a101fe4d "r == 0",
    file=file@entry=0x7f95a101fe3f "tinycthread.c", line=line@entry=1011,
    function=function@entry=0x7f95a101fe80 <__PRETTY_FUNCTION__.6405> "rwlock_wrlock") at assert.c:101
#4  0x00007f95a0ff64e6 in rwlock_wrlock (rwl=rwl@entry=0x157fdd0) at tinycthread.c:1011
#5  0x00007f95a0ffa82b in rd_kafka_metadata_refresh_topics (rk=rk@entry=0x157fa40, rkb=rkb@entry=0x0,
    topics=topics@entry=0x7f959db0ab10, force=force@entry=1, reason=reason@entry=0x7f95a1020612 "partition leader query")
    at rdkafka_metadata.c:723
#6  0x00007f95a0ffab88 in rd_kafka_metadata_leader_query_tmr_cb (rkts=0x1580018, arg=<optimized out>)
    at rdkafka_metadata.c:952
#7  0x00007f95a0fb5488 in rd_kafka_timers_run (rkts=rkts@entry=0x1580018, timeout_us=timeout_us@entry=0)
    at rdkafka_timer.c:251
#8  0x00007f95a0f96927 in rd_kafka_thread_main (arg=arg@entry=0x157fa40) at rdkafka.c:1170
#9  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#10 0x00007f95a056c184 in start_thread (arg=0x7f959db0c700) at pthread_create.c:312
#11 0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111


(gdb) thread appl all bt

Thread 30 (Thread 0x7f958aff5700 (LWP 50)):
#0  0x00007f959f608c5d in poll () at ../sysdeps/unix/syscall-template.S:81
#1  0x00007f959f62679e in __poll_chk (fds=<optimized out>, nfds=<optimized out>, timeout=<optimized out>,
    fdslen=<optimized out>) at poll_chk.c:27
#2  0x00007f95a0fbd012 in poll (__timeout=1000, __nfds=<optimized out>, __fds=<optimized out>)
    at /usr/include/x86_64-linux-gnu/bits/poll2.h:41
#3  rd_kafka_transport_poll (rktrans=rktrans@entry=0x7f95740009f0, tmout=tmout@entry=1000) at rdkafka_transport.c:1268
#4  0x00007f95a0fbd09b in rd_kafka_transport_io_serve (rktrans=0x7f95740009f0, timeout_ms=1000) at rdkafka_transport.c:1127
#5  0x00007f95a0fabe8e in rd_kafka_broker_serve (rkb=rkb@entry=0x7f957c007ff0, timeout_ms=timeout_ms@entry=0)
    at rdkafka_broker.c:3224
#6  0x00007f95a0fad7dc in rd_kafka_broker_producer_serve (rkb=<optimized out>) at rdkafka_broker.c:3419
#7  rd_kafka_broker_thread_main (arg=arg@entry=0x7f957c007ff0) at rdkafka_broker.c:4690
#8  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#9  0x00007f95a056c184 in start_thread (arg=0x7f958aff5700) at pthread_create.c:312
#10 0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 29 (Thread 0x7f95a144f980 (LWP 15)):
#0  0x00007f959f60d8d3 in select () at ../sysdeps/unix/syscall-template.S:81
#1  0x000000000041294f in main (argc=9, argv=0x8) at /var/lib/jenkins/workspace/MasterDatapath/linecards/cc/main.c:618

Thread 28 (Thread 0x7f952cbff700 (LWP 55)):
#0  0x00007f959f60d8d3 in select () at ../sysdeps/unix/syscall-template.S:81
#1  0x0000000000415c48 in eventsPoll (eInfo=0x0) at /var/lib/jenkins/workspace/MasterDatapath/linecards/cc/eventsPoll.c:66
#2  0x00007f95a056c184 in start_thread (arg=0x7f952cbff700) at pthread_create.c:312
#3  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 27 (Thread 0x7f959d30b700 (LWP 31)):
#0  pthread_cond_timedwait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_timedwait.S:238
#1  0x00007f95a0ff6165 in cnd_timedwait_ms (cnd=cnd@entry=0x1580a38, mtx=mtx@entry=0x1580a10,
    timeout_ms=timeout_ms@entry=10) at tinycthread.c:501
#2  0x00007f95a0fc3a3e in rd_kafka_q_pop_serve (rkq=0x1580a10, timeout_ms=timeout_ms@entry=10, version=version@entry=0,
    cb_type=cb_type@entry=2, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:329
---Type <return> to continue, or q <return> to quit---
#3  0x00007f95a0fc3b80 in rd_kafka_q_pop (rkq=<optimized out>, timeout_ms=timeout_ms@entry=10, version=version@entry=0)
    at rdkafka_queue.c:360
#4  0x00007f95a0fabe78 in rd_kafka_broker_serve (rkb=rkb@entry=0x15802f0, timeout_ms=timeout_ms@entry=10)
    at rdkafka_broker.c:3219
#5  0x00007f95a0fac221 in rd_kafka_broker_ua_idle (rkb=rkb@entry=0x15802f0, timeout_ms=timeout_ms@entry=0)
    at rdkafka_broker.c:3269
#6  0x00007f95a0fac972 in rd_kafka_broker_thread_main (arg=arg@entry=0x15802f0) at rdkafka_broker.c:4688
#7  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#8  0x00007f95a056c184 in start_thread (arg=0x7f959d30b700) at pthread_create.c:312
#9  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 26 (Thread 0x7f959eb0e700 (LWP 28)):
#0  pthread_cond_timedwait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_timedwait.S:238
#1  0x00007f95a0ff6165 in cnd_timedwait_ms (cnd=cnd@entry=0x157eea8, mtx=mtx@entry=0x157ee80,
    timeout_ms=timeout_ms@entry=10) at tinycthread.c:501
#2  0x00007f95a0fc3a3e in rd_kafka_q_pop_serve (rkq=0x157ee80, timeout_ms=timeout_ms@entry=10, version=version@entry=0,
    cb_type=cb_type@entry=2, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:329
#3  0x00007f95a0fc3b80 in rd_kafka_q_pop (rkq=<optimized out>, timeout_ms=timeout_ms@entry=10, version=version@entry=0)
    at rdkafka_queue.c:360
#4  0x00007f95a0fabe78 in rd_kafka_broker_serve (rkb=rkb@entry=0x157e760, timeout_ms=timeout_ms@entry=10)
    at rdkafka_broker.c:3219
#5  0x00007f95a0fac221 in rd_kafka_broker_ua_idle (rkb=rkb@entry=0x157e760, timeout_ms=timeout_ms@entry=0)
    at rdkafka_broker.c:3269
#6  0x00007f95a0fac972 in rd_kafka_broker_thread_main (arg=arg@entry=0x157e760) at rdkafka_broker.c:4688
#7  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#8  0x00007f95a056c184 in start_thread (arg=0x7f959eb0e700) at pthread_create.c:312
#9  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 25 (Thread 0x7f9588ff1700 (LWP 54)):
#0  pthread_cond_timedwait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_timedwait.S:238
#1  0x00007f95a0ff6165 in cnd_timedwait_ms (cnd=cnd@entry=0x15602f8, mtx=mtx@entry=0x15602d0,
    timeout_ms=timeout_ms@entry=999) at tinycthread.c:501
#2  0x00007f95a0fc3a3e in rd_kafka_q_pop_serve (rkq=rkq@entry=0x15602d0, timeout_ms=999, version=version@entry=0,
---Type <return> to continue, or q <return> to quit---
    cb_type=cb_type@entry=2, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:329
#3  0x00007f95a0fc3b80 in rd_kafka_q_pop (rkq=rkq@entry=0x15602d0, timeout_ms=<optimized out>, version=version@entry=0)
    at rdkafka_queue.c:360
#4  0x00007f95a0f9b50c in rd_kafka_consume0 (rk=0x157d950, rkq=0x15602d0, timeout_ms=<optimized out>) at rdkafka.c:1837
#5  0x00000000004136ab in kafkaConsumer (tInfo=0x15602fc)
    at /var/lib/jenkins/workspace/MasterDatapath/linecards/cc/kafkaClient.c:403
#6  0x00007f95a056c184 in start_thread (arg=0x7f9588ff1700) at pthread_create.c:312
#7  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 24 (Thread 0x7f95897f2700 (LWP 53)):
#0  0x00007f959f5dcd4d in nanosleep () at ../sysdeps/unix/syscall-template.S:81
#1  0x00007f959f5dcbe4 in __sleep (seconds=0) at ../sysdeps/unix/sysv/linux/sleep.c:137
#2  0x00000000004152bf in statsPoll (sInfo=0x7f95897f09a0)
    at /var/lib/jenkins/workspace/MasterDatapath/linecards/cc/statsPoll.c:233
#3  0x00007f95a056c184 in start_thread (arg=0x7f95897f2700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 23 (Thread 0x7f9589ff3700 (LWP 52)):
#0  0x00007f959f616693 in epoll_wait () at ../sysdeps/unix/syscall-template.S:81
#1  0x0000000000726a54 in eal_intr_handle_interrupts (totalfds=<optimized out>, pfd=24)
    at /var/lib/jenkins/workspace/MasterDatapath/3rd/dpdk/src/lib/librte_eal/linuxapp/eal/eal_interrupts.c:770
#2  eal_intr_thread_main (arg=<optimized out>)
    at /var/lib/jenkins/workspace/MasterDatapath/3rd/dpdk/src/lib/librte_eal/linuxapp/eal/eal_interrupts.c:854
#3  0x00007f95a056c184 in start_thread (arg=0x7f9589ff3700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 22 (Thread 0x7f958a7f4700 (LWP 51)):
#0  0x00007f959f608c5d in poll () at ../sysdeps/unix/syscall-template.S:81
#1  0x00007f959f62679e in __poll_chk (fds=<optimized out>, nfds=<optimized out>, timeout=<optimized out>,
    fdslen=<optimized out>) at poll_chk.c:27
#2  0x00007f95a0fbd012 in poll (__timeout=1000, __nfds=<optimized out>, __fds=<optimized out>)
    at /usr/include/x86_64-linux-gnu/bits/poll2.h:41
#3  rd_kafka_transport_poll (rktrans=rktrans@entry=0x7f95800023b0, tmout=tmout@entry=1000) at rdkafka_transport.c:1268
---Type <return> to continue, or q <return> to quit---
#4  0x00007f95a0fbd09b in rd_kafka_transport_io_serve (rktrans=0x7f95800023b0, timeout_ms=1000) at rdkafka_transport.c:1127
#5  0x00007f95a0fabe8e in rd_kafka_broker_serve (rkb=rkb@entry=0x7f9584008200, timeout_ms=timeout_ms@entry=0)
    at rdkafka_broker.c:3224
#6  0x00007f95a0facb84 in rd_kafka_broker_consumer_serve (rkb=0x7f9584008200) at rdkafka_broker.c:4581
#7  rd_kafka_broker_thread_main (arg=arg@entry=0x7f9584008200) at rdkafka_broker.c:4692
#8  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#9  0x00007f95a056c184 in start_thread (arg=0x7f958a7f4700) at pthread_create.c:312
#10 0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 21 (Thread 0x7f958b7f6700 (LWP 49)):
#0  pthread_cond_timedwait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_timedwait.S:238
#1  0x00000000006da4d0 in as_cluster_tender (data=0x1560390) at src/main/aerospike/as_cluster.c:674
#2  0x00007f95a056c184 in start_thread (arg=0x7f958b7f6700) at pthread_create.c:312
#3  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 20 (Thread 0x7f958bff7700 (LWP 48)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958bff5c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958bff7700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 19 (Thread 0x7f958c7f8700 (LWP 47)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958c7f6c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958c7f8700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 18 (Thread 0x7f958cff9700 (LWP 46)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
---Type <return> to continue, or q <return> to quit---
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958cff7c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958cff9700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 17 (Thread 0x7f958d7fa700 (LWP 45)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958d7f8c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958d7fa700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 16 (Thread 0x7f958dffb700 (LWP 44)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958dff9c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958dffb700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 15 (Thread 0x7f958e7fc700 (LWP 43)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958e7fac90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958e7fc700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 14 (Thread 0x7f958effd700 (LWP 42)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958effbc90, ms_wait=ms_wait@entry=-1)
---Type <return> to continue, or q <return> to quit---
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958effd700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 13 (Thread 0x7f958f7fe700 (LWP 41)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958f7fcc90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958f7fe700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 12 (Thread 0x7f958ffff700 (LWP 40)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f958fffdc90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f958ffff700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 11 (Thread 0x7f9594ff9700 (LWP 39)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f9594ff7c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f9594ff9700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 10 (Thread 0x7f95957fa700 (LWP 38)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f95957f8c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
---Type <return> to continue, or q <return> to quit---
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f95957fa700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 9 (Thread 0x7f9595ffb700 (LWP 37)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f9595ff9c90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f9595ffb700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 8 (Thread 0x7f95967fc700 (LWP 36)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f95967fac90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f95967fc700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 7 (Thread 0x7f9596ffd700 (LWP 35)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f9596ffbc90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f9596ffd700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 6 (Thread 0x7f95977fe700 (LWP 34)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f95977fcc90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
---Type <return> to continue, or q <return> to quit---
#3  0x00007f95a056c184 in start_thread (arg=0x7f95977fe700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 5 (Thread 0x7f9597fff700 (LWP 33)):
#0  pthread_cond_wait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_wait.S:185
#1  0x00000000006f30d3 in cf_queue_pop (q=0x15819a0, buf=buf@entry=0x7f9597ffdc90, ms_wait=ms_wait@entry=-1)
    at src/main/citrusleaf/cf_queue.c:358
#2  0x00000000006ee2dc in as_thread_worker (data=0x15603f8) at src/main/aerospike/as_thread_pool.c:45
#3  0x00007f95a056c184 in start_thread (arg=0x7f9597fff700) at pthread_create.c:312
#4  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 4 (Thread 0x7f959cb0a700 (LWP 32)):
#0  pthread_cond_timedwait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_timedwait.S:238
#1  0x00007f95a0ff6165 in cnd_timedwait_ms (cnd=cnd@entry=0x15813b8, mtx=mtx@entry=0x1581390,
    timeout_ms=timeout_ms@entry=10) at tinycthread.c:501
#2  0x00007f95a0fc3a3e in rd_kafka_q_pop_serve (rkq=0x1581390, timeout_ms=timeout_ms@entry=10, version=version@entry=0,
    cb_type=cb_type@entry=2, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:329
#3  0x00007f95a0fc3b80 in rd_kafka_q_pop (rkq=<optimized out>, timeout_ms=timeout_ms@entry=10, version=version@entry=0)
    at rdkafka_queue.c:360
#4  0x00007f95a0fabe78 in rd_kafka_broker_serve (rkb=rkb@entry=0x1580c60, timeout_ms=timeout_ms@entry=10)
    at rdkafka_broker.c:3219
#5  0x00007f95a0fac221 in rd_kafka_broker_ua_idle (rkb=rkb@entry=0x1580c60, timeout_ms=timeout_ms@entry=0)
    at rdkafka_broker.c:3269
#6  0x00007f95a0fac972 in rd_kafka_broker_thread_main (arg=arg@entry=0x1580c60) at rdkafka_broker.c:4688
#7  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#8  0x00007f95a056c184 in start_thread (arg=0x7f959cb0a700) at pthread_create.c:312
#9  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 3 (Thread 0x7f959e30d700 (LWP 29)):
#0  0x00007f959f608c5d in poll () at ../sysdeps/unix/syscall-template.S:81
#1  0x00007f959f62679e in __poll_chk (fds=<optimized out>, nfds=<optimized out>, timeout=<optimized out>,
    fdslen=<optimized out>) at poll_chk.c:27
#2  0x00007f95a0fbd012 in poll (__timeout=1000, __nfds=<optimized out>, __fds=<optimized out>)
---Type <return> to continue, or q <return> to quit---
    at /usr/include/x86_64-linux-gnu/bits/poll2.h:41
#3  rd_kafka_transport_poll (rktrans=rktrans@entry=0x7f95900009c0, tmout=tmout@entry=1000) at rdkafka_transport.c:1268
#4  0x00007f95a0fbd09b in rd_kafka_transport_io_serve (rktrans=0x7f95900009c0, timeout_ms=1000) at rdkafka_transport.c:1127
#5  0x00007f95a0fabe8e in rd_kafka_broker_serve (rkb=rkb@entry=0x157f0d0, timeout_ms=timeout_ms@entry=10)
    at rdkafka_broker.c:3224
#6  0x00007f95a0fac221 in rd_kafka_broker_ua_idle (rkb=rkb@entry=0x157f0d0, timeout_ms=timeout_ms@entry=0)
    at rdkafka_broker.c:3269
#7  0x00007f95a0fac972 in rd_kafka_broker_thread_main (arg=arg@entry=0x157f0d0) at rdkafka_broker.c:4688
#8  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#9  0x00007f95a056c184 in start_thread (arg=0x7f959e30d700) at pthread_create.c:312
#10 0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 2 (Thread 0x7f959f30f700 (LWP 27)):
#0  pthread_cond_timedwait@@GLIBC_2.3.2 () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_cond_timedwait.S:238
#1  0x00007f95a0ff6165 in cnd_timedwait_ms (cnd=cnd@entry=0x15606d8, mtx=mtx@entry=0x15606b0, timeout_ms=<optimized out>)
    at tinycthread.c:501
#2  0x00007f95a0fc3ca5 in rd_kafka_q_serve (rkq=0x15606b0, timeout_ms=<optimized out>, max_cnt=max_cnt@entry=0,
    cb_type=cb_type@entry=1, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:406
#3  0x00007f95a0f9690c in rd_kafka_thread_main (arg=arg@entry=0x157d950) at rdkafka.c:1166
#4  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#5  0x00007f95a056c184 in start_thread (arg=0x7f959f30f700) at pthread_create.c:312
#6  0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111

Thread 1 (Thread 0x7f959db0c700 (LWP 30)):
#0  0x00007f959f54ec37 in __GI_raise (sig=sig@entry=6) at ../nptl/sysdeps/unix/sysv/linux/raise.c:56
#1  0x00007f959f552028 in __GI_abort () at abort.c:89
#2  0x00007f959f547bf6 in __assert_fail_base (fmt=0x7f959f69c018 "%s%s%s:%u: %s%sAssertion `%s' failed.\n%n",
    assertion=assertion@entry=0x7f95a101fe4d "r == 0", file=file@entry=0x7f95a101fe3f "tinycthread.c",
    line=line@entry=1011, function=function@entry=0x7f95a101fe80 <__PRETTY_FUNCTION__.6405> "rwlock_wrlock") at assert.c:92
#3  0x00007f959f547ca2 in __GI___assert_fail (assertion=assertion@entry=0x7f95a101fe4d "r == 0",
    file=file@entry=0x7f95a101fe3f "tinycthread.c", line=line@entry=1011,
    function=function@entry=0x7f95a101fe80 <__PRETTY_FUNCTION__.6405> "rwlock_wrlock") at assert.c:101
#4  0x00007f95a0ff64e6 in rwlock_wrlock (rwl=rwl@entry=0x157fdd0) at tinycthread.c:1011
---Type <return> to continue, or q <return> to quit---
#5  0x00007f95a0ffa82b in rd_kafka_metadata_refresh_topics (rk=rk@entry=0x157fa40, rkb=rkb@entry=0x0,
    topics=topics@entry=0x7f959db0ab10, force=force@entry=1, reason=reason@entry=0x7f95a1020612 "partition leader query")
    at rdkafka_metadata.c:723
#6  0x00007f95a0ffab88 in rd_kafka_metadata_leader_query_tmr_cb (rkts=0x1580018, arg=<optimized out>)
    at rdkafka_metadata.c:952
#7  0x00007f95a0fb5488 in rd_kafka_timers_run (rkts=rkts@entry=0x1580018, timeout_us=timeout_us@entry=0)
    at rdkafka_timer.c:251
#8  0x00007f95a0f96927 in rd_kafka_thread_main (arg=arg@entry=0x157fa40) at rdkafka.c:1170
#9  0x00007f95a0ff5ee7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:624
#10 0x00007f95a056c184 in start_thread (arg=0x7f959db0c700) at pthread_create.c:312
#11 0x00007f959f615ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111
(gdb)

@lathakris
Copy link
Author

happened again here is the bt for all threads. thx,

@lathakris
Copy link
Author

The scenario in which this happens, each time a new topic is created by the consumer and we produce to that new topic. The way I create a new topic handle is I destroy the old topic (rd_kafka_topic_destroy) and create a new one with rd_kafka_topic_new. After this call I use kafka_produce to send out the message on the new topic.

Is this the right way to handle dynamic topics ?

@edenhill
Copy link
Contributor

The rd_kafka_topic_t objects are refcounted and the application must keep the topic_new() and topic_destroy() calls symmetric, and not use the topic_t object after calling topic_destroy().

Also note that no objects (such as topic_t) may be shared between rd_kafka_t instances.

@lathakris
Copy link
Author

Yes I am following those rules.

I have about 9 producer topics and I initialize them. Out of the 9 one keeps changing, so I initialize it with a default topic name, and destroy and recreate it each time I get a request to produce on the new topic. I am seeing this issue only after adding this dynamic topic feature. It was fine all the while. I use one producer - rd_kafka_t and I have the following attributes for each producer topic.
{
rd_kafka_topic_t *rkt;
rd_kafka_topic_conf_t *topic_conf;
int32_t partition;
}prodInfo_t

and I create with the following

rd_kafka_topic_conf_new()
rd_kafka_topic_new()

and destroy and recreate as follows.

rd_kafka_topic_destroy()
rd_kafka_topic_conf_new()
rd_kafka_topic_new()

(I set the topic_conf to null after creation each time) Is this code path fine ? thx.

@edenhill
Copy link
Contributor

Topic objects are meant to be long-lived, could you try caching the topic object instead?

@lathakris
Copy link
Author

ok I did not find a way to change the topic name on a given object hence I did this approach ? Could you please suggest.
I had to use rd_kafka_topic_new() call to change the topic name hence had to destroy the old one.

@edenhill
Copy link
Contributor

Oh, right, a new topic name, then you'll need a new topic object too, as you say.

I'm not sure where to go from here, maybe you should try running the program through helgrind or valgrind?
Also check what the return value from pthread_rwlock_wrlock() is (r) is in that assert in your core file.

@lathakris
Copy link
Author

'r' is optimized out in that frame.. Let me see if we can keep static topics and change the design. I have never seen this before introducing dynamic topics!!

@lathakris
Copy link
Author

thanks for your help.

@edenhill
Copy link
Contributor

If you have the same client-side configuration for all topics you can use the default_topic_conf to set the default topic config, and then use the topic_t-less rd_kafka_producev() call that takes a topic string instead of a topic object.

The high-level KafkaConsumer does not make use of the topic_t object in its public APIs either.

@lathakris
Copy link
Author

ok this sounds like a good option. I was not aware of this API. Is it ok if I use this API - rd_kafka_producev() for the particular dynamic topic alone and use the standard rd_kafka-produce() for the other static topics (can happen in multiple threads like before). thank you.

@edenhill
Copy link
Contributor

Yes, perfectly fine, all it does is look up the underlying topic_t for you and dealing with its refcounts without you needing to

@lathakris
Copy link
Author

ok cool let me give this a try and see how it goes. I didn't want to change that old code which is already working. Thank you so much!

@lathakris
Copy link
Author

The issue is still happening even after using rd_kafka_producev. I have anyways changed the code to use rd_kafka_producev() to be on the safe side.

I traced the code with gdb and I think there might be an issue. The r is set to
errno of 35 /* Resource deadlock would occur */
Here is the issue.
File : rdkafka_metadata.c

'rd_kafka_metadata_leader_query_tmr_cb' calls 'rd_kafka_metadata_refresh_topics'.

'rd_kafka_metadata_refresh_topics' is trying to get the 'rd_kafka_wrlock(rk);'

but then the lock is already held in 'rd_kafka_metadata_leader_query_tmr_cb' before calling 'rd_kafka_metadata_refresh_topics'.
Looks like this lock/unlock in 'rd_kafka_metadata_leader_query_tmr_cb' is not needed.
The lock/unlock present in 'rd_kafka_metadata_refresh_topics' is good for all operations.

please check the code and let me know.

I am wondering how this is happening now, does new topics cause this rd_kafka_metadata_leader_query_tmr_cb and 'rd_kafka_metadata_refresh_topics(rk, NULL, &topics, 1/force/, "partition leader query");' to execute ??

Thank you.

@lathakris
Copy link
Author

So I removed the lock/unlock in 'rd_kafka_metadata_leader_query_tmr_cb' and itseems to work fine. Please let me know how to proceed. thank you.

@edenhill
Copy link
Contributor

You are absolutely right, good find, thank you!

I'll push a fix to master (and 0.11.0) soon.

@edenhill edenhill added this to the next feature milestone Jul 13, 2017
@edenhill edenhill added bug and removed wait-info labels Jul 13, 2017
@lathakris
Copy link
Author

Thank you. So I can upgrade to 0.11.0 from 0.9.5 when that happens ? I am currently using 0.9.5. For now I will patch the library myself using your code change. Thanks once again.

@edenhill
Copy link
Contributor

Yep

@edenhill edenhill changed the title assert in kafka library Recursive locking on periodic refresh of leader-less partition Oct 24, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

No branches or pull requests

2 participants