Hi all, Following the PR discussion on this KIP's Admin API PR ( https://github.com/apache/kafka/pull/10964), we have changed the Admin API proposed in this KIP to use a separate ListConsumerGroupOffsetsSpec class to specify per-group spec including partitions for which offsets are fetched.
The new APIs are: - ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map<String, ListConsumerGroupOffsetsSpec> groupSpecs, ListConsumerGroupOffsetsOptions options); - ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map<String, ListConsumerGroupOffsetsSpec> groupSpecs); Using a spec class rather than directly setting partitions in the map allows us to evolve the group spec more easily in future. It also avoids a Map with null values for the case where all partitions of the groups are being fetched. We have also replaced the proposed method that returns Map<String, KafkaFuture<Map<TopicPartition, OffsetAndMetadata>>> in ListConsumerGroupOffsetsResult with a method that takes the groupId as argument to simplify the API. The new API is: - public KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> partitionsToOffsetAndMetadata(String groupId); Please let me know if there are any concerns with these changes. Thank you, Rajini On Sat, Jul 3, 2021 at 12:18 AM Sanjana Kaundinya <skaundi...@gmail.com> wrote: > Hello Everyone, > > I recently opened a PR for KIP-709 and it was pointed out that we still > need to come to a consensus on the Admin APIs. Specifically the concern was > around the `ListConsumerGroupOffsetsOptions` class. Currently that class > contains a List<TopicPartition> that acts as a filter for the specific > topic partitions the client wants to fetch offsets for a specific group. > Originally I had planned to extend this by adding a map of type Map<String, > List<TopicPartition>> so when specifying topic partitions, the called could > specify it on a per group basis with the `ListConsumerGroupOffsetsOptions` > class. However it was noted that this is not the typical way that the > “Options” class is used for the requests. Instead they’re normally used as > additional options for the request, and generally the data for the request > is passed in as a constructor. Since we are taking the time to change this > API, might as well try to use some best practices and change how we use the > `ListConsumerGroupOffsetsOptions` class. I propose we change the > `listConsumerGroupOffsets` API as follows: > > Earlier it was proposed that the following will be the method signatures > we would add to Admin.java: > > default ListConsumerGroupOffsetsResult > listConsumerGroupOffsets(List<String> groupIds) { > return listConsumerGroupOffsets(groupIds, new > ListConsumerGroupOffsetsOptions(groupIds)); > } > ListConsumerGroupOffsetsResult listConsumerGroupOffsets(List<String> > groupIds, ListConsumerGroupOffsetsOptions options); > > I propose we change the signatures to the following instead: > > default ListConsumerGroupOffsetsResult > listConsumerGroupOffsets(Map<String, List<TopicPartition>> > groupToTopicPartitions) { > return listConsumerGroupOffsets(groupToTopicPartitions, new > ListConsumerGroupOffsetOptions()); > } > ListConsumerGroupOffsetsResult list listConsumerGroupOffsets(Map<String, > List<TopicPartition>> groupToTopicPartitions, > ListConsumerGroupOffsetOptions options); > > This way we are transferring the data for the requests passed in as > parameters and this frees up the ListConsumerGroupOffsetOptions class to be > used in the future to apply different options to the request. Eventually we > will deprecate the single group listConsumerGroupOffsets method, and with > that the ListConsumerGroupOffsetsOptions method signature will also be > different, no longer storing the data for the topic partitions that we want > to retrieve offsets for. In essence, as part of this change, we will leave > the ListConsumerGroupOffsetsOptions unchanged, and eventually remove the > List<TopicPartition> we store there when we remove the deprecated single > listConsumerGroupOffsets method. > > Appreciate any feedback/discussion on this - thank you! > > Cheers, > Sanjana > On May 14, 2021, 4:07 PM -0700, Sanjana Kaundinya <skaundi...@gmail.com>, > wrote: > > Hi Everyone, > > I’ve begun working on this KIP now and found that another class will be > needing public changes. I have updated the KIP to reflect this, so just > wanted to update the dev list as well. You can find the updated KIP here: > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258 > > Thanks, > > Sanjana > > On Jan 27, 2021, 4:18 AM -0800, Thomas Scott <t...@confluent.io>, wrote: > > > Hi Magnus, > > > > > > Thanks for the review, I've added //moved and explanation as requested. > > > > > > Thanks > > > > > > Tom > > > > > > > > > On Wed, Jan 27, 2021 at 12:05 PM Magnus Edenhill <mag...@edenhill.se> > wrote: > > > > > > > Hey Thomas, > > > > > > > > I'm late to the game. > > > > > > > > It looks like the "top level" ErrorCode moved from the top-level to > the > > > > Group array, which makes sense, > > > > but it would be good if it was marked as // MOVED in the KIP and > also a > > > > note that top level errors that > > > > are unrelated to the group will be returned as per-group errors. > > > > > > > > > > > > Regards, > > > > Magnus > > > > > > > > > > > > Den tis 26 jan. 2021 kl 15:42 skrev Thomas Scott <t...@confluent.io>: > > > > > > > > > Thanks David I've updated it. > > > > > > > > > > On Tue, Jan 26, 2021 at 1:55 PM David Jacot <dja...@confluent.io> > wrote: > > > > > > > > > > > Great. That answers my question! > > > > > > > > > > > > Thomas, I suggest adding a Related/Future Work section in the > > > > > > KIP to link KIP-699 more explicitly. > > > > > > > > > > > > Thanks, > > > > > > David > > > > > > > > > > > > On Tue, Jan 26, 2021 at 1:30 PM Thomas Scott <t...@confluent.io> > wrote: > > > > > > > > > > > > > Hi Mickael/David, > > > > > > > > > > > > > > I feel like the combination of these 2 KIPs gives the complete > > > > > solution > > > > > > > but they can be implemented independently. I have added a > description > > > > > and > > > > > > > links to KIP-699 to KIP-709 to this effect. > > > > > > > > > > > > > > Thanks > > > > > > > > > > > > > > Tom > > > > > > > > > > > > > > > > > > > > > On Tue, Jan 26, 2021 at 11:44 AM Mickael Maison < > > > > > > mickael.mai...@gmail.com> > > > > > > > wrote: > > > > > > > > > > > > > > > Hi Thomas, > > > > > > > > Thanks, the KIP looks good. > > > > > > > > > > > > > > > > David, > > > > > > > > I started working on exactly that a few weeks ago: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-699%3A+FindCoordinators > > > > > > > > I hope to complete my draft and start a discussion later on > this > > > > > week. > > > > > > > > > > > > > > > > Thanks > > > > > > > > > > > > > > > > On Tue, Jan 26, 2021 at 10:06 AM David Jacot < > dja...@confluent.io> > > > > > > > wrote: > > > > > > > > > > > > > > > > > > Hi Thomas, > > > > > > > > > > > > > > > > > > Thanks for the KIP. Overall, the KIP looks good to me. > > > > > > > > > > > > > > > > > > I have only one question: The FindCoordinator API only > supports > > > > > > > > > resolving one group id at the time. If we want to get the > offsets > > > > > for > > > > > > > > > say N groups, that means that we have to first issue N > > > > > > FindCoordinator > > > > > > > > > requests, wait for the responses, group by coordinators, > and then > > > > > > > > > send a OffsetFetch request per coordinator. I wonder if we > should > > > > > > > > > also extend the FindCoordinator API to support resolving > multiple > > > > > > > > > groups as well. This would make the implementation in the > admin > > > > > > > > > client a bit easier and would ensure that we can handle > multiple > > > > > > > > > groups end-to-end. Have you thought about this? > > > > > > > > > > > > > > > > > > Best, > > > > > > > > > David > > > > > > > > > > > > > > > > > > On Tue, Jan 26, 2021 at 10:13 AM Rajini Sivaram < > > > > > > > rajinisiva...@gmail.com > > > > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > Hi Thomas, > > > > > > > > > > > > > > > > > > > > Thanks for the KIP, this is a useful addition for admin > use > > > > > cases. > > > > > > It > > > > > > > > may > > > > > > > > > > be worth starting the voting thread soon if we want to > get this > > > > > > into > > > > > > > > 2.8.0. > > > > > > > > > > > > > > > > > > > > Regards, > > > > > > > > > > > > > > > > > > > > Rajini > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Mon, Jan 25, 2021 at 1:52 PM Thomas Scott < > t...@confluent.io > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > Thanks Ismael, that's a lot better. I've updated the > KIP with > > > > > > this > > > > > > > > > > > behaviour instead. > > > > > > > > > > > > > > > > > > > > > > On Mon, Jan 25, 2021 at 11:42 AM Ismael Juma < > > > > > ism...@juma.me.uk> > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > Thanks for the KIP, Thomas. One question below: > > > > > > > > > > > > > > > > > > > > > > > > Should an Admin client with this new functionality > be used > > > > > > > against > > > > > > > > an > > > > > > > > > > old > > > > > > > > > > > > > broker that cannot handle these requests then the > methods > > > > > > will > > > > > > > > throw > > > > > > > > > > > > > UnsupportedVersionException as per the usual > pattern. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Did we consider automatically falling back to the > single > > > > > group > > > > > > id > > > > > > > > > > request > > > > > > > > > > > > if the more efficient one is not supported? > > > > > > > > > > > > > > > > > > > > > > > > Ismael > > > > > > > > > > > > > > > > > > > > > > > > On Mon, Jan 25, 2021 at 3:34 AM Thomas Scott < > > > > > t...@confluent.io > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > Hi, > > > > > > > > > > > > > > > > > > > > > > > > > > I'm starting this thread to discuss KIP-709 to > extend > > > > > > > OffsetFetch > > > > > > > > > > > > requests > > > > > > > > > > > > > to accept multiple group ids. Please check out the > KIP > > > > > here: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258 > > > > > > > > > > > > > > > > > > > > > > > > > > Any comments much appreciated. > > > > > > > > > > > > > > > > > > > > > > > > > > thanks, > > > > > > > > > > > > > > > > > > > > > > > > > > Tom > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >