John,

There is not a specific JIRA for this change as it is only implemented in
the new Java producer:

https://issues.apache.org/jira/browse/KAFKA-1239

Related classes are RecordAccumulator and MemoryRecords:

https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java


Fang,

Do you have a way to re-produce the issue? I.e. if you have the exact same
produce data at hand, could you validate that their cumulated size is less
than the limit and then try sending them to Kafka and see if it always
triggers the problem?



Guozhang

On Mon, Mar 7, 2016 at 10:23 AM, Fang Wong <fw...@salesforce.com> wrote:

> No, we don't have compression turned on the batch size is the default:
> 16384.
> But the message size is very small, even with that batch size, it is
> impossible to exceed the size limit.
>
> Thanks,
> Fang
>
> On Sun, Mar 6, 2016 at 6:09 PM, John Dennison <dennison.j...@gmail.com>
> wrote:
>
> > Guozhang,
> >
> > Do you know the ticket for for changing the "batching criterion from
> > #.messages to bytes." I am unable to find it. Working on porting
> > a similar change to pykafka.
> >
> >
> > John
> >
> >
> > On Sat, Mar 5, 2016 at 4:29 PM, Guozhang Wang <wangg...@gmail.com>
> wrote:
> >
> > > Hello,
> > >
> > > Did you have compression turned on and batching (in terms of
> #.messages)?
> > > In that case the whole compressed message set is treated as a single
> > > message on the broker and hence could possibly exceed the limit.
> > >
> > > In newer versions we have changed the batching criterion from
> #.messages
> > to
> > > bytes, which is aimed at resolving such issues.
> > >
> > > Guozhang
> > >
> > > On Thu, Mar 3, 2016 at 1:04 PM, Fang Wong <fw...@salesforce.com>
> wrote:
> > >
> > > > Got the following error message with Kafka 0.8.2.1:
> > > > [2016-02-26 20:33:43,025] INFO Closing socket connection to /x due to
> > > > invalid request: Request of length 1937006964 is not valid, it is
> > larger
> > > > than the maximum size of 104857600 bytes. (kafka.network.Processor)
> > > >
> > > > Didn't send a large message at all, it seems like encoding issue or
> > > partial
> > > > request, any suggestion how to fix it?
> > > >
> > > > The code is like below:
> > > >
> > > >     ByteArrayOutputStream bos = new ByteArrayOutputStream();
> > > >
> > > >     DataOutputStream dos = new DataOutputStream(bos);
> > > >
> > > >     dos.writeLong(System.currentTimeMillis());
> > > >
> > > >     OutputStreamWriter byteWriter = new OutputStreamWriter(bos,
> > > > com.force.commons.text.EncodingUtil.UTF_ENCODING);
> > > >
> > > >     gson.toJson(obj, byteWriter);
> > > >
> > > >     byte[] payload = bos.toByteArray();
> > > >
> > > >     ProducerRecord<String, byte[]> data = new ProducerRecord<String,
> > > > byte[]>(“Topic”, 0, null, payload);
> > > >
> > > >     kafkaProducer.send(data);
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>



-- 
-- Guozhang

Reply via email to