Re: 2-phase commit and kafka

2021-04-17 Thread Vishal Santoshi
Thank you for the clarification. That is what I was thinking. Regards. On Sat, Apr 17, 2021 at 3:24 AM Arvid Heise wrote: > Hi Vishal, > > no afaik Flink only cancels on restore from a checkpoint or savepoint > (manual or automatic). That's what I meant with > >> they may linger for a longer t

Re: 2-phase commit and kafka

2021-04-17 Thread Arvid Heise
Hi Vishal, no afaik Flink only cancels on restore from a checkpoint or savepoint (manual or automatic). That's what I meant with > they may linger for a longer time if you stop an application entirely (for > example for an upgrade). > Your upgraded application will hopefully be up before Kafka t

Re: 2-phase commit and kafka

2021-04-16 Thread Vishal Santoshi
Thanks for the feedback and. glad I am on the right track. > Outstanding transactions should be automatically aborted on restart by Flink. Let me understand this 1. Flink pipe is cancelled and has dangling kafka transactions. 2. A new Flink pipe ( not restored from a checkpoint or sp ) is start

Re: 2-phase commit and kafka

2021-04-16 Thread Arvid Heise
Hi Vishal, I think you pretty much nailed it. Outstanding transactions should be automatically aborted on restart by Flink. Flink (re)uses a pool of transaction ids, such that all possible transactions by Flink are canceled on restart. I guess the biggest downside of using a large transaction ti

2-phase commit and kafka

2021-04-16 Thread Vishal Santoshi
Hello folks So AFAIK data loss on exactly once will happen if - start a transaction on kafka. - pre commit done ( kafka is prepared for the commit ) - commit fails ( kafka went own or n/w issue or what ever ). kafka has an uncommitted transaction - pipe was down for