Hi Guy, You raise a very good point. Supporting XA sounds like a good way to integrate Kafka and it's something that I think we should support at some point in the future. For this KIP, though, we thought we focus on a more basic functionality keeping the following in mind:
1. XA is not universally supported and it would be good to integrate with systems that just have local transactions support (which would include systems that are not "proper" databases, like Zookeeper, RocksDB, etc.). 2. More advanced functionality, like XA, we should be able to implement on top of KIP-939 as a library or a recipe. I would like to hear your thoughts on #2 specifically -- do you think that we actually need to amend KIP-939 to enable XA in the future, or could the XA changes be done incrementally on top of KIP-939? -Artem On Wed, Aug 23, 2023 at 5:23 AM <g...@atomikos.com> wrote: > Hi, > > Nice idea, but you could maximise compatibility if you adhere to XA > standard APIs rather than Kafka internal APIs. > > We at Atomikos offer 2PC coordination and recovery and we are happy to > help you design this, it's a service we usually offer for free to backend > vendors / systems. > > Let me know if you'd like to explore? > > Guy > > > On 2023/08/17 06:39:57 Artem Livshits wrote: > > Hello, > > > > This is a discussion thread for > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC > > . > > > > The KIP proposes extending Kafka transaction support (that already uses > 2PC > > under the hood) to enable atomicity of dual writes to Kafka and an > external > > database, and helps to fix a long standing Flink issue. > > > > An example of code that uses the dual write recipe with JDBC and should > > work for most SQL databases is here > > https://github.com/apache/kafka/pull/14231. > > > > The FLIP for the sister fix in Flink is here > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255071710 > > > > -Artem > > > Sent with Spark >