Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-04-07 Thread Hongshun Wang
ng shutdown.= > > > > > > > > In the case of COORDINATOR_LOAD_IN_PROGRESS I guess we could have > some > > > sort of retry loop – governed by a configuration parameter. > > > > > > > > * Do we need to do anything special

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-04-06 Thread Arvid Heise
e debugging would be > > more straight forward. Better still cpi;d be use a uuid as part of the > > identifier; this would seem more normal practise rather than managing an > > incrementing counter and filling the gaps as they become available. > > > * I am not very fami

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-04-06 Thread Hongshun Wang
t you mean by > “ it also knows which transactions still need to be re-committed. It aborts > all other open transactions.”. When/why do we need to re-commit > transactions and when do we abort? > > * I wonder why “we expect 3 transactional ids to be in use per > subtask”. >

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-04-05 Thread Arvid Heise
in state CompleteCommit? Can we not > be the first transaction so have no older ones – or have I misunderstood? > > Kind regards, David. > > > From: Arvid Heise > Date: Wednesday, 26 March 2025 at 06:49 > To: dev@flink.apache.org > Subject: [EXTERNAL] Re: [DISCUSS] F

RE: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-03-26 Thread David Radley
CompleteCommit? Can we not be the first transaction so have no older ones – or have I misunderstood? Kind regards, David. From: Arvid Heise Date: Wednesday, 26 March 2025 at 06:49 To: dev@flink.apache.org Subject: [EXTERNAL] Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-03-25 Thread Arvid Heise
Hi Hongshun, this thread is about addressing your first pain point. It would be awesome if you could read the FLIP and provide feedback whether this approach would work in your setup. The other issues won't need a FLIP but need a smaller bugfix. Best, Arvid On Tue, Mar 25, 2025 at 7:57 AM Hong

Re: [DISCUSS] FLIP-511: Support transaction id pooling in Kafka connector

2025-03-24 Thread Hongshun Wang
Hi heise, Thanks for pushing this job. Currently, kafka sinks in transactions are not productive, which will influence the whole kafka cluster. As we discussed before, I have three problems when using Kafka exactly-once sink : 1. Too much transaction id https://issues.apache.org/jira/browse/FLINK-