Hi David,

Thanks for the KIP!
It's a light-weight transactional proposal for single producer, cool!
+1 for it!

Luke


On Sat, Sep 10, 2022 at 1:14 AM David Arthur <davidart...@apache.org> wrote:

> Starting a new thread to avoid issues with mail client threading.
>
> Original thread follows:
>
> Hey folks, I'd like to start a discussion on the idea of adding
> transactions in the KRaft controller. This will allow us to overcome
> the current limitation of atomic batch sizes in Raft which lets us do
> things like create topics with a huge number of partitions.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-868+Metadata+Transactions
>
> Thanks!
>
> ---
>
> Colin McCabe said:
>
> Thanks for this KIP, David!
>
> In the "motivation" section, it might help to give a concrete example
> of an operation we want to be atomic. My favorite one is probably
> CreateTopics since it's easy to see that we want to create all of a
> topic or none of it, and a topic could be a potentially unbounded
> number of records (although hopefully people have reasonable create
> topic policy classes in place...)
>
> In "broker support", it would be good to clarify that we will buffer
> the records in the MetadataDelta and not publish a new MetadataImage
> until the transaction is over. This is an implementation detail, but
> it's a simple one and I think it will make it easier to understand how
> this works.
>
> In the "Raft Transactions" section of "Rejected Alternatives," I'd add
> that managing buffering in the Raft layer would be a lot less
> efficient than doing it in the controller / broker layer. We would end
> up accumulating big lists of records which would then have to be
> applied when the transaction completed, rather than building up a
> MetadataDelta (or updating the controller state) incrementally.
>
> Maybe we want to introduce the concept of "last stable offset" to be
> the last committed offset that is NOT part of an ongoing transaction?
> Just a nomenclature suggestion...
>
> best,
> Colin
>

Reply via email to