Dinesh,

Your fetch.message.max.bytes is 188743680 < 155MB, but you said some
messages can be as large as 180MB. Could you try to set it to be larger
than, say 200MB and see if it helps?

Guozhang

On Tue, Jan 13, 2015 at 4:18 AM, dinesh kumar <dinesh...@gmail.com> wrote:

> Hi,
> I am been facing some JAVA high level consumer related issues lately and
> would like to understand more on this.
>
> We have 9 bare-metals (48 core, 250 GB, Terabytes of Hard disks) running
> *Kafka
> 0.8.2* and 5 independent VM (8 core, 60 GB) running zookeeper.
>
> I have a topic that has key as metadata and value as a file. The file can
> be as large as *180 MB.* We have a topic with 90 partitions. Sometimes
> there will be only one consumer consuming from the topic. When the consumer
> group for my topic has a *lag in the range of 200's* and when I start a
> consumer (no other consumer running before) there is *no data* coming
> through to the consumer.
>
> Please find below my consumer parameters.
>
> "zookeeper.connect"                => <zookeepers>,
> "group.id"                         => "default",
> "consumer.timeout.ms"              => "-1",
> "auto.offset.reset"                => "smallest",
> "auto.commit.enable"               => "false",
> "consumer.timeout.ms"          => "-1",
> "zookeeper.session.timeout.ms" => "100000",
> "zookeeper.connection.timeout.ms"  => "6000",
> "zookeeper.sync.time.ms"           => "2000",
> "rebalance.backoff.ms"             =>  "20000",
> "rebalance.max.retries"            => "50"
> "fetch.message.max.bytes"      => "188743680",
> "fetch.size"                   => "18874368"
>
>
>
> This problem occurs only when the *auto.offset.reset *property is
> *smallest.
> *I am able to get data if the offset is largest. I tried using the *console
> consumer* for the same topic and consumer group with *--from-beginning*
> option, I can see the data getting printed. I looked into the
> ConsoleConsumer code and I saw that there was no
> *fetch.message.max.bytes *property
> in the consumer option.
>
> So I removed the *fetch.message.max.bytes *from my code and the consumer
> started working but was throwing exception when the message is large.
>
> So *fetch.message.max.bytes *seemed to be the problem but I cannot do
> without it as my messages a big files. Can someone explain to me what is
> the issue here? I also adjusted the *fetch.size *parameter according to my
> max message size but it did not help.
>
>
> To summerize, I would like to understand what is happening in the consumer
> end when handling large lags with big *fetch.message.max.bytes. *
>
>
> Thanks,
> Dinesh
>



-- 
-- Guozhang

Reply via email to