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

Consumption underflow error #2090

Closed
4 of 7 tasks
breischl opened this issue Nov 5, 2018 · 11 comments
Closed
4 of 7 tasks

Consumption underflow error #2090

breischl opened this issue Nov 5, 2018 · 11 comments

Comments

@breischl
Copy link

breischl commented Nov 5, 2018

Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ

Description

I have run into two partitions on a particular server that only consume to a certain point, and then stop. Other partitions on the same topic consume fine. The problem manifests in kafkacat v1.3.1 (using librdkafka v0.11.4) and also the Confluent .NET consumer v0.11.6. It does not happen in Java consumers, hence why I am reporting it here.

How to reproduce

The problem is reproducible on two particular partitions of a topic in one environment, but nowhere else that I've found. I'm not sure how those partitions got into that state.

The attached logs are produced via kafkacat because I could get full debug logging from it more easily.

$ kafkacat -V
kafkacat - Apache Kafka producer and consumer tool
https://github.com/edenhill/kafkacat
Copyright (c) 2014-2015, Magnus Edenhill
Version 1.3.1 (JSON) (librdkafka 0.11.4 builtin.features=gzip,snappy,ssl,sasl,regex,lz4,sasl_gssapi,sasl_plain,sasl_scram,plugins)

$ kafkacat -C -b mykafka.server.url:9092 -o 234611 -f '%o\n' -t myTopic -p 3 -d all > kcat.offsets.log 2>kcat.log

Full logs in kcat.log.zip, but interesting excerpt:

7|1541436064.078|RECV|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Received FetchResponse (v4, 1048644 bytes, CorrId 4, rtt 328.19ms)
%7|1541436064.078|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Topic myTopic [3] MessageSet size 1048576, error "Success", MaxOffset 324159, Ver 2/2

## Note underflow here
%7|1541436064.078|PROTOUFLOW|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Protocol read buffer underflow at 24886/1048644 (rd_kafka_msgset_reader_msg_v0_1:553): expected 18446744071575537664 bytes > 1023758 remaining bytes (truncated response from broker (ok))
%7|1541436064.078|CONSUME|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Enqueue 92 message(s) (92 ops) on myTopic [3] fetch queue (qlen 92, v2, last_offset 237428)
%7|1541436064.078|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Fetch topic myTopic [3] at offset 237429 (v2)
%7|1541436064.078|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Fetch 1/1/10 toppar(s)
%7|1541436064.079|SEND|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Sent FetchRequest (v4, 88 bytes @ 0, CorrId 5)
%7|1541436064.307|RECV|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Received FetchResponse (v4, 1048644 bytes, CorrId 5, rtt 228.27ms)
%7|1541436064.307|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Topic myTopic [3] MessageSet size 1048576, error "Success", MaxOffset 324159, Ver 2/2

## Another underflow
%7|1541436064.308|PROTOUFLOW|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Protocol read buffer underflow at 16748/16923 (rd_kafka_msgset_reader_msg_v2:674): expected 18446744073709551611 bytes > 175 remaining bytes (truncated response from broker (ok))
%7|1541436064.308|CONSUME|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Enqueue 57 message(s) (57 ops) on myTopic [3] fetch queue (qlen 57, v2, last_offset 243804)

## This offset is way past the end of the partition
%7|1541436064.308|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Fetch topic myTopic [3] at offset 6205542 (v2)
%7|1541436064.308|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Fetch 1/1/10 toppar(s)
%7|1541436064.308|SEND|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Sent FetchRequest (v4, 88 bytes @ 0, CorrId 6)
%7|1541436064.343|RECV|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Received FetchResponse (v4, 68 bytes, CorrId 6, rtt 34.98ms)
%7|1541436064.343|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Topic myTopic [3] MessageSet size 0, error "Broker: Offset out of range", MaxOffset -1, Ver 2/2
%7|1541436064.343|BACKOFF|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: myTopic [3]: Fetch backoff for 500ms: Broker: Offset out of range
%7|1541436064.343|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition myTopic [3] changed fetch state active -> offset-query
%7|1541436064.343|OFFSET|rdkafka#consumer-1| [thrd:main]: myTopic [3]: offset reset (at offset 6205542) to END: Broker: Offset out of range: Broker: Offset out of range
%7|1541436064.343|OFFREQ|rdkafka#consumer-1| [thrd:main]: mykafka.server.url:9092/100: Partition myTopic [3]: querying for logical offset END (opv 2)
%7|1541436064.343|OFFSET|rdkafka#consumer-1| [thrd:main]: mykafka.server.url:9092/100: OffsetRequest (v0, opv 0) for 1 topic(s) and 1 partition(s)
%7|1541436064.343|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition myTopic [3] changed fetch state offset-query -> offset-wait
%7|1541436064.343|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Topic myTopic [3] in state offset-wait at offset 234611 (0/100000 msgs, 0/1048576 kb queued, opv 2) is not fetchable: not in active fetch state
%7|1541436064.343|FETCHADD|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Removed myTopic [3] from fetch list (0 entries, opv 2)
%7|1541436064.343|SEND|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Sent OffsetRequest (v0, 75 bytes @ 0, CorrId 7)
%7|1541436064.374|RECV|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Received OffsetResponse (v0, 52 bytes, CorrId 7, rtt 31.32ms)
%7|1541436064.374|OFFSET|rdkafka#consumer-1| [thrd:main]: mykafka.server.url:9092/100: Offset reply for topic myTopic [3] (v2 vs v2)
%7|1541436064.375|OFFSET|rdkafka#consumer-1| [thrd:main]: Offset END request for myTopic [3] returned offset 324159 (324159)
%7|1541436064.375|PARTSTATE|rdkafka#consumer-1| [thrd:main]: Partition myTopic [3] changed fetch state offset-wait -> active

## Skipped over many messages, now consuming from end of topic
%7|1541436064.375|FETCH|rdkafka#consumer-1| [thrd:main]: Partition myTopic [3] start fetching at offset 324159
%7|1541436064.375|WAKEUP|rdkafka#consumer-1| [thrd:main]: mykafka.server.url:9092/100: Wake-up

kcat.offsets.log shows the offsets of consumed messages starting from 234611 as specified in the initial command, and ending at 243804, which is presumably where we hit the underflow.

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

  • librdkafka version (release number or git tag): 0.11.4 and 0.11.6
  • Apache Kafka version: Confluent 4.1.2, => Kafka 1.1.1`
  • librdkafka client configuration: <REPLACE with e.g., message.timeout.ms=123, auto.reset.offset=earliest, ..>
  • Operating system: kafkacat on MacOS 10.12.6, .NET on Windows Server 2008 R2 Standard
  • Provide logs (with debug=.. as necessary) from librdkafka (via kafkacat)
  • Provide broker log excerpts
  • Critical issue
@edenhill
Copy link
Contributor

edenhill commented Nov 5, 2018

Thanks for a great report!

These two lines are very interesting:

%7|1541436064.078|PROTOUFLOW|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Protocol read buffer underflow at 24886/1048644 (rd_kafka_msgset_reader_msg_v0_1:553): expected 18446744071575537664 bytes > 1023758 remaining bytes (truncated response from broker (ok))

%7|1541436064.308|PROTOUFLOW|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Protocol read buffer underflow at 16748/16923 (rd_kafka_msgset_reader_msg_v2:674): expected 18446744073709551611 bytes > 175 remaining bytes (truncated response from broker (ok))

Underflow (e.g., short responses) are expected due to the way the broker uses sendfile.
But what we see here is something different, the bytes the client interpretes within the response are either wrong, or there is a client bug.

Both these lines are while parsing the Message Key.

Could you try enabling crc checks? check.crcs=true in the client configuration.

Do you know what client implementation (e.g., Kafka's official Java client) and version that was used to produce these messages?

What is the broker log.message.format.version property set to for this topic?

@breischl
Copy link
Author

breischl commented Nov 5, 2018

Adding check.crcs=true to kafkacat ended with:

%7|1541451268.429|PROTOUFLOW|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Protocol read buffer underflow at 1048422/1048644 (rd_kafka_msgset_reader_msg_v0_1:492): expected 228 bytes > 222 remaining bytes (truncated response from broker (ok))
%7|1541451268.429|CONSUME|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Enqueue 4321 message(s) (4377 ops) on myTopic [3] fetch queue (qlen 4377, v2, last_offset 263864)
%7|1541451268.429|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Fetch topic myTopic [3] at offset 263865 (v2)
%7|1541451268.429|FETCH|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Fetch 1/1/10 toppar(s)
%7|1541451268.429|SEND|rdkafka#consumer-1| [thrd:mykafka.server.url:9092/bootstrap]: mykafka.server.url:9092/100: Sent FetchRequest (v4, 88 bytes @ 0, CorrId 5)
% ERROR: Topic myTopic [3] error: Message at offset 237524 (295 bytes) failed CRC32 check (original 0x32 != calculated 0x29d03092)

I can paste the entire log if needed, I'm just being lazy about scrubbing all the internal data from it.

The messages were produced using KStreams v0.11.0.3, which I believe uses the same version of Kafka producer under the covers.

We are currently using log.message.format.version 0.10.1. We did recently upgrade from Confluent 3.1.1 to 4.1.2, but I think the earlier version would've used the same log.message.format.version as we didn't set it explicitly and the underlying Kafka version was 0.10.1.

@edenhill
Copy link
Contributor

I'm suspecting this is an issue with KStreams producing transaction control messages to a 0.10.1-format topic.

@breischl
Copy link
Author

Ok. Do you think I should report it over there instead?

For posterity, I did manage to work around this in my code by setting check.crcs=true, and then basically just ignoring the errors so that we'd read past them. We don't get the affected messages, but in this particular case that's better than dying entirely.

@edenhill
Copy link
Contributor

What is your KStreams config? Specifically, what is processing.guarantee set to?

@breischl
Copy link
Author

We do not set processing.guarantee, so it should be using the default value of at_least_once.

@edenhill
Copy link
Contributor

Okay, then it is probably not an issue with transactional messages.

Are these, from the client's perspective, badly formatted messages a one time thing or do they keep occurring in these partitions?
Would it be possible to get a tcpdump packet capture of the consumer consuming these messages? (along with a debug=fetch,msg,queue log).

@breischl
Copy link
Author

Unfortunately I can't give you a packet capture - our corporate security people would have my head. Is there anything less detailed that would help?

@edenhill
Copy link
Contributor

edenhill commented Nov 15, 2018

What I need is a binary dump of the FetchResponse to understand how it is encoded.
I understand there are security issues with sharing this information, can you please reach out to me directly at magnus (at) confluent.io to figure something out.

@breischl
Copy link
Author

breischl commented Nov 19, 2018

We have a support contract with Confluent - they saw this and reached out on a separate channel. Since we have a contract with them, I was able to give them a full tcpdump. I believe they were going to coordinate with you on this, but if not please ping me or this thread again and I'll try to help.

@edenhill
Copy link
Contributor

Perfect, we'll continue the discussion there instead.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

2 participants