Re: [DISCUSS] Kafka distributed transaction support

2024-01-03 Thread Artem Livshits
Hi Rowland, KIP-939 provides a foundation for using a two-phase commit protocol with Kafka (allows it to be a participant) that can be used to implement various concrete protocols, such as XA but not only XA. The benefit of supporting a foundational construct (and not just one concrete protocol

Re: [DISCUSS] Kafka distributed transaction support

2024-01-02 Thread Justine Olshan
I believe Artem also had some conversations offline about XA. If I recall correctly, he didn't plan to include it in KIP-939 but was happy to leave room for potential KIPs in the future. Please feel free to continue the conversation on the thread. :) Justine On Tue, Jan 2, 2024 at 12:05 PM Greg

Re: [DISCUSS] Kafka distributed transaction support

2024-01-02 Thread Greg Harris
Hi Rowland, First of all, welcome to the community, and thanks for thinking about the future of Kafka! I'm not very familiar with X/Open XA, but from the documentation I read, it appears most related to KIP-939: Support Participation in 2PC [1] currently in-progress. You may be interested in cont

Re: [DISCUSS] Kafka distributed transaction support

2024-01-02 Thread Andrew Schofield
Hi Rowland, KIP-939 https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC introduces two-phase commit, but not XA support. I believe the KIP is expected to be delivered in Kafka 4.0 later this year. The way that it works and the use cases that it’s intended fo

[DISCUSS] Kafka distributed transaction support

2024-01-01 Thread Rowland Smith
Hi All, I am new to the Kafka developer community. After learning more about Kafka's transactional capabilities recently, I became interested in exploring what would be required to provide support for the XA interface specified in the X/ Open Distributed Processing Model in the Kafka producer clie