So if I understand correctly, even if I delay flushing, the consumer will
get the messages as soon as the broker receives them and put them into page
cache (assuming producer doesn't wait for acks from brokers)?

And will the decrease of log.flush interval help reduce latency between
producer and consumer?

Thanks.


On Fri, Aug 14, 2015 at 11:57 AM, Kishore Senji <kse...@gmail.com> wrote:

> Thank you Gwen for correcting me. This document (
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Replication) in
> "Writes" section also has specified the same thing as you have mentioned.
> One thing is not clear to me as to what happens when the Replicas add the
> message to memory but the leader fails before acking to the producer. Later
> the leader replica is chosen to be the leader for the partition, it will
> advance the HW to its LEO (which has the message). The producer can resend
> the same message thinking it failed and there will be a duplicate message.
> Is my understanding correct here?
>
> On Thu, Aug 13, 2015 at 10:50 PM, Gwen Shapira <g...@confluent.io> wrote:
>
> > On Thu, Aug 13, 2015 at 4:10 PM, Kishore Senji <kse...@gmail.com> wrote:
> >
> > > Consumers can only fetch data up to the committed offset and the reason
> > is
> > > reliability and durability on a broker crash (some consumers might get
> > the
> > > new data and some may not as the data is not yet committed and lost).
> > Data
> > > will be committed when it is flushed. So if you delay the flushing,
> > > consumers won't get those messages until that time.
> > >
> >
> > As far as I know, this is not accurate.
> >
> > A message is considered committed when all ISR replicas received it (this
> > much is documented). This doesn't need to include writing to disk, which
> > will happen asynchronously.
> >
> >
> > >
> > > Even though you flush periodically based on log.flush.interval.messages
> > and
> > > log.flush.interval.ms, if the segment file is in the pagecache, the
> > > consumers will still benefit from that pagecache and OS wouldn't read
> it
> > > again from disk.
> > >
> > > On Thu, Aug 13, 2015 at 2:54 PM Yuheng Du <yuheng.du.h...@gmail.com>
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > As I understand it, kafka brokers will store the incoming messages
> into
> > > > pagecache as much as possible and then flush them into disk, right?
> > > >
> > > > But in my experiment where 90 producers is publishing data into 6
> > > brokers,
> > > > I see that the log directory on disk where broker stores the data is
> > > > constantly increasing (every seconds.) So why this is happening? Does
> > > this
> > > > has to do with the default "log.flush.interval" setting?
> > > >
> > > > I want the broker to write to disk less often when serving some
> on-line
> > > > consumers to reduce latency. I tested in my broker the disk write
> speed
> > > is
> > > > around 110MB/s.
> > > >
> > > > Thanks for any replies.
> > > >
> > >
> >
>

Reply via email to