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

rkb->rkb_fetching > 0 error #84

Closed
winbatch opened this issue Feb 21, 2014 · 12 comments
Closed

rkb->rkb_fetching > 0 error #84

winbatch opened this issue Feb 21, 2014 · 12 comments
Assignees
Labels

Comments

@winbatch
Copy link

Not sure what this means, but it doesn't look good ;)

Got this while doing a simple consumer.

rdkafka_broker.c:2800: rd_kafka_broker_fetch_reply: Assertion `rkb->rkb_fetching > 0' failed.
Aborted

@edenhill
Copy link
Contributor

Is this reproducible?
If so, can you reproduce with debug="msg,topic,broker" enabled?

2014-02-21 10:48 GMT+07:00 winbatch notifications@github.com:

Not sure what this means, but it doesn't look good ;)

Got this while doing a simple consumer.

rdkafka_broker.c:2800: rd_kafka_broker_fetch_reply: Assertion
`rkb->rkb_fetching > 0' failed.
Aborted

Reply to this email directly or view it on GitHubhttps://github.com//issues/84
.

@winbatch
Copy link
Author

only happened once, we'll see if I can reproduce it. What does
rkb_fetching > 0 mean?

On Thu, Feb 20, 2014 at 11:08 PM, Magnus Edenhill
notifications@github.comwrote:

Is this reproducible?
If so, can you reproduce with debug="msg,topic,broker" enabled?

2014-02-21 10:48 GMT+07:00 winbatch notifications@github.com:

Not sure what this means, but it doesn't look good ;)

Got this while doing a simple consumer.

rdkafka_broker.c:2800: rd_kafka_broker_fetch_reply: Assertion
`rkb->rkb_fetching > 0' failed.
Aborted

Reply to this email directly or view it on GitHub<
https://github.com/edenhill/librdkafka/issues/84>
.

Reply to this email directly or view it on GitHubhttps://github.com//issues/84#issuecomment-35697540
.

@edenhill
Copy link
Contributor

Each broker thread must only have one outstanding Fetch request (containing
all consuming and underflow toppars for that broker).
The reply handler asserts that there was in fact an outstanding Fetch
request, but in this case there wasnt, which is very weird.

How many topics and partitions were you consuming from?
Was this close crash to the start or stop of a consumer?

2014-02-21 11:13 GMT+07:00 winbatch notifications@github.com:

only happened once, we'll see if I can reproduce it. What does
rkb_fetching > 0 mean?

On Thu, Feb 20, 2014 at 11:08 PM, Magnus Edenhill
notifications@github.comwrote:

Is this reproducible?
If so, can you reproduce with debug="msg,topic,broker" enabled?

2014-02-21 10:48 GMT+07:00 winbatch notifications@github.com:

Not sure what this means, but it doesn't look good ;)

Got this while doing a simple consumer.

rdkafka_broker.c:2800: rd_kafka_broker_fetch_reply: Assertion
`rkb->rkb_fetching > 0' failed.
Aborted

Reply to this email directly or view it on GitHub<
https://github.com/edenhill/librdkafka/issues/84>
.

Reply to this email directly or view it on GitHub<
https://github.com/edenhill/librdkafka/issues/84#issuecomment-35697540>

.

Reply to this email directly or view it on GitHubhttps://github.com//issues/84#issuecomment-35697751
.

@winbatch
Copy link
Author

I believe it was a 10 partition topic. Note that I am starting 10 'loops'
and consuming from all. Not sure what you mean by the 'start' or 'stop'.
Do you mean start or stop of the partition offsets? Or did you mean was
it running for a while before it crashed? If so, yes, it was running for a
while.

BTW, if you think it's more likely to happen with more partitions - I happen to have a topic with 500 partitions if you want me to try that.

On Thu, Feb 20, 2014 at 11:22 PM, Magnus Edenhill
notifications@github.comwrote:

Each broker thread must only have one outstanding Fetch request (containing
all consuming and underflow toppars for that broker).
The reply handler asserts that there was in fact an outstanding Fetch
request, but in this case there wasnt, which is very weird.

How many topics and partitions were you consuming from?
Was this close crash to the start or stop of a consumer?

2014-02-21 11:13 GMT+07:00 winbatch notifications@github.com:

only happened once, we'll see if I can reproduce it. What does
rkb_fetching > 0 mean?

On Thu, Feb 20, 2014 at 11:08 PM, Magnus Edenhill
notifications@github.comwrote:

Is this reproducible?
If so, can you reproduce with debug="msg,topic,broker" enabled?

2014-02-21 10:48 GMT+07:00 winbatch notifications@github.com:

Not sure what this means, but it doesn't look good ;)

Got this while doing a simple consumer.

rdkafka_broker.c:2800: rd_kafka_broker_fetch_reply: Assertion
`rkb->rkb_fetching > 0' failed.
Aborted

Reply to this email directly or view it on GitHub<
https://github.com/edenhill/librdkafka/issues/84>
.

Reply to this email directly or view it on GitHub<
https://github.com/edenhill/librdkafka/issues/84#issuecomment-35697540>

.

Reply to this email directly or view it on GitHub<
https://github.com/edenhill/librdkafka/issues/84#issuecomment-35697751>
.

Reply to this email directly or view it on GitHubhttps://github.com//issues/84#issuecomment-35698056
.

@edenhill
Copy link
Contributor

500! now we're talking, yes, please try it on the fortune500 topic with the debug enabled (there will be a lot of output...)

By start and stop I mean close (in time) to the rd_kafka_consume_start|stop calls, but it seems this wasnt the case here.

@winbatch
Copy link
Author

certainly not near the stop, since I don't call stop - this is to run
forever. The only reason it stopped is because it crashed ;)

I'll see if I can reproduce it and get some debug logs.

On Thu, Feb 20, 2014 at 11:31 PM, Magnus Edenhill
notifications@github.comwrote:

500! now we're talking, yes, please try it on the fortune500 topic with
the debug enabled (there will be a lot of output...)

By start and stop I mean close (in time) to the
rd_kafka_consume_start|stop calls, but it seems this wasnt the case here.

Reply to this email directly or view it on GitHubhttps://github.com//issues/84#issuecomment-35698396
.

@winbatch
Copy link
Author

You know, it occurred to me. If you're taking the bold step of aborting the program with the assert, couldn't you print out your debug information (that 'rk_dump' function) right before the abort? I jknow it wouldn't have the entire history, but might give you what you need?

@edenhill
Copy link
Contributor

Thats a very good idea.

@winbatch
Copy link
Author

(please don't tell me you're going to post the above in a loop once for each partition? ;) )

@edenhill edenhill self-assigned this Mar 14, 2014
edenhill added a commit that referenced this issue Mar 19, 2014
This assert version calls rd_kafka_dump() (if an rk is available)
thus providing some more information before abort():ing.

This is winbatch's idea from issue #84
@edenhill
Copy link
Contributor

Did you have a chance to repro this on your 500 topic cluster?

@FrankVanYoung
Copy link

I also find this problem and it is reproducible. Kafka server has 4 topics and the topics from which our program read messages has 3 partitions. Here is the stack information:
(gdb) bt full
#0 0x00000030d1230285 in raise () from /lib64/libc.so.6
No symbol table info available.
#1 0x00000030d1231d30 in abort () from /lib64/libc.so.6
No symbol table info available.
#2 0x00000030d1229706 in assert_fail () from /lib64/libc.so.6
No symbol table info available.
#3 0x00000000004144db in rd_kafka_broker_fetch_reply (rkb=0x1112cc90, err=RD_KAFKA_RESP_ERR__MSG_TIMED_OUT, reply=0x0,
request=0x2aaab6239ae0, opaque=0x80) at rdkafka_broker.c:2800
__PRETTY_FUNCTION
= "rd_kafka_broker_fetch_reply"
#4 0x000000000040d495 in rd_kafka_broker_waitresp_timeout_scan (rkb=0x1112cc90, now=19111670806464) at rdkafka_broker.c:287
rkbuf = 0x2aaab6239ae0
tmp = 0x0
cnt = 2
PRETTY_FUNCTION = "rd_kafka_broker_waitresp_timeout_scan"
#5 0x00000000004125bc in rd_kafka_broker_consumer_serve (arg=) at rdkafka_broker.c:3224
cnt = 0
now = 19111670806464
PRETTY_FUNCTION = "rd_kafka_broker_consumer_serve"
#6 rd_kafka_broker_thread_main (arg=) at rdkafka_broker.c:3272
rkb = 0x1112cc90
rk = 0x11125950
#7 0x00000030d1e0677d in start_thread () from /lib64/libpthread.so.0
No symbol table info available.
#8 0x00000030d12d325d in clone () from /lib64/libc.so.6
No symbol table info available.

@edenhill
Copy link
Contributor

Thanks!

The crash was caused by retrying timed out Fetch requests, reproducible by bringing down the broker and starting it again.

Now fixed in master.
Please verify on your end.

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

3 participants