Hi,

Also you could potentially check tools like Cruise Control
<https://github.com/linkedin/cruise-control>, which can help with
performing both manual and automated rebalances to help with more even
distributions based on plenty of different options.

On Thu, Mar 28, 2024 at 8:39 AM Luke Chen <show...@gmail.com> wrote:

> Hi Abhishek,
>
> For Zookeeper's mode, there's no workaround, unfortunately.
> But you can upgrade your cluster to the latest kafka version (v3.7.0) to
> migrate to KRaft mode.
> For your information, in KRaft mode, we use `StripedReplicaPlacer`, which
> will fix your problem.
> ref:
>
> https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java#L94
>
> Thank you.
> Luke
>
>
> On Thu, Mar 28, 2024 at 12:55 AM Abhishek Singla <
> abhisheksingla...@gmail.com> wrote:
>
> > Yes, it’s similar.
> >
> > Replicas are evenly distribute among racks but not among brokers within
> > rack even if no. of brokers are same in all racks.
> >
> > Is there a workaround for this?
> >
> > On Wed, 27 Mar 2024 at 5:36 PM, Chia-Ping Tsai <chia7...@apache.org>
> > wrote:
> >
> > > hi Abhishek
> > >
> > > Is this issue similar to the unbalance you had met?
> > >
> > > https://issues.apache.org/jira/browse/KAFKA-10368
> > >
> > > best,
> > > chia-ping
> > >
> > > On 2024/03/23 21:06:59 Abhishek Singla wrote:
> > > > Hi Team,
> > > >
> > > > Kafka version: 2_2.12-2.6.0
> > > > Zookeeper version: 3.8.x
> > > >
> > > > We have a Kafka Cluster of 12 brokers spread equally across 3 racks.
> > > Topic
> > > > gets auto created with default num.partitions=6 and
> > replication_factor=3.
> > > > It is observed that replicas are equally distributed over racks but
> > > within
> > > > the rack the replicas are randomly distributed like sometimes 3,3,0,0
> > or
> > > > sometimes 3:2:1 or sometime 2,2,1,1
> > > >
> > > > Is there a configuration to evenly distribute replicas across brokers
> > > > within a rack, maybe some sort of round robin strategy 2,2,1,1?
> > > >
> > > > And also it is observed that over time 1 broker ends up having way
> more
> > > > replicas across topics than the other broker in the same rack. Is
> > there a
> > > > config for even distribution of replicas across topics also?
> > > >
> > > > Regards,
> > > > Abhishek Singla
> > > >
> > >
> >
>

Reply via email to