Re: Data loss after a Kafka Broker restart scenario

2017-05-24 Thread Jun Rao
Hi, Fathima, Yes, the most efficient way to verify if a message is sent successfully is through the producer callback. You can take a look at PrintInfoCallback in org.apache.kafka.toolsVerifiableProducer as an example. Our system tests use that to verify if any data loss has occurred. Thanks, Ju

Re: Data loss after a Kafka Broker restart scenario

2017-05-22 Thread Fathima Amara
Hi Jun, Do you mean by using CallBack mechanism? Since I am new to kafka would you mind directing me how to do it if it's not to be done using CallBack? Fathima.

Re: Data loss after a Kafka Broker restart scenario

2017-05-19 Thread Jun Rao
Hi, Fathima, Did you check if produced messages are acked successfully? Only successfully acked messages are guaranteed to be preserved on the broker. Thanks, Jun On Thu, May 18, 2017 at 12:10 AM, Fathima Amara wrote: > Hi Jun, > > Thanks alot for the reply. As suggested, I tried running my a

Re: Data loss after a Kafka Broker restart scenario

2017-05-18 Thread Fathima Amara
Hi Jun, Thanks alot for the reply. As suggested, I tried running my application in the trunk. But I still do encounter data loss! Are there any specific configs that I need to change from its default value? What could be the reason for this? Fathima

Re: Data loss after a Kafka broker restart scenario.

2017-05-17 Thread Tom Crayford
Fathima, In 0.11 there will be such a mechanism (see KIP-98), but in current versions, you have to eat the duplicates if you want to not lose messages. On Wed, May 17, 2017 at 5:31 AM, Fathima Amara wrote: > Hi Mathieu, > > Thanks for replying. I've already tried by setting "retries" to higher

Re: Data loss after a Kafka broker restart scenario.

2017-05-16 Thread Fathima Amara
Hi Mathieu, Thanks for replying. I've already tried by setting "retries" to higher values(maximum up to 3). Since this introduces duplicates which I do not require, I brought the "retries" value back to 0. I would like to know whether there is a way to achieve "exactly-once" guarantee having in

Re: Data loss after a Kafka Broker restart scenario

2017-05-16 Thread Jun Rao
Hi, Fathima, There is a known data loss issue that's described in KIP-101 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation). The issue happens rarely, but has been exposed in some of our system te

Re: Data loss after a Kafka broker restart scenario.

2017-05-16 Thread Mathieu Fenniak
Hi Fathima, Setting "retries=0" on the producer means that an attempt to produce a message, if it encounters an error, will result in that message being lost. It's likely the producer will encounter intermittent errors when you kill one broker in the cluster. I'd suggest trying this test with a