[ 
https://issues.apache.org/jira/browse/KAFKA-4169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16159621#comment-16159621
 ] 

Joel Koshy commented on KAFKA-4169:
-----------------------------------

I had been meaning to comment on this a while back but did not get around to 
it. However, someone was asking about this today. I think the history behind 
this is as follows:
* The current config doc dates back to the initial implementation of the 
producer: {{The maximum size of a request in bytes. This is also effectively a 
cap on the maximum record size...}}
* When the new producer was first implemented, it (initially) did not support 
compression. With that constraint, the above statement is true - it is 
effectively a per-record serialized cap.
* When compression was added shortly after the initial implementation, the 
above configuration did not quite make sense but was never amended.
* It really should be checked in the sender but then we may also want to divide 
up partitions into smaller requests (if there are multiple partitions in the 
request).
* I don't think there was any intent at any point in time to do an individual 
record size limit check. It probably does not make sense to do that given that 
the {{message.max.bytes}} property on the broker applies to a compressed 
record-set, never an individual record.

> Calculation of message size is too conservative for compressed messages
> -----------------------------------------------------------------------
>
>                 Key: KAFKA-4169
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4169
>             Project: Kafka
>          Issue Type: Bug
>          Components: producer 
>    Affects Versions: 0.10.0.1
>            Reporter: Dustin Cote
>
> Currently the producer uses the uncompressed message size to check against 
> {{max.request.size}} even if a {{compression.type}} is defined.  This can be 
> reproduced as follows:
> {code}
> # dd if=/dev/zero of=/tmp/outsmaller.dat bs=1024 count=1000
> # cat /tmp/out.dat | bin/kafka-console-producer --broker-list localhost:9092 
> --topic tester --producer-property compression.type=gzip
> {code}
> The above code creates a file that is the same size as the default for 
> {{max.request.size}} and the added overhead of the message pushes the 
> uncompressed size over the limit.  Compressing the message ahead of time 
> allows the message to go through.  When the message is blocked, the following 
> exception is produced:
> {code}
> [2016-09-14 08:56:19,558] ERROR Error when sending message to topic tester 
> with key: null, value: 1048576 bytes with error: 
> (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback)
> org.apache.kafka.common.errors.RecordTooLargeException: The message is 
> 1048610 bytes when serialized which is larger than the maximum request size 
> you have configured with the max.request.size configuration.
> {code}
> For completeness, I have confirmed that the console producer is setting 
> {{compression.type}} properly by enabling DEBUG so this appears to be a 
> problem in the size estimate of the message itself.  I would suggest we 
> compress before we serialize instead of the other way around to avoid this.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to