Hi PoAn, Thanks for the KIP. I have some questions:
JK1: (Realized this is similar to LM5) On `--describe --verbose` proposed changes - doesn't `--describe $group` default to printing the offsets? Perhaps you're referring to the `--state` argument? Also, would that mean the default `--describe $group --verbose` command would not print the added field to `--offsets verbose` (leader-epoch) or would it? JK2: Looking at ConsumerGroupCommand.java, the existing "ASSIGNMENT" column under `--members --verbose` does group the topic partitions by topic but does not prefix the grouping with the topic name like you're proposal: "my_topic:0,1;new_topic:0,1". Should we do apply the same format for the classic group as well? On 2024/11/20 21:32:41 "Lianet M." wrote: > Hello PoAn, just of couple more minor comments: > > LM4. Regarding the new “protocol” field added to MemberDescription. Should > we consider reusing the existing GroupProtocol enum instead of String? > (It’s the one we use from the consumer side to refer to the protocol in > use, just missing Share I notice). > > LM5. Regarding the change to the output for kafka-consumer-groups, the > command shown does not include the —state option, but the output shows > state info (state, #members, epochs). I would guess that we want to modify > the output only when we describe a group with the —state —verbose option, > is my understanding right? If my understanding is right we’re just missing > adding the —state in the example, and the KIP does not introduce any > changes to the —describe —verbose option. (If not, it would mean a bigger > change to the output of —describe —verbose which I expect is not the > intention?) > > Thanks! > > Lianet > > > On Wed, Nov 20, 2024 at 2:12 AM PoAn Yang <yangp...@gmail.com> wrote: > > > Hi Chia-Ping, > > > > Thanks for the review and suggestions. > > > > CI0: Thanks for the reminder. Update validVersions in > > ConsumerGroupDescribeRequest to 0-1. > > > > CI1: Yes, I use `ConsumerGroupMember#useClassicProtocol` to check whether > > a member in consumer group uses “classic” or “consumer” protocol. > > > > CI2: Yes, a member in share group always uses “share” protocol. > > > > CI3: Add a table to show meaning of “classic”, “consumer”, and “share” > > protocol. > > > > BTW, the vote thread is in > > https://lists.apache.org/thread/rb25tf75tzf4c7jqqldlo5jh9w8chsq6. > > > > Thanks, > > PoAn > > > > > On Nov 20, 2024, at 11:46 AM, Chia-Ping Tsai <chia7...@apache.org> > > wrote: > > > > > > hi PoAn > > > > > > CI0: We have to bump the version of ConsumerGroupDescribeRequest as > > well, so server can distinguish the new and old behavior. > > > > > > CI1: the type of new filed is string, so I guess you plan to use > > `ConsumerGroupMember#useClassicProtocol` [0] flag to return either > > "classic" or "consumer", right? > > > > > > CI2: `MemberDescription` is used by `ShareGroupDescription` too, so the > > new filed protocol in shared group is always "shared", right? > > > > > > CI3: Could you consider adding a table to show the value of the protocol > > field in each case? Andrew has a beautiful table in KIP-1043 that lists all > > possible protocol names. > > > > > > [0] > > https://github.com/apache/kafka/blob/441a6d0b790f4a17b454caeea7588a6b90fbd9db/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java#L454 > > > > > > > > Best, > > > Chia-Ping > > > > > > On 2024/11/19 15:45:16 PoAn Yang wrote: > > >> Hi David, > > >> > > >> Thanks for the reminder. > > >> > > >> DJ4: Add related content to the KIP. > > >> Bump validVersions in ConsumerGroupDescribeResponse to “0-1”. > > >> Add a new field “Protocol” to ConsumerGroupDescribeResponse. > > >> > > >> Thanks, > > >> PoAn > > >> > > >>> On Nov 19, 2024, at 3:57 PM, David Jacot <dja...@confluent.io.INVALID> > > wrote: > > >>> > > >>> Hi PoAn, > > >>> > > >>> DJ4: Could you please also include the required changes to the > > >>> ConsumerGroupDescribed API in the public interface section? We > > basically > > >>> need to bump the version, add the new field, etc. > > >>> > > >>> Thanks, > > >>> David > > >>> > > >>> On Mon, Nov 18, 2024 at 5:40 AM PoAn Yang <yangp...@gmail.com> wrote: > > >>> > > >>>> Hi David, > > >>>> > > >>>> DJ4: Add a new field protocol to MemberDescription, > > >>>> so the command line tool can show protocol information when users > > describe > > >>>> members. > > >>>> > > >>>> If there is no further suggestion, I will start a vote thread today. > > >>>> > > >>>> Thanks, > > >>>> PoAn > > >>>> > > >>>>> On Nov 15, 2024, at 11:50 PM, David Jacot > > <dja...@confluent.io.INVALID> > > >>>> wrote: > > >>>>> > > >>>>> Hi, > > >>>>> > > >>>>> DJ2: Using "-" sounds good to me. > > >>>>> > > >>>>> DJ3: That seems reasonable to me. > > >>>>> > > >>>>> DJ4: Why not add it right now? I don't want to change the output of > > the > > >>>>> tool too many times. > > >>>>> > > >>>>> Best, > > >>>>> David > > >>>>> > > >>>>> On Fri, Nov 15, 2024 at 3:23 PM PoAn Yang <yangp...@gmail.com> > > wrote: > > >>>>> > > >>>>>> Hi David / Andrew, > > >>>>>> > > >>>>>> Thanks for review. Thanks Andrew for picking up > > kafka-share-groups.sh > > >>>>>> implementation. > > >>>>>> I will handle kafka-consumer-groups.sh. > > >>>>>> > > >>>>>> DJ3: After discussing with @Chia-Ping Tsai, we think that using new > > >>>> format > > >>>>>> is more clear. > > >>>>>> > > >>>>>> The new format will be like > > >>>>>> > > >>>> > > <topic-name1>:<partition-id1>,<partition-id2>;<topic-name2>:<partition-id1>,<partition-id2> > > >>>>>> > > >>>>>> Using colon(:) to concat topic name and partition IDs. > > >>>>>> Using comma(,) to concat partition IDs within same topic name. > > >>>>>> Using semicolon(;) to concat topic strings. > > >>>>>> > > >>>>>> AS3: Fix it with kafka-share-groups.sh. Thanks. > > >>>>>> > > >>>>>> If there is no further suggestion, I will start a vote thread next > > >>>> Monday. > > >>>>>> > > >>>>>> Thanks, > > >>>>>> PoAn > > >>>>>> > > >>>>>>> On Nov 14, 2024, at 9:05 PM, Andrew Schofield < > > >>>>>> andrew_schofield_j...@outlook.com> wrote: > > >>>>>>> > > >>>>>>> Hi PoAn, > > >>>>>>> DJ2: I was just going to comment that "-" would be a more > > appropriate > > >>>>>> missing value, but > > >>>>>>> you got there first. > > >>>>>>> > > >>>>>>> AS3: The examples for kafka-share-groups.sh include > > >>>>>> kafka-consumer-groups.sh in the > > >>>>>>> command line. > > >>>>>>> > > >>>>>>> If this is accepted in time, I'm happy to pick up the > > implementation of > > >>>>>> the share groups > > >>>>>>> part of this if it helps. > > >>>>>>> > > >>>>>>> Thanks, > > >>>>>>> Andrew > > >>>>>>> > > >>>>>>> ________________________________________ > > >>>>>>> From: Frank Yang <yangp...@gmail.com> > > >>>>>>> Sent: 14 November 2024 10:48 > > >>>>>>> To: dev@kafka.apache.org <dev@kafka.apache.org> > > >>>>>>> Subject: Re: [DISCUSS] KIP-1099: Extend kafka-consumer-groups > > command > > >>>>>> line tool to support new consumer group > > >>>>>>> > > >>>>>>> Hi David, > > >>>>>>> > > >>>>>>> Thanks for the review and suggestion! I would like to get this in > > AK > > >>>> 4.0 > > >>>>>> as well. I will do my best. > > >>>>>>> > > >>>>>>> DJ1: Update KIP to put GROUP-EPOCH and TARGET-ASSIGNMENT-EPOCH > > before > > >>>>>> #MEMBERS. > > >>>>>>> > > >>>>>>> DJ2: I prefer to follow current missing column value “-“. > > (reference < > > >>>>>> > > >>>> > > https://github.com/apache/kafka/blob/18199028a672fd973ac37bf26316994babc2a6da/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java#L92 > > >>>>>>> ) > > >>>>>>> > > >>>>>>> DJ3: Update KIP to use CURRENT-EPOCH CURRENT-ASSIGNMENT > > TARGET-EPOCH > > >>>>>> TARGET-ASSIGNMENT. > > >>>>>>> Remove GROUP-EPOCH. > > >>>>>>> > > >>>>>>> For assignment value, it follows current output (reference < > > >>>>>> > > >>>> > > https://github.com/apache/kafka/blob/18199028a672fd973ac37bf26316994babc2a6da/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java#L413-L418 > > >>>>> ). > > >>>>>> I think the form of `topicid-partitionid` is more clear. > > >>>>>>> If we would like to use this form, I will update both output in > > >>>>>> kafka-consumer-groups.sh and kafka-share-groups.sh. > > >>>>>>> > > >>>>>>> DJ4: It looks like DescribeGroupsResponseData only has protocol > > type at > > >>>>>> group level. > > >>>>>>> Both DescribeGroupsResponseData and > > ConsumerGroupDescribeResponseData > > >>>>>> don’t have protocol at member level. > > >>>>>>> Could we use a followup to add it? > > >>>>>>> > > >>>>>>> DJ5: Update KIP to put LEADER-EPOCH before CURRENT-OFFSET. > > >>>>>>> > > >>>>>>> Thanks, > > >>>>>>> PoAn > > >>>>>>> > > >>>>>>>> On Nov 14, 2024, at 3:43 PM, David Jacot > > <dja...@confluent.io.INVALID > > >>>>> > > >>>>>> wrote: > > >>>>>>>> > > >>>>>>>> Hi PoAn, > > >>>>>>>> > > >>>>>>>> Thanks for the KIP! I have a few minor comments/suggestions: > > >>>>>>>> > > >>>>>>>> DJ1: In the output of `--describe --verbose`, I would suggest > > putting > > >>>>>>>> `GROUP-EPOCH` and `TARGET-ASSIGNMENT-EPOCH` before `#MEMBERS`. > > >>>>>>>> DJ2: Continuing on the above, I assume that we will print out N/A > > for > > >>>>>> the > > >>>>>>>> fields not supported by classic groups. Is this correct? > > >>>>>>>> DJ3: In the output of `--describe --members --verbose`, I wonder > > if we > > >>>>>>>> should use the following order and terms: CURRENT-EPOCH > > >>>>>> CURRENT-ASSIGNMENT > > >>>>>>>> TARGET-EPOCH TARGET-ASSIGNMENT . I would remove the GROUP-EPOCH > > >>>> because > > >>>>>> it > > >>>>>>>> is already in the group description. The value `(0)` used for the > > >>>>>>>> assignment is incorrect. Here I suppose that we will print out the > > >>>> list > > >>>>>> of > > >>>>>>>> partitions in the form of `topicid-partitionid`. > > >>>>>>>> DJ4: Continuing on the above, I wonder if we should also add the > > >>>>>> protocol > > >>>>>>>> used `classic` or `consumer`. For context, it is possible to have > > >>>>>> `classic` > > >>>>>>>> members and `consumer` members in a `consumer` group during an > > online > > >>>>>>>> upgrade from the classic protocol to the consumer protocol. Having > > >>>> this > > >>>>>>>> information may be handy for administrators. What do you think? > > >>>>>>>> DJ5: In the output of `--describe --offsets --verbose`, I would > > >>>> suggest > > >>>>>>>> putting `LEADER-EPOCH` closer to `CURRENT-OFFSET`. > > >>>>>>>> > > >>>>>>>> It would be great if we could get this one in AK 4.0 as it > > changes the > > >>>>>>>> output of the command. > > >>>>>>>> > > >>>>>>>> Thanks, > > >>>>>>>> DJ > > >>>>>>>> > > >>>>>>>> On Fri, Nov 1, 2024 at 7:40 AM Frank Yang <yangp...@gmail.com> > > wrote: > > >>>>>>>> > > >>>>>>>>> Hi Sean / Andrew / Lianet, > > >>>>>>>>> > > >>>>>>>>> Thanks for all your review and suggestions. > > >>>>>>>>> > > >>>>>>>>> AS1, LM1, LM4: Change to add KIP-848 information when users give > > >>>>>> —verbose > > >>>>>>>>> option. > > >>>>>>>>> —describe —verbose: shows group epoch and target assignment > > epoch. > > >>>>>>>>> —describe —members —verbose: shows above information, member > > epoch, > > >>>> and > > >>>>>>>>> target assignment. > > >>>>>>>>> > > >>>>>>>>> AS2: Change to use MEMBER-EPOCH to align with KIP-848 definition. > > >>>>>>>>> > > >>>>>>>>> LM2: For classic group, it doesn’t have epoch, so I use Optional > > >>>>>> fields in > > >>>>>>>>> ConsumerGroupDescription. > > >>>>>>>>> For share group, it relies on KIP-848. It must have epoch, so I > > use > > >>>> int > > >>>>>>>>> fields in ShareGroupDescription. > > >>>>>>>>> > > >>>>>>>>> LM3: Remove —state change. We can get group level information by > > >>>>>> —describe > > >>>>>>>>> —verbose. > > >>>>>>>>> > > >>>>>>>>> SQ1: Add LEADER-EPOCH when users give —describe —offsets > > —verbose. > > >>>>>>>>> > > >>>>>>>>> Thanks. > > >>>>>>>>> PoAn > > >>>>>>>>> > > >>>>>>>>>> On Nov 1, 2024, at 5:08 AM, Lianet M. <liane...@gmail.com> > > wrote: > > >>>>>>>>>> > > >>>>>>>>>> Hello Frank, thanks for the KIP! A few comments: > > >>>>>>>>>> > > >>>>>>>>>> LM1. I strongly agree with Andrew's suggestion of moving this > > into a > > >>>>>>>>>> --verbose option. I expect someone would only attempt to make > > sense > > >>>> of > > >>>>>>>>> the > > >>>>>>>>>> epochs while debugging an issue, not while trying to view the > > group > > >>>> or > > >>>>>>>>>> member's info. (Also member-epoch makes more sense to me, even > > more > > >>>>>> if we > > >>>>>>>>>> end up agreeing in a --verbose). Related to both issues, my > > take is > > >>>>>> that > > >>>>>>>>>> whoever is close to the protocol will expect member-epoch, > > whoever > > >>>> is > > >>>>>> not > > >>>>>>>>>> will probably won't even need to see the epochs at all. > > >>>>>>>>>> > > >>>>>>>>>> LM2. Why are the epochs defined as Optional (don't we expect to > > >>>> always > > >>>>>>>>> have > > >>>>>>>>>> them? with 0 initially, defined on the broker side for the group > > >>>> ones, > > >>>>>>>>> and > > >>>>>>>>>> client side for the member) > > >>>>>>>>>> > > >>>>>>>>>> LM3. Why is the KIP including the –state option in the proposed > > >>>>>> changes? > > >>>>>>>>> ( > > >>>>>>>>>> > > >>>>>>>>> > > >>>>>> > > >>>> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=327977411#KIP1099:Extendkafkaconsumergroupscommandlinetooltosupportnewconsumergroup---state > > >>>>>>>>>> ). > > >>>>>>>>>> I get that the output would change in that example, but it’s not > > >>>>>> because > > >>>>>>>>> of > > >>>>>>>>>> any change to the –state option itself. It's because of the > > change > > >>>>>>>>> proposed > > >>>>>>>>>> to the –described (required with the --state), and the changes > > to > > >>>>>>>>>> --describe are already explained above (seems confusing, got me > > >>>>>> looking > > >>>>>>>>> for > > >>>>>>>>>> a change to the state filter which seemed unrelated). > > >>>>>>>>>> > > >>>>>>>>>> LM4. Group epoch and target assignment epoch are conceptually > > at the > > >>>>>>>>> group > > >>>>>>>>>> level. vs member epoch that lands at a member level. So wonder > > if we > > >>>>>>>>> should > > >>>>>>>>>> show them accordingly (ex. using the --verbose option) > > >>>>>>>>>> –describe –verbose => shows group epoch and target assignment > > epoch > > >>>>>>>>>> –describe –members –verbose => shows member epoch > > (along > > >>>>>>>>>> with group epoch and target assignment epoch) > > >>>>>>>>>> > > >>>>>>>>>> Thanks! > > >>>>>>>>>> > > >>>>>>>>>> Lianet > > >>>>>>>>>> > > >>>>>>>>>> On Thu, Oct 31, 2024 at 7:30 AM Andrew Schofield < > > >>>>>>>>>> andrew_schofield_j...@outlook.com> wrote: > > >>>>>>>>>> > > >>>>>>>>>>> Hi PoAn, > > >>>>>>>>>>> Thanks for the KIP. I have a few comments. > > >>>>>>>>>>> > > >>>>>>>>>>> AS1: It seems to me that these new additions are most useful to > > >>>>>> people > > >>>>>>>>>>> trying to understand > > >>>>>>>>>>> the progress of rebalancing in quite some detail. The > > information > > >>>> is > > >>>>>>>>>>> really not understandable > > >>>>>>>>>>> for most users who do not have deep knowledge of KIP-848/932. > > >>>>>>>>>>> > > >>>>>>>>>>> As a result, I suggest for kafka-share-groups.sh that you add a > > >>>>>>>>> --members > > >>>>>>>>>>> --verbose option > > >>>>>>>>>>> and only include the new information in the output for that > > option, > > >>>>>>>>> rather > > >>>>>>>>>>> than changing the > > >>>>>>>>>>> non-verbose --members output. > > >>>>>>>>>>> > > >>>>>>>>>>> I also make a similar suggestion for kafka-consumer-groups.sh > > >>>>>> --members > > >>>>>>>>>>> and only add the > > >>>>>>>>>>> new information for the --verbose output. > > >>>>>>>>>>> > > >>>>>>>>>>> AS2: I strongly suggest that you use MEMBER-EPOCH instead of > > >>>>>>>>>>> CONSUMER-EPOCH. > > >>>>>>>>>>> I think it's more confusing not following the terminology of > > the > > >>>>>> KIPs. > > >>>>>>>>> For > > >>>>>>>>>>> one thing, > > >>>>>>>>>>> we already have "member" in the admin client such as > > >>>>>> MemberDescription. > > >>>>>>>>>>> It's not a > > >>>>>>>>>>> new term. > > >>>>>>>>>>> > > >>>>>>>>>>> > > >>>>>>>>>>> Thanks, > > >>>>>>>>>>> Andrew > > >>>>>>>>>>> ________________________________________ > > >>>>>>>>>>> From: PoAn Yang <pay...@apache.org> > > >>>>>>>>>>> Sent: 25 October 2024 13:55 > > >>>>>>>>>>> To: dev@kafka.apache.org <dev@kafka.apache.org> > > >>>>>>>>>>> Subject: Re: [DISCUSS] KIP-1099: Extend kafka-consumer-groups > > >>>> command > > >>>>>>>>> line > > >>>>>>>>>>> tool to support new consumer group > > >>>>>>>>>>> > > >>>>>>>>>>> Hi Lucas, > > >>>>>>>>>>> > > >>>>>>>>>>> Thanks for the review! > > >>>>>>>>>>> > > >>>>>>>>>>> 1) Yes, I add related change for kafka-share-groups.sh to the > > KIP. > > >>>>>> Could > > >>>>>>>>>>> you take a look? Thanks for the suggestion. > > >>>>>>>>>>> > > >>>>>>>>>>> 2) We use CONSUMER-ID as member ID. If we use MEMBER-EPOCH > > here, > > >>>>>> users > > >>>>>>>>> may > > >>>>>>>>>>> confuse what is different between CONSUMER and MEMBER. > > >>>>>>>>>>> > > >>>>>>>>>>> Thanks, > > >>>>>>>>>>> PoAn > > >>>>>>>>>>> > > >>>>>>>>>>> On 2024/10/23 13:28:17 Lucas Brutschy wrote: > > >>>>>>>>>>>> Hi Frank, > > >>>>>>>>>>>> > > >>>>>>>>>>>> thanks for the KIP! > > >>>>>>>>>>>> > > >>>>>>>>>>>> 1) For consistency, should we do the same for > > >>>>>>>>>>>> kafka-share-groups.sh, ShareGroupDescription, etc. ? Even if > > we do > > >>>>>> not > > >>>>>>>>>>>> implement it right now if the share group implementation may > > still > > >>>>>> be > > >>>>>>>>>>>> incomplete, it may make sense to include it in the KIP. > > >>>>>>>>>>>> > > >>>>>>>>>>>> 2) Why call it CONSUMER-EPOCH, not MEMBER-EPOCH? That would > > seem > > >>>>>> more > > >>>>>>>>>>>> consistent. > > >>>>>>>>>>>> > > >>>>>>>>>>>> Cheers, > > >>>>>>>>>>>> Lucas > > >>>>>>>>>>>> > > >>>>>>>>>>>> On Wed, Oct 23, 2024 at 2:41 PM Frank Yang < > > yangp...@gmail.com> > > >>>>>> wrote: > > >>>>>>>>>>>> > > >>>>>>>>>>>>> Hi all, > > >>>>>>>>>>>>> > > >>>>>>>>>>>>> I would like to kick off the discussion of KIP-1099. This KIP > > >>>>>> enhances > > >>>>>>>>>>> the > > >>>>>>>>>>>>> kafka-consumer-groups tools to include state which is > > introduced > > >>>> by > > >>>>>>>>>>> KIP-848. > > >>>>>>>>>>>>> > > >>>>>>>>>>>>> KIP-1099: Extend kafka-consumer-groups command line tool to > > >>>> support > > >>>>>>>>> new > > >>>>>>>>>>>>> consumer group - Apache Kafka - Apache Software Foundation > > >>>>>>>>>>>>> < > > >>>>>>>>>>> > > >>>>>>>>> > > >>>>>> > > >>>> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1099%3A+Extend+kafka-consumer-groups+command+line+tool+to+support+new+consumer+group > > >>>>>>>>>>>> > > >>>>>>>>>>>>> cwiki.apache.org > > >>>>>>>>>>>>> < > > >>>>>>>>>>> > > >>>>>>>>> > > >>>>>> > > >>>> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1099%3A+Extend+kafka-consumer-groups+command+line+tool+to+support+new+consumer+group > > >>>>>>>>>>>> > > >>>>>>>>>>>>> [image: favicon.ico] > > >>>>>>>>>>>>> < > > >>>>>>>>>>> > > >>>>>>>>> > > >>>>>> > > >>>> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1099%3A+Extend+kafka-consumer-groups+command+line+tool+to+support+new+consumer+group > > >>>>>>>>>>>> > > >>>>>>>>>>>>> < > > >>>>>>>>>>> > > >>>>>>>>> > > >>>>>> > > >>>> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1099%3A+Extend+kafka-consumer-groups+command+line+tool+to+support+new+consumer+group > > >>>>>>>>>>>> > > >>>>>>>>>>>>> > > >>>>>>>>>>>>> Thank you, > > >>>>>>>>>>>>> PoAn > > >>>>>>>>>>>>> > > >>>>>>>>>>>> > > >>>>>>>>>>> > > >>>>>>>>> > > >>>>>>>>> > > >>>>>> > > >>>>>> > > >>>> > > >>>> > > >> > > >> > > > > >