Hi Sridhar,

If your min.insync.replicas value is set to 3, then kafka won’t be able to move 
replicas until there are three replicas listed in the ISR. I would look into 
the health of broker 21 — it’s either down or unhealthy. It’s the only one not 
showing in the ISR list. 

—
Peter Bukowinski

> On Jul 27, 2021, at 1:12 AM, Sridhar Rao <sridharnuvv...@gmail.com> wrote:
> 
> Hi Fabio Pardi,
> 
> Thanks for your prompt response.
> Split brain was our suspicion and we are investigating other possibilities.
> Perhaps our understanding of the problem might be incorrect at the moment.
> The issue started when one of the broker instances went down abruptly (3
> brokers, 3 zookeepers) and the cluster was unstable.
> 
> Later, we were able to restart the affected broker instance followed by
> rolling restart of other 2 brokers. The cluster was stabilized at this
> point.
> However, we noticed un-repl partitions and Preferred Replica imbalance
> irregularities.
> 
> [xxx(user):/xxx/install/1.0.0/bin] ./kafka-topics.sh --describe --zookeeper
> zookeeper1:2181 --under-replicated-partitions
>        Topic: ABC      Partition: 3    Leader: 31      Replicas: 31,21,11
>     Isr: 31,11
>        Topic: __consumer_offsets       Partition: 1    Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 3    Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 7    Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 9    Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 13   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 15   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 19   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 21   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 25   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 27   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 31   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 33   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 37   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 43   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 45   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: __consumer_offsets       Partition: 49   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: _kafka_lenses_alerts     Partition: 0    Leader: 31
> Replicas: 31,21,11      Isr: 31,11
>        Topic: _kafka_lenses_alerts_settings    Partition: 0    Leader: 31
>     Replicas: 31,21,11      Isr: 31,11
>        Topic: _kafka_lenses_processors Partition: 0    Leader: 31
> Replicas: 31,21,11      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 0    Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 4    Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 6    Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 10   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 12   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 16   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 18   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 22   Leader: 31
> Replicas: 21,11,31      Isr: 31,11
>        Topic: connect-kfkxxxprd-offset Partition: 24   Leader: 31
> Replicas: 31,11,21      Isr: 31,11
>        Topic: connect-kfkxxxprd-status Partition: 3    Leader: 31
> Replicas: 21,31,11      Isr: 31,11
> 
> On Tue, Jul 27, 2021 at 9:46 AM Fabio Pardi <f.pa...@portavita.eu> wrote:
> 
>> 
>> 
>> On 27/07/2021 09:19, Sridhar Rao wrote:
>>> Hi Everyone,
>>> 
>>> Recently we noticed a high number of under-replicated-partitions after
>>> zookeeper split brain issue.
>>> We tried fixing the issue by executing ./kafka-reassign-partitions.sh
>>> procedure. However Kafka refuses to re-assign the partitions in ISR and
>>> un-repl partitions remain the same.
>>> 
>>> Kafka version: Apache Kafka 1.0.0
>>> 
>>> Any advice in this regard would be highly appreciated. Please feel free
>> to
>>> request any missing information. Thank you!
>>> 
>> 
>> Zookeeper needs a quorum to elect a leader therefore a split brain is
>> unlikely. Can you describe how split brain occurred? (number of instances,
>> what happened and how it was resolved)
>> 
>> Given that, if your topic/partitions topology did not change, (that I
>> know) even a split brain should not affect your data, therefore your ISR.
>> 
>> Running the below command should tell you which partitions are under
>> replicated
>> 
>> kafka-topics.sh --describe
>> 
>> I think it might be a good starting point to understand what is going on.
>> 
>> 
>> To blindly reassign partitions is in my experience not an ideal solution,
>> because you will have data shuffling around unnecessarily.
>> 
>> 
>> regards,
>> 
>> fabio pardi
>> 
>> 

Reply via email to