Andiya, On Wed, 15 Jan 2020 at 21:59, Anindya Haldar <anindya.hal...@oracle.com> wrote:
> Okay, let’s say > > - the application is using a non-transactional producer, shared across > multiple threads > - the linger.ms and buffer.memory is non-zero, and so is batch.size such > that messages are actually batched > - the replication factor is 3 > - the minimum number of ISRs is 2 > - the parameter ack is set to ‘all’ > > Now the application calls send(), get a future back, and then calls get() > on the future. At some point (driven by the batching related parameters and > a number of other factors) the get() call to the future returns > successfully. > > Precisely at this point does Kafka guarantee that the message has been > persisted to the leader’s and all the ISRs’ logs? By persisted, I mean > written to the replication logs, but may or may not yet have been committed > to the storage media by the fsync() call. > At this stage the ISRs (this includes leader) are all acknowledged. > > If the answer is yes, it looks good from here. If the answer is no, then > what else does the application need to do? > > Sincerely, > Anindya Haldar > Oracle Responsys > > > > On Jan 15, 2020, at 12:31 PM, M. Manna <manme...@gmail.com> wrote: > > > > 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://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_Transactional-2BMessaging-2Bin-2BKafka&d=DwIFaQ&c=RoP1YumCXCgaWHvlZYR8PZh8Bv7qIrMUB65eapI_JnE&r=vmJiAMDGSxNeZnFztNs5ITB_i_Z3h3VtLPGma9y7cKI&m=qtRoal09Ax8f1wskhpGkLJz8loX98EAVCX95pMjnI8s&s=laTP9-1xOTyb1L9AFMVLYSlvZE-nfgJ7N4rsL3NyZvU&e= > > > https://urldefense.proofpoint.com/v2/url?u=https-3A__www.confluent.io_blog_transactions-2Dapache-2Dkafka_&d=DwIFaQ&c=RoP1YumCXCgaWHvlZYR8PZh8Bv7qIrMUB65eapI_JnE&r=vmJiAMDGSxNeZnFztNs5ITB_i_Z3h3VtLPGma9y7cKI&m=qtRoal09Ax8f1wskhpGkLJz8loX98EAVCX95pMjnI8s&s=SMCrXdI5TvfT6FEiqpQAA_8f8x8RA2MRFzrOKJmCFFc&e= > > > > Also, if you are interested to become more technical, please check the > > codebase for KafkaProducer and see what doSend() and wakeup() is doing: > > > > > > > https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_kafka_blob_5c00191ea957fef425bf5dbbe47d70e41249e2d6_clients_src_main_java_org_apache_kafka_clients_producer_KafkaProducer.java-23L832&d=DwIFaQ&c=RoP1YumCXCgaWHvlZYR8PZh8Bv7qIrMUB65eapI_JnE&r=vmJiAMDGSxNeZnFztNs5ITB_i_Z3h3VtLPGma9y7cKI&m=qtRoal09Ax8f1wskhpGkLJz8loX98EAVCX95pMjnI8s&s=xruUiNP1BFXu6CziC0aB00HcoX7GyH8HNalyLp-CYlI&e= > > > > 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 > >>>>>> > >>>>>> > >>>> > >>>> > >> > >> > >