Hi Sushant, Thanks for the updates and answers.
Follow-up on LM3 and LM4 (memory and client-side buffering), no more concerns. The important bit that wasn't clear to me is that the broker will not return new records on a ShareFetch response to a RENEW. That is really what ensures that the client-side buffering will remain bounded (to the initial batch that is being RENEWED and kept on the client). LM5: The new acquisitionLockTimeout exposed in the ShareConsumer API is defined as Optional. I imagine that the reason is only because we will not know about the acquisition until we get a ShareFetchResponse, correct? (With the acquisition timeout always known, but on the broker). Just to double check my understanding on this decision. LM6: About the head-of-line blocking concern, I agree with the approach of not imposing any limits initially, and that we surely need visibility too. Did we consider some metrics? Ex. a high level view of how often an app is having to RENEW records? I imagine that a basic renewal rate, telling us that a client app is having to RENEW 90% of the batches will be a helpful warning sign. Or the average time records remain being RENEWED would probably suggest how to tune the acquisition timeout. This can surely be considered as a follow-up depending on how far we want to go. Thanks! Lianet On Thu, Oct 16, 2025 at 3:21 PM Andrew Schofield <[email protected]> wrote: > > Hi Sushant, > Thanks for the updates. > > AS5: You mention that a broker which does not support v2 of > ShareFetch/Acknowledge > would not be able to support RENEW. You then mention that the exception > COULD be UnsupportedVersionException. It would be good if the KIP > specified the exception to be used explicitly. > > In practice, this will be very unlikely to occur, assuming that this KIP > is delivered in Apache Kafka 4.2 because earlier versions are not > production-ready. As a result, I would not define an exception specifically > for this case and UnsupportedVersionException with an appropriate > message should suffice. > > AS6: In the section on broker-side changes, you say that a ShareFetch > which includes RENEW acknowledgements will not return any records. > Also, it will return as soon as the acknowledgements have been > processed, regardless of the value of MaxWaitMs in the request. > We want to return the acknowledgement error code promptly without > running down the renewed acquisition lock time. > > > Thanks, > Andrew > > > On 13 Oct 2025, at 09:51, Andrew Schofield <[email protected]> > > wrote: > > > > Hi Sushant, > > Thanks for the updates. > > > > AS4: Adding UnsupportedAcknowledgeTypeException is OK, but > > I don’t think it’s correct to inherit from InvalidConfigurationException. > > Because that’s an ApiException, it’s related to a specific error code > > in the Kafka protocol. I don’t think we want to introduce a new > > error code in the Kafka protocol, which would imply that the broker > > is able to return it. > > > > Two options given that. First, you could instead extend KafkaException > > for your new exception class. Second, you could just use the > > standard Java IllegalArgumentException if the acknowledge type is > > not supported. > > > > > > Thanks, > > Andrew > > > >> On 9 Oct 2025, at 11:48, Sushant Mahajan <[email protected]> wrote: > >> > >> Hi Andrew, > >> Thanks for the suggestions. I have incorporated all of them and broken down > >> proposed changes into two sections for better readability. > >> > >> For AS2: The proposal is to buffer renew acked records on the share > >> consumer side and give them to the application appended to subsequent poll > >> results. This will happen until the record is re-delivered by the broker > >> after timeout. > >> > >> Regards, > >> Sushant Mahajan > >> > >> > >> On Thu, 9 Oct 2025, 00:58 Andrew Schofield, < > >> [email protected]> wrote: > >> > >>> Hi Sushant, > >>> Thanks for the KIP. I tried replying previously but messed up so let's try > >>> again. > >>> > >>> AS1: In the section of Proposed Changes, the KIP states that the > >>> acknowledge(ConsumerRecord, AcknowledgementType) method > >>> causes RPCs to be sent. This is incorrect. Only the poll and commit > >>> methods do this. > >>> > >>> AS2: For a situation in which the application is processing for an > >>> extended period, I would expect it to continue to call > >>> ShareConsumer.poll(Duration) repeatedly. This method returns > >>> a set of records to process. In the case where the application > >>> has renewed its acquisition locks, I would expect the renewed > >>> records to be returned from the next call to poll(Duration) as a > >>> way of confirming which records are still in the process of being > >>> delivered. > >>> > >>> AS3: We should specify the error handling where the application > >>> tries to renew but the broker does not support v2 of the updated > >>> RPCs. I think ShareConsumer.acknowledge() should throw > >>> an exception in this case. > >>> > >>> Thanks, > >>> Andrew > >>> ________________________________________ > >>> From: Sushant Mahajan <[email protected]> > >>> Sent: 07 October 2025 11:16 > >>> To: [email protected] <[email protected]> > >>> Subject: [DISCUSS] KIP-1222: Acquisition lock timeout renewal in share > >>> consumer explicit mode > >>> > >>> I’d like to start the discussion for KIP-1222: Acquisition lock timeout > >>> renewal in share consumer explicit mode. > >>> > >>> JIRA: https://issues.apache.org/jira/browse/KAFKA-19742 > >>> KIP Wiki: > >>> > >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1222%3A+Acquisition+lock+timeout+renewal+in+share+consumer+explicit+mode > >>> > >>> Regards, > >>> Sushant Mahajan > >>> > > >
