Guaranteeing cross-session idempotency with KIP-98

2017-01-26 Thread Eugen Dueck
KIP-98 says > transaction.app.id: A unique and persistent way to identify a producer. This is used to ensure idempotency and to enable transaction recovery or rollback across producer sessions. This is optional: you will lose cross-session guarantees if this is blank. which might suggest th

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-29 Thread Eugen Dueck
Thanks Apurva - replies inline. On 2017年01月27日 15:19, Apurva Mehta wrote: Eugen, moving your email to the main thread so that it doesn't get split. The `transaction.app.id` is a prerequisite for using transactional APIs. And only messages wrapped inside transactions will enjoy idempotent guaran

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Eugen Dueck
> The PID and sequence numbers are totally transparent to applications. Now that you say it, the cwiki makes that point already pretty clear by the way the Producer API is (not) being changed. Sorry for taking your time on this. In other words, at the point where messages enter the Kafka syst