Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-19 Thread Rowland Smith
way transactions. > > On the downside, an explicit prepare RPC would have a performance hit on > the happy path in every single transaction. > > -Artem > > On Tue, Feb 6, 2024 at 7:35 PM Rowland Smith wrote: > > > Hi Artem, > > > > I am not sure what you me

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-06 Thread Rowland Smith
antages such as > elimination the needs for monitoring, tooling or providing additional > guarantees. Let me know if you think of a guarantee that prepare RPC would > provide. > > -Artem > > On Mon, Feb 5, 2024 at 6:22 PM Rowland Smith wrote: > > > Hi Artem, > > >

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-05 Thread Rowland Smith
separate store, say in a compacted topic. This way, the > core Kafka protocol could be decoupled from specific implementations (and > extra performance requirements that a specific implementation may impose) > and serve as a foundation for multiple implementations. > > -Artem > > On

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-04 Thread Rowland Smith
s. > > -Artem > > On Thu, Jan 4, 2024 at 6:14 PM Rowland Smith wrote: > > > It is probably me. I copied the original message subject into a new > email. > > Perhaps that is not enough to link them. > > > > It was not my understanding from reading KIP-939 that

Permission to contribute

2024-01-13 Thread Rowland Smith
I would like permission to contribute to Kafka. I have created Wiki and Jira ID's 'rowls'. I will be working with a KIP for XA support. -- *Rowland E. Smith* P: (862) 260-4163 M: (201) 396-3842

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-01-04 Thread Rowland Smith
wanted to > clarify what the current KIP is proposing. > > Thanks, > Justine > > On Wed, Jan 3, 2024 at 7:49 PM Rowland Smith wrote: > > > Hi Artem, > > > > I saw your response in the thread I started discussing Kafka distributed > > transaction suppo

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-01-03 Thread Rowland Smith
Hi Artem, I saw your response in the thread I started discussing Kafka distributed transaction support and the XA interface. I would like to work with you to add XA support to Kafka on top of the excellent foundational work that you have started with KIP-939. I agree that explicit XA support

[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