Currently, we retain the last replica in isr only when unclean leader
election is disabled. We probably should always retain the last replica in
isr. Could you file a jira to track this?

Thanks,

Jun

On Wed, Apr 8, 2015 at 9:30 AM, Valentin <kafka-9999...@sblk.de> wrote:

>
> Hi all,
>
> I have faced a strange situation with a broker this week where some
> partitions of a topic with replication factor 1 became unavailable and
> stayed unavailable even after the broker node that had (network?) issues
> was back in the cluster. Even multiple restarts of the broker did not get
> the partitions with replication factor 1 on it back online. While at the
> same time it had no issues to get partitions with a replication factor 2
> and another online Isr back into sync.
> How can a broker that is the only replica for a partition not be part of
> the Isr list? How could it be out of sync? Does that even make sense?
> Now I am wondering whether someone has an idea on what might cause a
> situation like this.
>
> Some example output, broker02(2) was the one having issues:
>
> broker01 root:/opt/kafka/bin # ./kafka-topics.sh --zookeeper 172.16.x.x
> --describe
> Topic:T1        PartitionCount:10       ReplicationFactor:1     Configs:
>         Topic: T1       Partition: 0    Leader: -1      Replicas: 2
>  Isr:
>         Topic: T1       Partition: 1    Leader: 3       Replicas: 3
>  Isr: 3
>         Topic: T1       Partition: 2    Leader: 1       Replicas: 1
>  Isr: 1
>         Topic: T1       Partition: 3    Leader: -1      Replicas: 2
>  Isr:
>         Topic: T1       Partition: 4    Leader: 3       Replicas: 3
>  Isr: 3
>         Topic: T1       Partition: 5    Leader: 1       Replicas: 1
>  Isr: 1
>         Topic: T1       Partition: 6    Leader: -1      Replicas: 2
>  Isr:
>         Topic: T1       Partition: 7    Leader: 3       Replicas: 3
>  Isr: 3
>         Topic: T1       Partition: 8    Leader: 1       Replicas: 1
>  Isr: 1
>         Topic: T1       Partition: 9    Leader: -1      Replicas: 2
>  Isr:
> Topic:T2        PartitionCount:10       ReplicationFactor:2     Configs:
>         Topic: T2       Partition: 0    Leader: 2       Replicas: 2,3
>  Isr: 3,2
>         Topic: T2       Partition: 1    Leader: 3       Replicas: 3,1
>  Isr: 3,1
>         Topic: T2       Partition: 2    Leader: 1       Replicas: 1,2
>  Isr: 1,2
>         Topic: T2       Partition: 3    Leader: 2       Replicas: 2,1
>  Isr: 1,2
>         Topic: T2       Partition: 4    Leader: 3       Replicas: 3,2
>  Isr: 3,2
>         Topic: T2       Partition: 5    Leader: 1       Replicas: 1,3
>  Isr: 3,1
>         Topic: T2       Partition: 6    Leader: 2       Replicas: 2,3
>  Isr: 3,2
>         Topic: T2       Partition: 7    Leader: 3       Replicas: 3,1
>  Isr: 3,1
>         Topic: T2       Partition: 8    Leader: 1       Replicas: 1,2
>  Isr: 1,2
>         Topic: T2       Partition: 9    Leader: 2       Replicas: 2,1
>  Isr: 1,2
> ...
>
>
> All partitions visible in this log were affected:
> broker02 root:/monsoon/log/kafka # cat kafka-state-change.log
> [2015-04-07 11:33:03,943] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T3,5] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:03,976] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T3,2] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,022] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T5,7] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,032] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T6,7] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,049] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T5,4] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,058] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T6,4] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,088] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T5,1] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,110] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T6,1] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,131] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T7,0] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,152] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T1,9] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,171] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T1,6] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,183] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T1,3] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,205] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T1,0] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,245] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T4,7] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,284] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T4,4] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,313] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T4,1] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
> [2015-04-07 11:33:04,327] ERROR Broker 2 received LeaderAndIsrRequest with
> correlation id 5240 from controller 3 epoch 28 for partition [T3,8] but
> cannot become follower since the new leader -1 is unavailable.
> (state.change.logger)
>
>
> We have a very low data retention time (log.roll.hours=1) so the actual
> data files are already empty:
> broker02 root:/mnt/kafkadata/data # ls -al T1*
> T1-0:
> total 8
> drwxr-xr-x  2 kafka kafka     4096 Mar 25 15:41 .
> drwxr-xr-x 47 kafka kafka     4096 Apr  7 11:48 ..
> -rw-r--r--  1 kafka kafka 10485760 Apr  7 11:33 00000000000000000220.index
> -rw-r--r--  1 kafka kafka        0 Apr  6 10:56 00000000000000000220.log
>
> T1-3:
> total 8
> drwxr-xr-x  2 kafka kafka     4096 Mar 25 11:52 .
> drwxr-xr-x 47 kafka kafka     4096 Apr  7 11:48 ..
> -rw-r--r--  1 kafka kafka 10485760 Apr  7 11:33 00000000000000000302.index
> -rw-r--r--  1 kafka kafka        0 Apr  6 10:56 00000000000000000302.log
>
> T1-6:
> total 8
> drwxr-xr-x  2 kafka kafka     4096 Mar 25 14:41 .
> drwxr-xr-x 47 kafka kafka     4096 Apr  7 11:48 ..
> -rw-r--r--  1 kafka kafka 10485760 Apr  7 11:33 00000000000000000297.index
> -rw-r--r--  1 kafka kafka        0 Apr  6 10:56 00000000000000000297.log
>
> T1-9:
> total 8
> drwxr-xr-x  2 kafka kafka     4096 Mar 25 15:55 .
> drwxr-xr-x 47 kafka kafka     4096 Apr  7 11:48 ..
> -rw-r--r--  1 kafka kafka 10485760 Apr  7 11:33 00000000000000000266.index
> -rw-r--r--  1 kafka kafka        0 Apr  6 10:56 00000000000000000266.log
>
>
> To fix the issue we had to change unclean.leader.election.enable from
> false to true and restart all 3 brokers.
> Is that really the intended approach in such a scenario?
>
> Greetings
> Valentin
>

Reply via email to