Hi Andrew, Thanks for the updated KIP. Few queries below:
1. What is the use-case of deliveryCount in ShareFetchResponse? 2. During delete share groups, Do we need to clean any in-memory state from share-partition leaders? 3. Any metrics for the share-coordinator? Thanks Manikumar On Wed, Feb 21, 2024 at 12:11 AM Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: > Hi Manikumar, > Thanks for your comments. > > 1. I believe that in general, there are not situations in which a dynamic > config > change is prevented because of the existence of a resource. So, if we > prevented > setting config `group.type=consumer` on resource G of GROUP type > if there was a share group G in existence, it would be a bit weird. > > I wonder whether changing the config name to `new.group.type` would help. > It’s > ensuring the type of a new group created. > > 2. The behaviour for a DEAD share group is intended to be the same as a > DEAD > consumer group. The group cannot be “reused” again as such, but the group > ID > can be used by a new group. > > 3. Yes. AlterShareGroupOffsets will cause a new SHARE_CHECKPOINT. > > 4. In common with Admin.deleteConsumerGroups, the underlying Kafka RPC > for Admin.deleteShareGroups is DeleteGroups. This is handled by the group > coordinator and it does this by writing control records (a tombstone in > this case). > The KIP doesn’t say anything about this because it’s the same as consumer > groups. > Perhaps it would be sensible to add a GroupType to DeleteGroupsRequest so > we can > make sure we are deleting the correct type of group. The fact that there > is not a specific > RPC for DeleteShareGroups seems correct to me. > > 5. I prefer using “o.a.k.clients.consumer” because it’s already a public > package and > many of the classes and interfaces such as ConsumerRecord are in that > package. > > I definitely need to add more information about how the Admin operations > work. > I will add a section to the KIP in the next version, later today. This > will fill in details for > your questions (3) and (4). > > Thanks, > Andrew > > > On 14 Feb 2024, at 18:04, Manikumar <manikumar.re...@gmail.com> wrote: > > > > Hi Andrew, > > > > Thanks for the KIP. A few comments below. > > > > 1. kafka-configs.sh (incrementalAlterConfigs) allows you to dynamically > > change the configs. Maybe in this case, we should not allow the user to > > change `group.type` if it's already set. > > 2. What's the behaviour after a group transitions into DEAD state. Do we > > add new control records to reset the state? Can we reuse the group again? > > 3. Are we going to write new control records after the > > AlterShareGroupOffsets API to reset the state? > > 4. Is there any API for DeleteShareGroups? I assume, group co-ordinator > is > > going to handle the API. If so, Does this mean the group co-ordinator > also > > needs to write control records? > > 5. How about using "org.apache.kafka.clients.consumer.share" package for > > new interfaces/classes? > > > > > > Thanks, > > Manikumar > >