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

Receive failed: Invalid response size 100000039 (0..100000000): increase receive.message.max.bytes #1616

Closed
3 of 7 tasks
du-do opened this issue Jan 3, 2018 · 20 comments
Closed
3 of 7 tasks

Comments

@du-do
Copy link

du-do commented Jan 3, 2018

Description

librdkafka v0.11.1
kafka broker v0.10.1.0
ReadHat 5 (x64)
consumer config: receive.message.max.bytes=100000000
message size : 5242880(5M)

I have three consumers, they are all getting messages from one broker. My consumers is slow, but the producer is fast. So over time, the messages will pile up on the broker.
Then i get this from the consumer all the time, and my consumer can not receive messages any more:
%3|1514941148.962|FAIL|rdkafka#consumer-1| [thrd:172.20.51.38:9092/bootstrap]: 172.20.51.38:9092/1: Receive failed: Invalid response size 100000039 (0..100000000): increase receive.message.max.bytes

I know that messaging should be balanced. However, the accumulation of messages led to the consumer failed to receive I think it is a problem.

I think broker send messages using receive.message.max.bytes ,But I do not understand why 39 bytes are added. Maybe it is the length of the head or something else. When i increase receive.message.max.bytes to 200000000, the problem is still not solved. Invalid response size is still 39 bytes more than my receive.message.max.bytes.

%3|1514947746.873|FAIL|rdkafka#consumer-1| [thrd:172.20.51.38:9092/bootstrap]: 172.20.51.38:9092/1: Receive failed: Invalid response size 200000039 (0..200000000): increase receive.message.max.bytes %3|1514947746.873|ERROR|rdkafka#consumer-1| [thrd:172.20.51.38:9092/bootstrap]: 172.20.51.38:9092/1: Receive failed: Invalid response size 200000039 (0..200000000): increase receive.message.max.bytes
I think this may be related to internal judgment.

When i use another topic, i found the Invalid response size changed too.It became 100000038!
3|1514873845.371|FAIL|rdkafka#consumer-1| [thrd:172.20.51.39:9092/bootstrap]: 172.20.51.39:9092/2: Receive failed: Invalid response size 100000038 (0..100000000): increase receive.message.max.bytes

So i believe Invalid response size = receive.message.max.bytes + 36 + length of the topic
Should this length be deducted when judging?
I am not sure whether my guess is correct or not. How to deal with this imbalance in sending and receiving information?

How to reproduce

Just make sure the producer generates messages faster than the consumer consumes

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.1
  • Apache Kafka version: <0.10.1.0>
  • librdkafka client configuration: <receive.message.max.bytes=100000000,fetch.message.max.bytes=100000000>
  • Operating system: <ReadHat 5 (x64)>
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@edenhill
Copy link
Contributor

edenhill commented Jan 3, 2018

When you increased receive.message.max.bytes to 2M, did you also increase fetch.message.max.bytes?

@du-do
Copy link
Author

du-do commented Jan 4, 2018

yeah,i increased receive.message.max.bytes and fetch.message.max.bytes to 200M.
I tried again, set receive.message.max.bytes=200M and fetch.message.max.bytes=100M. Then the problem is solved. Thank you very much. @edenhill

@du-do du-do closed this as completed Jan 4, 2018
@sheeysong
Copy link

sheeysong commented Feb 12, 2018

I encountered the same error again lately during the load test, I reported the same issue last Oct. This time, we log if message size is bigger than 1MB on the Golang-Producer side. Producer claimed never send anything bigger than 1MB per log, so the messages traveled from different DC to our Core DC-Kafka-Cluster by MirrorMaker, and our Golang-Consumer-Client threw this error.
However, the broker CPU only use 1/3, Topic leader skewed a bit, what can go wrong here?

%3|1518222481.951|ERROR|rdkafka#consumer-7| [thrd:ps6576.prn.XXX.com:9092/1]: ps6576.prn.XXX.com:9092/1: Receive failed: Invalid response size 1000000206 (0..1000000000): increase receive.message.max.bytes

@edenhill
Copy link
Contributor

@sheeysong
The overshoot is due to protocol overhead, but I suspect there's an issue in librdkafka's checking of maximum size that does not take the overhead into calculations, I will look into it.

@sheeysong
Copy link

Hi @edenhill ,
I updated my original issue : confluentinc/confluent-kafka-go#100
thanks for helping!

@sheeysong
Copy link

Hi @edenhill
My version: librdkafka-dev_0.11.0

@natb1
Copy link

natb1 commented Mar 12, 2018

I'm having the same issue and it doesn't seem to matter how high I set receive.message.max.bytes or how low/high I set fetch.message.max.bytes I always get the error Receive failed: Invalid response size ... increase receive.message.max.bytes with the response size being ~500 bytes > receive.message.max.bytes.

For example, I have tried setting receive.message.max.bytes to the max 1000000000. Then my understanding is that fetch.message.max.bytes * number of partitions consumed from (50) + messaging overhead must be < receive.message.max.bytes. Say, (1000000000 - 10%) / 50 = 18000000. For good measure I have also tried reducing fetch.message.max.bytes by an order of magnitude, and many other combinations.

The only thing that may be strange about my setup is I have enable.auto.offset.store set to false. I have java based consumers that are able to work through the topic fine.

Are there any other configurations I could be trying?

@natb1
Copy link

natb1 commented Mar 12, 2018

A little more context for the above:

>>> confluent_kafka.version()
('0.11.0', 720896)

My broker version is 1.0.0

I've simplified my consumer to just be the hello world consumer from conluent-kafka-python

from confluent_kafka import Consumer, KafkaError

c = Consumer({
    'bootstrap.servers': 'broker.kafka:9092',
    'group.id': 'mygroup',
    'fetch.message.max.bytes': 18000000,
    'receive.message.max.bytes': 1000000000,
    'log_level': 7,
    'default.topic.config': {'auto.offset.reset': 'smallest'}})
c.subscribe(['mytopic'])
count = 0
while True:
    count += 1
    if count % 10000 == 0:
        print(count)
    msg = c.poll()
    if msg.error():
        if msg.error().code() != KafkaError._PARTITION_EOF:
            print(f"message consumer error: {msg.error()}")
        else:
            print('kafka partition eof')
c.close()

The exact consumer error I am getting is:

%3|1520882829.655|ERROR|rdkafka#consumer-1| [thrd:kafka-green-1.broker.kafka.svc.cluster.local:9092/1]: kafka-green-1.broker.kafka.svc.cluster.local:9092/1: Receive failed: Invalid response size 1000000540 (0..1000000000): increase receive.message.max.bytes

Though after digging deeper I also see occasional:

"%4|1520883097.546|METADATA|rdkafka#consumer-1| [thrd:main]: kafka-green-0.broker.kafka.svc.cluster.local:9092/0: Metadata request failed: Local: Bad message format (64580264ms)

In the broker logs all I see is repeated:

INFO [GroupCoordinator 0]: Preparing to rebalance group mygroup with old generation 280 (__consumer_offsets-39) (kafka.coordinator.group.GroupCoordinator)
INFO [GroupCoordinator 0]: Stabilized group mygroup generation 281 (__consumer_offsets-39) (kafka.coordinator.group.GroupCoordinator)
INFO [GroupCoordinator 0]: Assignment received from leader for group mygroup for generation 281

Could the response size error be a red herring?

I have also updated the max.message.bytes on my topic. Here is the full topic config:

max.message.bytes: 1000000000
min.cleanable.dirty.ratio: 0.01
cleanup.policy: compact
retention.ms: 600000
segment.ms: 600000
partitions: 50
replication: 3

@natb1
Copy link

natb1 commented Mar 12, 2018

I think I've narrowed this down to an issue with the particular version of librdkafka-dev that is distributed with debian. This is different than the version of librdkafka-dev that is distributed with alpine which is working fine for me.

alpine:

>>> confluent_kafka.libversion()
('0.11.1', 721407)

debian:

>>> confluent_kafka.libversion()
('0.11.0', 721151)

@edenhill
Copy link
Contributor

edenhill commented Mar 20, 2018

The fix for this includes adding the (java) config property fetch.max.bytes, it is a minimal fix but might affect existing applications (that will need to be properly configured - nothing else).

https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes

@edenhill edenhill added bug and removed wait-info labels Mar 20, 2018
edenhill added a commit that referenced this issue Mar 20, 2018
…-74, #1616)

`receive.message.max.bytes` was used in FetchRequest.max_bytes to tell
the broker to limit the response size, but since the broker may overshoot
this value up to one message.max.bytes it resulted in an endless config
chase where receive.message.max.bytes was increased and the response
size would follow + a couple of extra bytes.
Using the new `fetch.max.bytes` property and making sure it is lower than
`receive.message.max.bytes` fixes the issue.

This also adds verification for these configuration values to
the rd_kafka_new() call.
@edenhill
Copy link
Contributor

This is fixed on the KIP-74 branch, it would be great if (all of) you could try out this branch and verify that the fix works. The CONFIGURATION.md docs have been updated to show the relation between receive.message.max.bytes, fetch.max.bytes and message.max.bytes.

How to:
git clone https://github.com/edenhill/librdkafka.git
cd librdkafka.git
git checkout KIP-74
./configure
make -j
sudo make install

then do your app thing.

edenhill added a commit that referenced this issue Mar 20, 2018
…-74, #1616)

`receive.message.max.bytes` was used in FetchRequest.max_bytes to tell
the broker to limit the response size, but since the broker may overshoot
this value up to one message.max.bytes it resulted in an endless config
chase where receive.message.max.bytes was increased and the response
size would follow + a couple of extra bytes.
Using the new `fetch.max.bytes` property and making sure it is lower than
`receive.message.max.bytes` fixes the issue.
edenhill added a commit that referenced this issue Mar 21, 2018
…-74, #1616)

`receive.message.max.bytes` was used in FetchRequest.max_bytes to tell
the broker to limit the response size, but since the broker may overshoot
this value up to one message.max.bytes it resulted in an endless config
chase where receive.message.max.bytes was increased and the response
size would follow + a couple of extra bytes.
Using the new `fetch.max.bytes` property and making sure it is lower than
`receive.message.max.bytes` fixes the issue.
@rsienko
Copy link

rsienko commented Jun 20, 2018

I also have this issue using version 0.11.4 and fetch.max.bytes. In my case, received size was 15 bytes too large. Increasing buffer to 4096 seems to have helped. Judging by the comments in the code, my topics tend to have long names and possibly that is the trigger.

Maybe this should be reopened to use a formula or at least configuration for the additional buffer size?

@edenhill
Copy link
Contributor

@feigt Interesting, how long are your topic names?

@rsienko
Copy link

rsienko commented Jun 20, 2018

About 40 characters :)

If you think that's impossible, here are some more details. 16 partitions, single producer consumer pair. Broker defaults, max message = max batch = fetch.max.bytes. Queue size 16 * fetch.max.bytes. The goal is to have enough memory to reliably fetch from all partitions with an upper bound.

Producer sends small batches of up to 1000 messages. I was testing scenario where consumer is incapable of keeping up and hitting max fetch size. This scenario failed because of this problem.

@ajpieri
Copy link

ajpieri commented Feb 8, 2021

What file are we talking about adding these max bytes properties to? Is this the kafka-shared-config?

My company is using the Confluent bundle, but I am getting this error, but for a producer. I am trying to connect via a python script. Here is my code. "kafka-confluent.apps.ocp-dev.gmso.com" is a route we setup.

`
from confluent_kafka import Producer
import json
import socket

def acked(err, msg):
if err is not None:
print("failed to deliver message: {}".format(err.str()))
else:
print("produced to: {} [{}] @ {}".format(msg.topic(), msg.partition(), msg.offset()))

p = Producer({'bootstrap.servers': 'kafka-confluent.apps.ocp-dev.gmso.com:80',
'sasl.mechanism': 'PLAIN',
'security.protocol': 'SASL_PLAINTEXT',
'client.id': socket.gethostname(),
'sasl.username': 'test',
'sasl.password': 'test123'
})

p.produce('tony_test_for_python', key="hello", value='Hello World', callback=acked)
p.flush(30)
`

There error I am getting is
%3|1612806548.850|FAIL|udw20idabast02#producer-1| [thrd:sasl_plaintext://kafka-confluent.apps.ocp-dev.gmso.com:80/boots]: sasl_plaintext://kafka-confluent.apps.ocp-dev.gmso.com:80/bootstrap: Receive failed: Invalid response size 1213486160 (0..100000000): increase receive.message.max.bytes (after 1ms in state APIVERSION_QUERY)

I added the following to my kafka-shared-config. The same yaml file that has message.max.bytes=2097164
fetch.max.bytes=100000000
receive.message.max.bytes=200000000

Is it "fetch.max.bytes" or "fetch.message.max.bytes"?

@edenhill
Copy link
Contributor

edenhill commented Feb 8, 2021

You seem to be connecting to an HTTP endpoint:

$ calc
; 1213486160
	1213486160 /* 0x48545450 */

$ ascii 0x48 0x54 0x54 0x50
ASCII 4/8 is decimal 072, hex 48, octal 110, bits 01001000: prints as `H'
ASCII 5/4 is decimal 084, hex 54, octal 124, bits 01010100: prints as `T'
ASCII 5/4 is decimal 084, hex 54, octal 124, bits 01010100: prints as `T'
ASCII 5/0 is decimal 080, hex 50, octal 120, bits 01010000: prints as `P'

@ajpieri
Copy link

ajpieri commented Feb 8, 2021

You seem to be connecting to an HTTP endpoint:

$ calc
; 1213486160
	1213486160 /* 0x48545450 */

$ ascii 0x48 0x54 0x54 0x50
ASCII 4/8 is decimal 072, hex 48, octal 110, bits 01001000: prints as `H'
ASCII 5/4 is decimal 084, hex 54, octal 124, bits 01010100: prints as `T'
ASCII 5/4 is decimal 084, hex 54, octal 124, bits 01010100: prints as `T'
ASCII 5/0 is decimal 080, hex 50, octal 120, bits 01010000: prints as `P'

What does that mean? I mean, I know what it means to connect to an HTTP endpoing, but I don't know what that means in this context. What should it be connecting to? How would I even change that?

I am pretty new to Kafka, and since we are using Confluent, there are very few things I setup myself. So, I am not sure what it should be using, or what I might be doing wrong.

@edenhill
Copy link
Contributor

edenhill commented Feb 9, 2021

kafka-confluent.apps.ocp-dev.gmso.com:80

You are connecting to a web-server (port), not Kafka (which usually runs on port 9092).

@ajpieri
Copy link

ajpieri commented Feb 9, 2021

I have tried port 9092. I have tried all kinds of different setups. We were trying 80 because of some network stuff we tried. However, it seems all I need to provide is the route, and it will add the 9092 if I don't specify a port. Although, in the below code I specified the 9092 port just to be sure. With the following code, I get the following error.

import json
import socket

def acked(err, msg):
    if err is not None:
        print("failed to deliver message: {}".format(err.str()))
    else:
        print("produced to: {} [{}] @ {}".format(msg.topic(), msg.partition(), msg.offset()))

p = Producer({
              'bootstrap.servers': 'kafka-confluent.apps.ocp-dev.gsmo.com:9092',
              'sasl.mechanism': 'PLAIN',
              'security.protocol': 'SASL_PLAINTEXT',
              'sasl.username': 'test',
              'sasl.password': 'test123',
            #   'security.protocol': 'SASL_SSL',
              'client.id': socket.gethostname()
              })
              
p.produce('python_test', key="test", value='Hello World', callback=acked)
p.flush(30)

Error:

%3|1612885183.118|FAIL|udw20idabast02#producer-1| [thrd:sasl_plaintext://kafka.b0.kafka-confluent.apps.ocp-dev.gsmo.com]: sasl_plaintext://kafka.b0.kafka-confluent.apps.ocp-dev.gsmo.com:9092/bootstrap: Connect to ipv4#72.52.179.174:9092 failed: Unknown error (after 1294ms in state CONNECT)
%3|1612885185.046|FAIL|udw20idabast02#producer-1| [thrd:sasl_plaintext://kafka.b0.kafka-confluent.apps.ocp-dev.gsmo.com]: sasl_plaintext://kafka.b0.kafka-confluent.apps.ocp-dev.gsmo.com:9092/bootstrap: Connect to ipv4#72.52.179.174:9092 failed: Unknown error (after 1222ms in state CONNECT, 1 identical error(s) suppressed) 

@edenhill
Copy link
Contributor

edenhill commented Feb 9, 2021

Looks like there are networking issues, that's outside the client's control.

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

6 participants