Hey Anindya,


On Wed, 15 Jan 2020 at 18:23, Anindya Haldar <anindya.hal...@oracle.com>
wrote:

> Thanks for the response.
>
> Essentially, we are looking for a confirmation that a send acknowledgement
> received at the client’s end will ensure the message is indeed persisted to
> the replication logs. We initially wondered whether the client has to make
> an explicit flush() call or whether it has to commit a producer transaction
> for that to happen. Based upon what I understand now from your response, a
> flush() or commitTransaction() call should not be necessary for this, and a
> send acknowledgement via the successful return from the get() call on the
> future will ensure the persistence of the message.
>
> Please feel free to correct me if I didn’t get it right.
>

 I'm sure you have done the reading, but to be in context of your question,
*commitTransaction()* is sufficient on it's own (see excerpt from *flush()*
doc below)

*Applications don't need to call this method for transactional producers,
> since the commitTransaction() will flush all buffered records before
> performing the commit. This ensures that all the send(ProducerRecord) calls
> made since the previous beginTransaction() are completed before the
> commit. *


  But you *do *need to call commitTransaction() (for txn based producers),
or flush() (for normal cases) to send the records *immediately*. Otherwise,
they will be sent when the data buffer is full (re: buffer.memory and
linger.ms).

  If you want to know more about transactions, there are some nice articles
regarding txn producers

https://cwiki.apache.org/confluence/display/KAFKA/Transactional+Messaging+in+Kafka
  https://www.confluent.io/blog/transactions-apache-kafka/

 Also, if you are interested to become more technical, please check the
codebase for KafkaProducer and see what doSend() and wakeup() is doing:


https://github.com/apache/kafka/blob/5c00191ea957fef425bf5dbbe47d70e41249e2d6/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L832

 I hope this helps.

 Regards,

>
> Sincerely,
> Anindya Haldar
> Oracle Responsys
>
>
> > On Jan 15, 2020, at 8:55 AM, M. Manna <manme...@gmail.com> wrote:
> >
> > Anindya,
> >
> > On Wed, 15 Jan 2020 at 16:49, Anindya Haldar <anindya.hal...@oracle.com>
> > wrote:
> >
> >> In our case, the minimum in-sync replicas is set to 2.
> >>
> >> Given that, what will be expected behavior for the scenario I outlined?
> >>
> >
> > This means you will get confirmation when 2 of them have acknowledged. so
> > you will always have 2 in-sync.
> >
> > Perhaps drilling each detail and having a long thread, you could explain
> > what is it you are trying to investigate/identify? We will be happy to
> help.
> >
> > Regards,
> >
> >
> >> Sincerely,
> >> Anindya Haldar
> >> Oracle Responsys
> >>
> >>
> >>> On Jan 15, 2020, at 6:38 AM, Ismael Juma <ism...@juma.me.uk> wrote:
> >>>
> >>> To all the in-sync replicas. You can set the minimum number of in-sync
> >>> replicas via the min.insync.replicas topic/broker config.
> >>>
> >>> Ismael
> >>>
> >>> On Tue, Jan 14, 2020 at 11:11 AM Anindya Haldar <
> >> anindya.hal...@oracle.com>
> >>> wrote:
> >>>
> >>>> I have a question related to the semantics of a producer send and the
> >> get
> >>>> calls on the future returned by the send call.
> >>>>
> >>>> - It is a Java application, using the Kafka Java client library
> >>>> - The application is set up to use 3 replicas and using acks=all for
> the
> >>>> producer
> >>>> - the application is using a non-zero value for linger.ms and
> >> batch.size
> >>>> parameters
> >>>> - The application is using a single non-transactional Kafka producer
> >>>> instance, shared across a number of threads
> >>>>
> >>>> With that,
> >>>>
> >>>> - Any application thread makes a send() call on the producer.
> >>>> - Then the same thread calls get() on the future returned by the last
> >>>> send() call
> >>>> - The get() call on the future returns after it gets the
> acknowledgement
> >>>> from the system for the message send
> >>>>
> >>>> At this point, is it guaranteed that the message has actually been
> >> written
> >>>> (but may not be committed by calling fsync) to ALL of the replicas’
> >>>> filesystems?
> >>>>
> >>>> Sincerely,
> >>>> Anindya Haldar
> >>>> Oracle Responsys
> >>>>
> >>>>
> >>
> >>
>
>

Reply via email to