Hi, I am not very comfortable with the proposal of this KIP. The main issue is that changing the number of partitions means that all group metadata is lost because the hashing changes. I wonder if we should just disallow changing the number of partitions entirely. Did we consider something like this?
Best, David Le mar. 4 avr. 2023 à 17:57, Divij Vaidya <divijvaidy...@gmail.com> a écrit : > FYI, a user faced this problem and reached out to us in the mailing list > [1]. Implementation of this KIP could have reduced the downtime for these > customers. > > Christo, would you like to create a JIRA and associate with the KIP so that > we can continue to collect cases in the JIRA where users have faced this > problem? > > [1] https://lists.apache.org/thread/zoowjshvdpkh5p0p7vqjd9fq8xvkr1nd > > -- > Divij Vaidya > > > > On Wed, Jan 18, 2023 at 9:52 AM Christo Lolov <christolo...@gmail.com> > wrote: > > > Greetings, > > > > I am bumping the below DISCUSSion thread for KIP-895. The KIP presents a > > situation where consumer groups are in an undefined state until a rolling > > restart of a cluster is performed. While I have demonstrated the > behaviour > > using a cluster using Zookeeper I believe the same problem can be shown > in > > a KRaft cluster. Please let me know your opinions on the problem and the > > presented solution. > > > > Best, > > Christo > > > > On Thursday, 29 December 2022 at 14:19:27 GMT, Christo > > > <christo_lo...@yahoo.com.invalid> wrote: > > > > > > > > > Hello! > > > I would like to start this discussion thread on KIP-895: Dynamically > > > refresh partition count of __consumer_offsets. > > > The KIP proposes to alter brokers so that they refresh the partition > > count > > > of __consumer_offsets used to determine group coordinators without > > > requiring a rolling restart of the cluster. > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-895%3A+Dynamically+refresh+partition+count+of+__consumer_offsets > > > > > > Let me know your thoughts on the matter! > > > Best, Christo > > > > > >