Hi Shekhar Prasad Rajak, > Don't we have timeout, session timeout at kafka side ?
Yes, Kafka does have a timeout and when the timeout fires, the transaction is aborted by the broker. However, if Flink successfully completed a checkpoint but didn't commit the transaction (e.g. the JVM crashed before committing the transaction to Kafka), the data should be committed to satisfy 2PC protocol promises. Relying on the timeout may belong to data loss in this scenario. Moreover, some Flink-Kafka setups use high values for Kafka transaction timeouts, in that case, even transactions that must be aborted (because they were not checkpointed), will not be aborted until timeout fires, blocking the LSO. The tool provides a way to resolve this immediately. Kind regards, Aleksandr On Wed, 8 Apr 2026 at 05:34, Shekhar Rajak <[email protected]> wrote: > > Thanks for clarifying but > > it holds the transaction open because it has noway to know whether the > > data should be committed or aborted. That > decision belongs to Flink, which is no longer running. > > Don't we have timeout, session timeout at kafka side ? After enabling 2PC we > must be using those config to make sure Kafka broker releases the locks and > have durability. > > Regards, > Shekhar Prasad Rajak > > > On Monday 30 March 2026 at 04:34:07 pm GMT+5:30, Aleksandr Savonin > <[email protected]> wrote: > > Hi Shekhar Rajak, > Good questions. > Let me clarify the transaction coordination model and why a > Flink-specific tool is needed. > > > Curious to know if we debug and analyse the root cause and try to > > fix/enhance in the Kafka transaction coordinator side, will this issue be > > resolved ? > > > Kafka has a built-in Transaction Coordinator, but it only manages the > Kafka protocol layer, it executes commit/abort when told to. > For exactly-once delivery, Flink acts as the orchestrator on top of > Kafka's transactions, deciding when to commit based on checkpoint > completion. The problem arises when Flink disappears (there is no > running Flink job anymore) and nobody is left to tell Kafka's > coordinator to commit or abort. Kafka's coordinator in this case is > behaving as designed - it holds the transaction open because it has no > way to know whether the data should be committed or aborted. That > decision belongs to Flink, which is no longer running. > > > Do we really need to think about transaction management tool specifically > > for kafka ? > > > The Kafka sink is one of the most widely used Flink connectors, and > exactly-once delivery with Kafka transactions is a primary production > use case in some companies. > > -- > Kind regards, > Aleksandr > -- Kind regards, Aleksandr
