Hi, Thanks for the questions and your interest in KIP-932. 1) I would expect each application thread using the share consumer to have its own instance of KafkaShareConsumer. I think it would be a bad idea to have one instance able to return multiple batches which would then be currently "in-flight". The state management in both the application and the KafkaShareConsumer would get tricky I think.
In Kafka 4.0, we had the behaviour where it was permitted in explicit mode not to acknowledge all records before calling ShareConsumer.poll(Duration). The behaviour was that the unacknowledged ConsumerRecord instances would be returned repeatedly until they'd all been acknowledged. Users of the API found this difficult to work with. It made it quite easy to make an application which accidentally didn't acknowledge anything and just was essentially a futile infinite loop. This is why the current behaviour is very prescriptive and the rules are very tight. If you get it wrong, you get an exception. Then you fix the code and it works. 2) I can see the value in ShareConsumer.acknowledgeAll but I think it's pretty simple to loop over the records with the existing interface. 3) We have already added another overload of ShareConsumer.acknowledge. void acknowledge(String topic, int partition, long offset, AcknowledgeType type) This will be in Kafka 4.2. Thanks, Andrew Schofield ________________________________________ From: RShekhar Prasad <rshekhar.pra...@gmail.com> Sent: 29 August 2025 07:51 To: dev@kafka.apache.org <dev@kafka.apache.org> Cc: apoorvmitta...@gmail.com <apoorvmitta...@gmail.com> Subject: Re: Kafka 4.1 Queue Semantics missing APIs 3. Acknowledge the records based on some ID like offset shareConsumer.acknowledge(record.offset(), ACCEPT); Regards, Shekhar Prasad Rajak, Contact : +918142478937Blog | Github | Twitter On Friday 29 August 2025 at 11:37:24 am GMT+5:30, RShekhar Prasad <rshekhar.pra...@gmail.com> wrote: Hello team, While exploring Kafka Queue semantics, we released there is some APIs that is useful but missing for example : 1. Multiple polls and ack later batch wise in explicit mode : ConsumerRecords<K, V> batch1 = shareConsumer.poll(Duration.ofMillis(100)); ConsumerRecords<K, V> batch2 = shareConsumer.poll(Duration.ofMillis(100)); // No IllegalStateException! ConsumerRecords<K, V> batch3 = shareConsumer.poll(Duration.ofMillis(100)); // Process all batches... shareConsumer.acknowledgeBatch(batch1, AcknowledgeType.ACCEPT); shareConsumer.acknowledgeBatch(batch2, AcknowledgeType.ACCEPT); shareConsumer.acknowledgeBatch(batch3, AcknowledgeType.ACCEPT); shareConsumer.commitSync(); // Commit all acknowledged batches If we have these APIs available then multiple threads can keep polling and committing once their processing is done, before the timeout (and release). In Implicit mode, we might miss(fail) the processing of the record but ACCEPT acknowledge in each poll(), hence that will not help in at least once semantics. 2. ack all API: shareConsumer.acknowledgeAll(AcknowledgeType.ACCEPT); shareConsumer.commitSync(); Do you guys have any thought on this ? Regards, Shekhar Prasad Rajak, Blog | Github | Twitter