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-34554
2. Too much unclosed producer ( if commit is lag too much from pre-commit:
https://issues.apache.org/jira/browse/FLINK-36569
3. maybe Invalid state: https://issues.apache.org/jira/browse/FLINK-37356

I do hope this flip can show how to solve this problem.

Best
Hongshun Wang



On Fri, Feb 28, 2025 at 10:01 PM Arvid Heise <ar...@apache.org> wrote:

> Dear Flink devs,
>
> I'd like to start discussion around an improvement of the exactly-once
> Kafka sink. Because of its current design, the sink currently puts a
> large strain on the main memory of the Kafka broker [1], which hinders
> adoption of the KafkaSink. Since the KafkaSink will be the only way to
> produce into Kafka for Flink 2.0, this limitation may in turn also
> limit Flink 2 adoption.
>
> I appreciate feedback not only on technical aspects but also on the
> requirements of the new approach [2]:
> - Are most users/customers already on Kafka 3.0 broker?
> - The new approach requires READ permission on the target topic (for
> DescribeProducer API) and DESCRIBE permission on the transaction (for
> ListTransaction API). Are there any concerns around this?
>
> The new behavior would be opt-in in the first phase while the current
> behavior remains the DEFAULT. If the community accepts the new
> behavior, we can turn it on by default and provide an option to
> opt-out.
>
> Note that this improvement is borderline FLIP-worthy; it touches the
> public interfaces of the connector only. However, by doing an official
> FLIP, I'm positive that we can gather feedback more quickly.
>
> I'm also looking for volunteers that test the implementation on their
> dev clusters [3].
>
> Best,
>
> Arvid
>
> [1] https://issues.apache.org/jira/browse/FLINK-34554
> [2]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-511%3A+Support+transaction+id+pooling+in+Kafka+connector
> [3] https://github.com/apache/flink-connector-kafka/pull/154
>

Reply via email to