Hi, I just double checked my configuration and the broker has message.max.bytes set to 1 gig, the consumers have the same setting for max fetch size. I've lowered this to 600 mb and still see the same error :(,
at the moment kafka is un-usable for me, the the only other alternative is writing my own client (as i'm doing with the producer), what a pain! On Wed, Jan 1, 2014 at 6:45 PM, Jun Rao <jun...@gmail.com> wrote: > In our wire protocol, we expect the first 4 bytes for a response to be its > size. If the actual size is larger than 2GB, what's stored in the those 4 > bytes is the overflowed value. This could cause some of the buffer size to > be smaller than it should be later on. If #partitions * fetch_size is > larger than 2GB in a single fetch request, you could hit this problem. You > can try reducing the fetch size. Ideally, the sender should catch this and > throw an exception, which we don't do currently. > > Thanks, > > Jun > > > On Wed, Jan 1, 2014 at 9:27 AM, Gerrit Jansen van Vuuren < > gerrit...@gmail.com> wrote: > > > Mm... Could be Im not sure if in a single request though. I am moving > allot > > of data. Any pointer at were in the code the overflow might start? > > On 1 Jan 2014 18:13, "Jun Rao" <jun...@gmail.com> wrote: > > > > > Are you fetching more than 2GB of data in a single fetch response > (across > > > all partitions)? Currently, we don't handle integer overflow properly. > > > > > > Thanks, > > > > > > Jun > > > > > > > > > On Wed, Jan 1, 2014 at 4:24 AM, Gerrit Jansen van Vuuren < > > > gerrit...@gmail.com> wrote: > > > > > > > While consuming from the topics I get an IlegalArgumentException and > > all > > > > consumption stops, the error keeps on throwing. > > > > > > > > I've tracked it down to FectchResponse.scala line 33 > > > > > > > > The error happens when the FetchResponsePartitionData object's > readFrom > > > > method calls: > > > > messageSetBuffer.limit(messageSetSize) > > > > > > > > I put in some debug code the the messageSetSize is 671758648, while > the > > > > buffer.capacity() gives 155733313, for some reason the buffer is > > smaller > > > > than the required message size. > > > > > > > > I don't know the consumer code enough to debug this. It doesn't > matter > > if > > > > compression is used or not. > > > > > > > > I've created a jira ticket for this: > > > > https://issues.apache.org/jira/browse/KAFKA-1196 > > > > > > > > this is a real pain for me because I'm unable to consume from kafka > at > > > all > > > > :( > > > > > > > > > > > > Any ideas on possible config? or code changes I could try to fix? > > > > > > > > Regards, > > > > Gerrit > > > > > > > > > >