You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
If the partitions move from one transactional producer to another. Consumer will make an OffsetFetch request, kafka broker will send UNSTABLE_OFFSETS exception if there are any pending transactions on the moved partition. As per kip-447, client should back-off and retry till UNSTABLE_OFFSETS are cleared due to transaction timeout.
However, librdkafka is just retrying twice and results in an assertion failure as shown below:
#0 0x00007f234d9ef277 in raise () from /lib64/libc.so.6
#1 0x00007f234d9f0968 in abort () from /lib64/libc.so.6
#2 0x0000000000687793 in Optiver::Application::AbortHandler () at ../../../x_common_libs/application/application.cc:443
#3 <signal handler called>
#4 0x00007f234d9ef277 in raise () from /lib64/libc.so.6
#5 0x00007f234d9f0968 in abort () from /lib64/libc.so.6
#6 0x00007f234d9e8096 in __assert_fail_base () from /lib64/libc.so.6
#7 0x00007f234d9e8142 in __assert_fail () from /lib64/libc.so.6
#8 0x00000000007a4bb4 in rd_kafka_consumer_err (rkq=0x10891a0, broker_id=broker_id@entry=-1, err=RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT, version=version@entry=0, topic=0x7f23381110b0 "SmashedAliens", rktp=rktp@entry=0x7f2338218f20, offset=offset@entry=-1001,
fmt=fmt@entry=0xa0d5b0 "Failed to fetch committed offset for group \"%s\" topic %s [%d]: %s") at rdkafka_op.c:497
#9 0x00000000007ea460 in rd_kafka_assignment_apply_offsets (err=RD_KAFKA_RESP_ERR_NO_ERROR, offsets=0x7f2338111670, rk=0x107c340) at rdkafka_assignment.c:158
#10 rd_kafka_assignment_handle_OffsetFetch (rk=0x107c340, rkb=<optimized out>, err=RD_KAFKA_RESP_ERR_NO_ERROR, reply=<optimized out>, request=<optimized out>, opaque=<optimized out>) at rdkafka_assignment.c:271
#11 0x000000000079c66b in rd_kafka_buf_callback (rk=<optimized out>, rkb=<optimized out>, err=<optimized out>, response=0x7f233c10b0f0, request=0x7f23382157d0) at rdkafka_buf.c:480
#12 0x00000000007a822b in rd_kafka_op_handle_std (rk=rk@entry=0x107c340, rkq=rkq@entry=0x7f234b2a72f0, rko=rko@entry=0x7f233c10ad90, cb_type=cb_type@entry=1) at rdkafka_op.c:819
#13 0x00000000007a82a8 in rd_kafka_op_handle (rk=0x107c340, rkq=0x7f234b2a72f0, rko=0x7f233c10ad90, cb_type=RD_KAFKA_Q_CB_CALLBACK, opaque=0x107c340, callback=0x768ae0 <rd_kafka_poll_cb>) at rdkafka_op.c:857
#14 0x00000000007a15c3 in rd_kafka_q_serve (rkq=0x10885e0, timeout_ms=<optimized out>, max_cnt=max_cnt@entry=0, cb_type=cb_type@entry=RD_KAFKA_Q_CB_CALLBACK, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:497
#15 0x0000000000766edc in rd_kafka_thread_main (arg=arg@entry=0x107c340) at rdkafka.c:2003
#16 0x00000000007f86c7 in _thrd_wrapper_function (aArg=<optimized out>) at tinycthread.c:576
#17 0x00007f234e9eee25 in start_thread () from /lib64/libpthread.so.0
#18 0x00007f234dab7bad in clone () from /lib64/libc.so.6
(gdb)
How to reproduce
startup multiple transactional producers (consumers) with same consumer group
SIGSTOP one of the producer, wait for a while (increase the transaction.time) so that transaction is kept pending for a longer duration
Checklist
Please provide the following information:
librdkafka version (release number or git tag): 1.6.0
Apache Kafka version: 2.5
librdkafka client configuration: `transaction timeout 5 mins
Operating system: Centos 7
Provide logs (with debug=.. as necessary) from librdkafka
Provide broker log excerpts
Critical issue
The text was updated successfully, but these errors were encountered:
The consumer will now retry indefinitely (or until the assignment is changed)
to retrieve committed offsets. This fixes the issue where only two retries
were attempted when outstanding transactions were blocking OffsetFetch
requests with `ERR_UNSTABLE_OFFSET_COMMIT`.
The consumer will now retry indefinitely (or until the assignment is changed)
to retrieve committed offsets. This fixes the issue where only two retries
were attempted when outstanding transactions were blocking OffsetFetch
requests with `ERR_UNSTABLE_OFFSET_COMMIT`.
Description
If the partitions move from one transactional producer to another. Consumer will make an OffsetFetch request, kafka broker will send UNSTABLE_OFFSETS exception if there are any pending transactions on the moved partition. As per kip-447, client should back-off and retry till UNSTABLE_OFFSETS are cleared due to transaction timeout.
However, librdkafka is just retrying twice and results in an assertion failure as shown below:
How to reproduce
Checklist
Please provide the following information:
1.6.0
2.5
Centos 7
debug=..
as necessary) from librdkafkaThe text was updated successfully, but these errors were encountered: