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

Evan Huus edited comment on KAFKA-1718 at 10/20/14 9:23 PM:
------------------------------------------------------------

That sounds plausible.

1. How do I verify if that is/isn't the problem I'm seeing? Is there some piece 
of backtrace or breakpoint I can check or something?
2. If that is the problem, what is a client supposed to do about it? Leave a 
few KiB spare and hope that that's enough? Is there no way for a client using 
compression to be sure that the broker will actually accept the payload (unless 
presumably the uncompressed payload is already small enough)?

Edit: actually, that can't be it. From my original report "When uncompressed, 
each message contains a message set of 999600 bytes". So unless the 
recompression on the broker's end *added* a substantial amount of data (which 
is improbable; the messages were all 0s)...


was (Author: eapache):
That sounds plausible.

1. How do I verify if that is/isn't the problem I'm seeing? Is there some piece 
of backtrace or breakpoint I can check or something?
2. If that is the problem, what is a client supposed to do about it? Leave a 
few KiB spare and hope that that's enough? Is there no way for a client using 
compression to be sure that the broker will actually accept the payload (unless 
presumably the uncompressed payload is already small enough)?

> "Message Size Too Large" error when only small messages produced with Snappy
> ----------------------------------------------------------------------------
>
>                 Key: KAFKA-1718
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1718
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.8.1.1
>            Reporter: Evan Huus
>            Priority: Critical
>
> I'm the primary author of the Go bindings, and while I originally received 
> this as a bug against my bindings, I'm coming to the conclusion that it's a 
> bug in the broker somehow.
> Specifically, take a look at the last two kafka packets in the following 
> packet capture: https://dl.dropboxusercontent.com/u/171647/kafka.pcapng (you 
> will need a trunk build of Wireshark to fully decode the kafka part of the 
> packets).
> The produce request contains two partitions on one topic. Each partition has 
> one message set (sizes 977205 bytes and 967362 bytes respectively). Each 
> message set is a sequential collection of snappy-compressed messages, each 
> message of size 46899. When uncompressed, each message contains a message set 
> of 999600 bytes, containing a sequence of uncompressed 1024-byte messages.
> However, the broker responds to this with a MessageSizeTooLarge error, full 
> stacktrace from the broker logs being:
> kafka.common.MessageSizeTooLargeException: Message size is 1070127 bytes 
> which exceeds the maximum configured message size of 1000012.
>       at kafka.log.Log$$anonfun$append$1.apply(Log.scala:267)
>       at kafka.log.Log$$anonfun$append$1.apply(Log.scala:265)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>       at kafka.utils.IteratorTemplate.foreach(IteratorTemplate.scala:32)
>       at kafka.log.Log.append(Log.scala:265)
>       at kafka.cluster.Partition.appendMessagesToLeader(Partition.scala:354)
>       at 
> kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:376)
>       at 
> kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:366)
>       at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
>       at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
>       at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
>       at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
>       at 
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
>       at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
>       at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
>       at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
>       at scala.collection.AbstractTraversable.map(Traversable.scala:105)
>       at kafka.server.KafkaApis.appendToLocalLog(KafkaApis.scala:366)
>       at kafka.server.KafkaApis.handleProducerRequest(KafkaApis.scala:292)
>       at kafka.server.KafkaApis.handle(KafkaApis.scala:185)
>       at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:42)
>       at java.lang.Thread.run(Thread.java:695)
> Since as far as I can tell none of the sizes in the actual produced packet 
> exceed the defined maximum, I can only assume that the broker is 
> miscalculating something somewhere and throwing the exception improperly.
> ---
> This issue can be reliably reproduced using an out-of-the-box binary download 
> of 0.8.1.1 and the following gist: 
> https://gist.github.com/eapache/ce0f15311c605a165ce7 (you will need to use 
> the `producer-ng` branch of the Sarama library).
> ---
> I am happy to provide any more information you might need, or to do relevant 
> experiments etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to