Hi Andrew,

Thank you for the detailed breakdown, especially the distinction between making 
the ShareConsumer transactional (Option A) vs. the Producer-assisted model 
(Option B).

I agree that Option B is the superior path for architectural consistency. By 
having the Producer "assist" the ShareConsumer, we can leverage the existing 
mechanisms.

I am looking at implementing a prepareAcknowledgementsForTransaction() method 
on the KafkaShareConsumer. This would return a ShareGroupTxnContext containing 
the MemberID and the specific record acquisitions. The KafkaProducer would then 
use this context to trigger the AddShareAcksToTxn RPC.
Regarding your points on KIP-1191 discussion thread, this Option B approach 
also allows us to solve the "Atomic DLQ" problem. If we include the DLQ write 
in the Producer's transaction, the Share Coordinator can wait for the 
WriteTxnMarker before finalising the move to the ARCHIVED state. This ensures 
the 1:1 audit trail remains intact even during broker failures.
I have updated the KIP and will keep improving/rephrasing it to clarify 
implementation: https://cwiki.apache.org/confluence/x/J448G 
 I am also starting to explore the broader challenge of Atomicity for 
cross-cluster scenarios. While standard transactions are intra-cluster, I am 
looking into "Acknowledgment Tokens" that could be recorded on a target cluster 
and mirrored back to the source coordinator to finalize state. I’d be 
interested to hear if the community sees value in laying the groundwork for 
that in this KIP.



Regards,Shekhar Prasad Rajak,




 

    On Sunday 1 March 2026 at 11:09:59 pm GMT+5:30, Shekhar Prasad Rajak via 
dev <[email protected]> wrote:  
 
 Thank you Andrew for the review.
It took sometime to analyse the review comments and understand Kafka Producer 
2PC implementation in depth.All the points are strongly valid and I have 
updated the KIP to adopt the RPC already exists and leverage the same methods 
for , CTP pattern  (Consume-Transform-Produce) for kafka as sink or no producer 
usecase.
Please have a look: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1289+Support+Hierarchical+Transactional+Acknowledgments+for+Share+Groups
 



Regards,Shekhar Prasad Rajak,




 

    On Thursday 26 February 2026 at 03:32:36 am GMT+5:30, Andrew Schofield 
<[email protected]> wrote:  
 
 Hi Shekhar,
Thanks for the KIP. Adding transactional support for acknowledgements is one of 
the future features I'd like to see for share groups, so I'm glad you've 
started this.

Here are some comments from an initial read.

AS1: Previously, Kafka transactions have always been a producer concept. You 
set a configuration called `transactional.id` in the producer configuration 
which gives authorization checking and helps with fencing. Then you use 
KafkaProducer methods like beginTransaction and commitTransaction to mark the 
start and end of the transaction. Only the producer is currently enabled for 
transactional operation, and the Kafka RPCs it uses include transactional ID as 
necessary to indicate operations which are transactional.

There will be differences for sure with share groups, but the overall model 
should probably still hold. Over the years, we've gradually tightened up the 
transaction protocol by adding features such as the producer epoch and 
two-phase commit. I would rather not re-invent everything. For example, 
WriteTxnMarkers is used by the transaction coordinator to tell the group 
coordinator to complete transactional operations on the __consumer_offsets 
topic. I would expect a similar kind of interaction with the share coordinator 
on the __share_group_state topic.

I suggest looking at how to adapt the existing Kafka protocol RPCs rather than 
defining new ones.

AS2: Why is there an acknowledge(ConsumerRecord<,>, AcknowledgeType, String) 
method? I would have expected that the transactional ID is a configuration, not 
a parameter of this method.

AS3: Have you considered how to write an application which uses a share group 
and producer, and coordinates their operations in the same transaction?


Thanks,
Andrew

On 2026/02/24 08:54:36 Shekhar Prasad Rajak via dev wrote:
> Hello team,
> In this discussion thread we will finalise the design for the KIP-1289 
> Support Hierarchical Transactional Acknowledgments for Share Groups. 
> 
> Share Groups currently support only immediate acknowledgement modes 
> (IMPLICIT/EXPLICIT). This causes data loss in distributed streaming 
> frameworks:
> 
> 1. Worker acknowledges records → Records removed from Kafka
> 2. Checkpoint fails before sink write
> 3. Records lost (acknowledged but never persisted)
> 
> Goal: Enable exactly-once read semantics via transactional acknowledgements.
> We already have similar pattern in Kafka producer transaction
> 
> KIP: KIP-1289 Support Hierarchical Transactional Acknowledgments for Share 
> Groups - Apache Kafka - Apache Software Foundation 
> 
> Looking forward to community's feedback! 
> 
> 
> Regards,Shekhar Prasad Rajak
> 
> 
    

Reply via email to