Hi Christo, This specific impl was preferred because the ShareAcquireMode values introduced in kip-1206 are not applicable to the case where a ShareFetch request contains RENEW acks. The 2 values defined in kip-1206 apply to fetching records in poll where the number of records may or may not exceed max.poll.records.
When ShareFetch contains a RENEW ack, none of the values of share acquire mode apply. So defining a new value here is logical and serves the dual purpose of signalling the broker to not send back records and return immediately after renewing and having a valid value for ShareAcquireMode in case of ShareFetch renewal. Regards, Sushant Mahajan On Wed, 22 Oct 2025, 19:09 Christo Lolov, <[email protected]> wrote: > Heya, > > Thanks for the quick reply! > > I understand. Is there a reason why you chose not to introduce a new > field instead? > > Best, > Christo > > On Wed, 22 Oct 2025 at 13:11, Sushant Mahajan <[email protected]> > wrote: > > > > Hi Christo, > > We chose this as this as the new field has nothing to with acquisition of > > records and it won't be added to the ShareAcquireMode enum either. > > > > 255 was chosen with the intention that if some new modes are added to the > > enum later, they their ids would line up properly. > > > > Though the value is part of acquiremode field, its purpose is quite > > different. > > > > On Wed, 22 Oct 2025, 17:29 Christo Lolov, <[email protected]> > wrote: > > > > > Hello! > > > > > > Thanks for the contribution! > > > > > > A small question from my side - what is the reasoning behind using 255 > > > rather than 2? > > > > > > Best, > > > Christo > > > > > > On Wed, 22 Oct 2025 at 12:16, Lianet M. <[email protected]> wrote: > > > > > > > > Thanks for the updates Sushant. No more comments from me. > > > > > > > > Best, > > > > Lianet > > > > > > > > On Tue, Oct 21, 2025, 10:26 a.m. Andrew Schofield < > > > [email protected]> > > > > wrote: > > > > > > > > > Hi Sushant, > > > > > Thanks for the updates. The way the consumer and broker communicate > > > > > the new behaviour is clearer and easier to validate in the latest > > > update. > > > > > > > > > > No more comments from me. > > > > > > > > > > Thanks, > > > > > Andrew > > > > > > > > > > > On 21 Oct 2025, at 11:58, Sushant Mahajan <[email protected]> > > > wrote: > > > > > > > > > > > > Hi Andrew, > > > > > > Thanks for the additional insights. Have incorporated those. > > > > > > > > > > > > AS5: wholly incorporated (will use ERRORS.UNSUPPORTED_VERSION > (35)) > > > > > > > > > > > > AS6: Have mentioned about ignored/zeroed fields. > > > > > > > > > > > > Regards, > > > > > > Sushant Mahajan > > > > > > > > > > > > On Fri, 17 Oct 2025, 00:51 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 > > > > > >>>>> > > > > > >>> > > > > > >> > > > > > >> > > > > > > > > > > > > > >
