[
https://issues.apache.org/jira/browse/KAFKA-4169?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17758545#comment-17758545
]
Pere Urbon-Bayes commented on KAFKA-4169:
-----------------------------------------
Hi,
giving it a bit more thought to this problem, I think the doc is wrong and
confusing, from the current docs at kafka.apache.org
> The maximum size of a request in bytes. This setting will limit the number of
> record batches the producer will send in a single request to avoid sending
> huge requests.
However, from the code, we can see:
{code:java}
int serializedSize =
AbstractRecords.estimateSizeInBytesUpperBound(apiVersions.maxUsableProduceMagic(),
compressionType, serializedKey, serializedValue, headers);
ensureValidRecordSize(serializedSize);
long timestamp = record.timestamp() == null ? nowMs : record.timestamp();{code}
{code:java}
/**
* Validate that the record size isn't too large
*/
private void ensureValidRecordSize(int size) {
if (size > maxRequestSize)
throw new RecordTooLargeException("The message is " + size +
" bytes when serialized which is larger than " + maxRequestSize + ", which is
the value of the " +
ProducerConfig.MAX_REQUEST_SIZE_CONFIG + " configuration.");
if (size > totalMemorySize)
throw new RecordTooLargeException("The message is " + size +
" bytes when serialized which is larger than the total memory buffer you have
configured with the " +
ProducerConfig.BUFFER_MEMORY_CONFIG +
" configuration.");
}{code}
both within KafkaProducer.java
mixed record size limits, with the size of the request, that if not mistaken,
are better abstracted with the batches, ex.ProducerBatch, right?
> 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
> Assignee: Pere Urbon-Bayes
> Priority: Major
> Attachments: Screenshot 2023-08-24 at 14.06.01.png
>
>
> 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
(v8.20.10#820010)