confluent-kafka-go: Crash on message retry enqueue
Description
We have large scale deployment of kafka producers using confluent-kafka-go and come across many occurrences of crashes associated with the following assertion failure:rd_kafka_assert(NULL, rkmq->rkmq_msg_cnt > 0) in rdkafka_msg.h. It appears to occur more often while we have under replicated partitions or while we perform partition reassignment.
The following stack trace has been found to be shared among all the crash occurrences:
Using host libthread_db library "/lib/x86_64-linux-gnu/libthread_db.so.1".
Program terminated with signal SIGABRT, Aborted.
#0 runtime.raise () at /usr/local/go/src/runtime/sys_linux_amd64.s:159
+------------------------------------------------------------------------------+
| backtrace |
+------------------------------------------------------------------------------+
#0 runtime.raise () at /usr/local/go/src/runtime/sys_linux_amd64.s:159
#1 0x0000000000458d82 in runtime.raisebadsignal (c=0x7f0fb77fbd98, sig=6) at /usr/local/go/src/runtime/signal_unix.go:491
#2 0x00000000004591a3 in runtime.badsignal (c=0x7f0fb77fbd98, sig=6) at /usr/local/go/src/runtime/signal_unix.go:600
#3 0x0000000000458988 in runtime.sigtrampgo (ctx=0x7f0fb77fbe40, info=0x7f0fb77fbf70, sig=6) at /usr/local/go/src/runtime/signal_unix.go:297
#4 0x0000000000471863 in runtime.sigtramp () at /usr/local/go/src/runtime/sys_linux_amd64.s:352
#5
#6 0x00007f0fdaecdc37 in __GI_raise (sig=sig@entry=6) at ../nptl/sysdeps/unix/sysv/linux/raise.c:56
#7 0x00007f0fdaed1028 in __GI_abort () at abort.c:89
#8 0x000000000066456a in rd_kafka_crash (file=file@entry=0x7c15c3 "rdkafka_msg.h", line=line@entry=263, function=function@entry=0x7c1980 <__FUNCTION__.19041> "rd_kafka_msgq_deq", rk=rk@entry=0x0, reason=reason@entry=0x7c1708 "assert: rkmq->rkmq_msg_cnt > 0") at rdkafka.c:3102
#9 0x000000000041a520 in rd_kafka_msgq_deq (do_count=, rkm=, rkmq=) at rdkafka_msg.h:263
#10 0x0000000000683e09 in rd_kafka_msgq_deq (do_count=, rkm=, rkmq=) at rdkafka_msg.h:264
#11 rd_kafka_msgq_age_scan (rkmq=0x1ba88, rkmq@entry=0x7f0ebc044a88, timedout=0x1bad5, timedout@entry=0x7f0fb77fc5f0, now=6, now@entry=2173403318391) at rdkafka_msg.c:577
#12 0x0000000000687875 in rd_kafka_topic_scan_all (rk=rk@entry=0x7f0fcc001400, now=2173403318391) at rdkafka_topic.c:1135
#13 0x0000000000662e56 in rd_kafka_topic_scan_tmr_cb (rkts=, arg=) at rdkafka.c:1194
#14 0x000000000068a448 in rd_kafka_timers_run (rkts=rkts@entry=0x7f0fcc001c18, timeout_us=timeout_us@entry=0) at rdkafka_timer.c:251
#15 0x000000000066c8d7 in rd_kafka_thread_main (arg=arg@entry=0x7f0fcc001400) at rdkafka.c:1270
#16 0x00000000006cdb97 in _thrd_wrapper_function (aArg=) at tinycthread.c:624
#17 0x00007f0fdb470184 in start_thread (arg=0x7f0fb77fe700) at pthread_create.c:312
#18 0x00007f0fdaf94ffd in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:111
How to reproduce
We tried and have not yet been successful reproducing in a lab environment. We’d like to know of any suggestions of changes in the librdkafka code such that it prints out things useful to diagnose this issue. Also, we briefly reviewed the later librdkafka commits and didn’t find anything that looks like it addresses this crash, but if we misidentified this, we would be happy to rebuild and retest with the lastest librdkafka commit in master.
Checklist
Please provide the following information:
- confluent-kafka-go and librdkafka version (
LibraryVersion()): confluent-kafka-go commit:1112e2c08a15bce669a99cc35284a690fd2086b8librdkafka:0.11.4-RC1B - Apache Kafka broker version: 0.11.0.1
- Client configuration:
ConfigMap{...}
"bootstrap.servers": brokerList,
"queue.buffering.max.ms": "500",
"compression.codec": "gzip",
"batch.num.messages": "256",
"security.protocol": "ssl",
"queue.buffering.max.messages": 8000,
"broker.version.fallback": "0.10.2.1",
"statistics.interval.ms": 60000,
- Operating system: ubuntu 14.04
- Provide client logs (with
"debug": ".."as necessary) We didn’t know witch debug setting would be useful for this so didn’t enable any debug flag. Please let us know if there is a debug flag we should enable. - Provide broker log excerpts
- Critical issue
About this issue
- Original URL
- State: closed
- Created 6 years ago
- Reactions: 1
- Comments: 39 (36 by maintainers)
@edenhill
Awesome, thank you!
I think I’ve found the (or a) issue: when a produce request fails with a retriable error the messages from the request are put back on the partition queue in the original order and the produce request is re-created. If multiple subsequent requests fail in short succession, such as when the partition leader changes which is the case in at least some of these crashes, the messages from the request are simply appended to the end of the queue. And this is where the bug lies: https://github.com/edenhill/librdkafka/blob/master/src/rdsysqueue.h#L251 The destination queue (i.e., rktp_xmit_msgq) tailq header’s
tqh_lastis not updated to point at the new last element, so the queue will be corrupt.I’ll make a fix soon.
I’m very impressed with the amount of information you’ve provided for this issue. ❤️
Ack, we’ve begun the process. Hopefully in a week or so, we’ll have the stack traces (and logs if you need them).
On Mon, Aug 13, 2018, 12:21 PM Magnus Edenhill notifications@github.com wrote: