Hi all,

Following the discussion thread [1], I'd like to start a vote on
KIP-1289: Support Transactional Acknowledgments for Share Groups.

KIP wiki:
KIP-1289 Support Transactional Acknowledgments for Share Groups - Apache Kafka 
- Apache Software Foundation 



 TL;DR
KIP-1289 adds the share-group equivalent of `sendOffsetsToTransaction`,
enabling exactly-once semantics for share-group-based read-process-write
pipelines. The proposal introduces:

- A new producer API:
  `producer.sendShareAcknowledgementsToTransaction(acks, shareGroupMetadata)`
- A new RPC: `TxnShareAcknowledgeRequest` 
- A new record state: `TX_PENDING` in the share-group state machine
- Broker-side staging via per-partition 2PC, finalized by the existing
  WriteTxnMarkers fanout

The design follows the same protocol-version-aware (TV1/TV2 - KIP-890 + 
KIP-939) and
producer-fencing patterns as the existing `sendOffsetsToTransaction`
flow, so it inherits all of TV2's latency improvements automatically.


A working proof-of-concept branch demonstrates the end-to-end flow,

https://github.com/apache/kafka/pull/22357 

The polished PR will follow once voting concludes.



Thanks,
Shekhar Rajak

[1] https://lists.apache.org/thread/mn1fn8t01n1mzn0tmh528mbpmqllbo31 


Reply via email to