Re: Kafka transaction error lead to data loss under end to end exact-once

2020-08-05 Thread Piotr Nowojski
Hi Lu, In this case, as it looks from the quite fragmented log/error message that you posted, the job has failed so Flink indeed detected some issue and that probably means a data loss in Kafka (in such case you could probably recover some lost records by reading with `read_uncommitted` mode from

Re: Kafka transaction error lead to data loss under end to end exact-once

2020-08-05 Thread Khachatryan Roman
Hi Lu, AFAIK, it's not going to be fixed. As you mentioned in the first email, Kafka should be configured so that it's transaction timeout is less than your max checkpoint duration. However, you should not only change transaction.timeout.ms in producer but also transaction.max.timeout.ms on your

Re: Kafka transaction error lead to data loss under end to end exact-once

2020-08-04 Thread Lu Niu
Hi, Khachatryan Thank you for the reply. Is that a problem that can be fixed? If so, is the fix on roadmap? Thanks! Best Lu On Tue, Aug 4, 2020 at 1:24 PM Khachatryan Roman < khachatryan.ro...@gmail.com> wrote: > Hi Lu, > > Yes, this error indicates data loss (unless there were no records in th

Re: Kafka transaction error lead to data loss under end to end exact-once

2020-08-04 Thread Khachatryan Roman
Hi Lu, Yes, this error indicates data loss (unless there were no records in the transactions). Regards, Roman On Mon, Aug 3, 2020 at 9:14 PM Lu Niu wrote: > Hi, > > We are using end to end exact-once flink + kafka and encountered belowing > exception which usually came after checkpoint failur

Kafka transaction error lead to data loss under end to end exact-once

2020-08-03 Thread Lu Niu
Hi, We are using end to end exact-once flink + kafka and encountered belowing exception which usually came after checkpoint failures: ``` *Caused by: org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an operation with an old epoch. Either there is a newer pro