Hi Aleksandr,

Thanks for your hard work on this CLI and for proactively surfacing these
production scenarios. I really appreciate your contributions.

Heise's work under FLIP-511 has already addressed most client-side
transaction issues. Are you still hitting blockers in production?

The CLI logic seems to duplicate what already runs during Flink Writer
restart/recovery. Could you clarify why a separate CLI is needed instead of
relying on the native recovery path? And I also don't think flink dist
needs a cli about a connector's cli which is better to add in  flink kafka
connector' repo.

One known gap remains: server-side transaction timeouts  without
ADD_PARTITIONS_TO_TXN , and server-side terminates the transaction. This
cannot be fixed client-side and only a Kafka broker update (KIP-890 [2])
will resolve it.


[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-511%3A+Support+transaction+id+pooling+in+Kafka+connector?src=contextnavpagetreemode
[2]
https://cwiki.apache.org/confluence/display/KAFKA/KIP-890%3A+Transactions+Server-Side+Defense

On Sat, Mar 28, 2026 at 8:12 PM Shekhar Rajak <[email protected]>
wrote:

> Hi,
> 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 ? Do we really need to think about transaction management tool
> specifically for kafka ?
> Regards,
> Shekhar Rajak,
>
>
>
>
>     On Wednesday 25 March 2026 at 05:51:20 pm GMT+5:30, Aleksandr Savonin <
> [email protected]> wrote:
>
>  Hi everyone,
> I'd like to start a discussion on FLIP-572 [1].
> When a Flink job using exactly-once KafkaSink fails and does not recover,
> Kafka transactions can remain in the ONGOING state, blocking all downstream
> read_committed consumers at the Last Stable Offset until the broker timeout
> expires.
> There is currently no built-in tooling to resolve this, Kafka's own
> kafka-transactions.sh cannot commit Flink transactions since that requires
> Flink-specific internals.
> This FLIP proposes a standalone CLI tool, that allows operators to abort or
> commit lingering transactions without a running Flink cluster.
> Looking forward to your feedback.
>
> Kind regards,
> Aleksandr Savonin
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-572%3A+Introduce+Flink-Kafka+Transactions+Management+Tool
>

Reply via email to