You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Ismael Juma (JIRA)" <ji...@apache.org> on 2017/06/02 07:36:04 UTC

[jira] [Comment Edited] (KAFKA-5032) Think through implications of max.message.size affecting record batches in message format V2

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

Ismael Juma edited comment on KAFKA-5032 at 6/2/17 7:35 AM:
------------------------------------------------------------

I looked into the config change and it looks like we include MessageSet.LogOverhead and DefaultRecordBatch.RECORD_BATCH_OVERHEAD (depending on the message format version) in the size computation used by the producer and the default `MAX_REQUEST_SIZE_CONFIG` is 1 * 1024 * 1024. Given that, it seems that we don't need to change the default broker config. In fact, it looks like there is no reason to add `MessageSet.LogOverhead` either, but I suggest we just leave this as it is. cc [~junrao] [~hachikuji] in case I am missing something.


was (Author: ijuma):
I looked into the config change and it looks like we include MessageSet.LogOverhead and DefaultRecordBatch.RECORD_BATCH_OVERHEAD (depending on the message format version) in the size computation used by the producer and the default `MAX_REQUEST_SIZE_CONFIG` is 1 * 1024 * 1024. Given that, it seems that we don't need to change the default broker config. In fact, it looks like there is no reason to add `MessageSet.LogOverhead` either, but I suggest we just leave this as it is. cc [~junrao][~hachikuji] in case I am missing something.

> Think through implications of max.message.size affecting record batches in message format V2
> --------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-5032
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5032
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: clients, core, producer 
>            Reporter: Ismael Juma
>            Priority: Critical
>              Labels: documentation, exactly-once
>             Fix For: 0.11.0.0
>
>
> It's worth noting that the new behaviour for uncompressed messages is the same as the existing behaviour for compressed messages.
> A few things to think about:
> 1. Do the producer settings max.request.size and batch.size still make sense and do we need to update the documentation? My conclusion is that things are still fine, but we may need to revise the docs.
> 2. Consider changing default max message set size to include record batch overhead. This is currently defined as:
> {code}
> val MessageMaxBytes = 1000000 + MessageSet.LogOverhead
> {code}
> We should consider changing it to (I haven't thought it through though):
> {code}
> val MessageMaxBytes = 1000000 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD
> {code}
> 3. When a record batch is too large, we throw RecordTooLargeException, which is confusing because there's also a RecordBatchTooLargeException. We should consider renaming these exceptions to make the behaviour clearer.
> 4. We should consider deprecating max.message.bytes (server config) and message.max.bytes (topic config) in favour of configs that make it clear that we are talking about record batches instead of individual messages.
> Part of the work in this JIRA is working out what should be done for 0.11.0.0 and what can be done later.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)