Pengwei created KAFKA-5480:
------------------------------
Summary: Partition Leader may not be elected although there is one
live replica in ISR
Key: KAFKA-5480
URL: https://issues.apache.org/jira/browse/KAFKA-5480
Project: Kafka
Issue Type: Bug
Affects Versions: 0.10.2.0, 0.9.0.1
Reporter: Pengwei
Currently we found a consumer blocking in the poll because of the coordinator
of this consumer group is not available.
Digging in the log, we found some of the __consumer_offsets' partitions' leader
are -1, so the coordinator not available is
because of leader is not available, the scene is as follow:
There are 3 brokers in the cluster, and the network of the cluster is not
stable. At the beginning, the partition [__consumer_offsets,3]
Leader is 3, ISR is [3, 1, 2]
1. Broker 1 become the controller:
[2017-06-10 15:48:30,006] INFO [Controller 1]: Broker 1 starting become
controller state transition (kafka.controller.KafkaController)
[2017-06-10 15:48:30,085] INFO [Controller 1]: Initialized controller epoch to
8 and zk version 7 (kafka.controller.KafkaController)
[2017-06-10 15:48:30,088] INFO [Controller 1]: Controller 1 incremented epoch
to 9 (kafka.controller.KafkaController)
2. Broker 2 soon becomes the controller, it is aware of all the brokers:
[2017-06-10 15:48:30,936] INFO [Controller 2]: Broker 2 starting become
controller state transition (kafka.controller.KafkaController)
[2017-06-10 15:48:30,936] INFO [Controller 2]: Initialized controller epoch to
9 and zk version 8 (kafka.controller.KafkaController)
[2017-06-10 15:48:30,943] INFO [Controller 2]: Controller 2 incremented epoch
to 10 (kafka.controller.KafkaController)
[2017-06-10 15:48:31,574] INFO [Controller 2]: Currently active brokers in the
cluster: Set(1, 2, 3) (kafka.controller.KafkaController)
[2017-06-10 15:48:31,574] INFO [Controller 2]: Currently shutting brokers in
the cluster: Set() (kafka.controller.KafkaController)
So broker 2 think Leader 3 is alive, does not need to elect leader.
3. Broker 1 is not resign until 15:48:32, but it is not aware of the broker 3:
[2017-06-10 15:48:31,470] INFO [Controller 1]: List of partitions to be
deleted: Map() (kafka.controller.KafkaController)
[2017-06-10 15:48:31,470] INFO [Controller 1]: Currently active brokers in the
cluster: Set(1, 2) (kafka.controller.KafkaController)
[2017-06-10 15:48:31,470] INFO [Controller 1]: Currently shutting brokers in
the cluster: Set() (kafka.controller.KafkaController)
and change the Leader to broker 1:
[2017-06-10 15:48:31,847] DEBUG [OfflinePartitionLeaderSelector]: Some broker
in ISR is alive for [__consumer_offsets,3]. Select 1 from ISR 1,2 to be the
leader. (kafka.controller.OfflinePartitionLeaderSelector)
broker 1 resign until 15:48:32 when the zk client is aware of the broker 2 has
change the controller's data:
kafka.common.ControllerMovedException: Broker 1 received update metadata
request with correlation id 4 from an old controller 1 with epoch 9. Latest
known controller epoch is 10
at
kafka.server.ReplicaManager.maybeUpdateMetadataCache(ReplicaManager.scala:621)
at
kafka.server.KafkaApis.handleUpdateMetadataRequest(KafkaApis.scala:163)
at kafka.server.KafkaApis.handle(KafkaApis.scala:76)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
at java.lang.Thread.run(Thread.java:748)
[2017-06-10 15:48:32,307] INFO New leader is 2
(kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
4. Then broker 2's controllerContext.partitionLeadershipInfo cached the Leader
is 3 ISR is [3,1,2], but in zk
Leader is 1 ISR is [1, 2]. It will keep this a long time until another zk
event happen.
5. After 1 day, broker 2 received the broker 1's broker change event:
[2017-06-12 21:43:18,287] INFO [BrokerChangeListener on Controller 2]: Broker
change listener fired for path /brokers/ids with children 2,3
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
[2017-06-12 21:43:18,293] INFO [BrokerChangeListener on Controller 2]: Newly
added brokers: , deleted brokers: 1, all live brokers: 2,3
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
then broker 2 will invoke onBrokerFailure for the deleted broker 1, but
because Leader is 3, it will not change the partition to OfflinePartition and
will not change the leader in
partitionStateMachine.triggerOnlinePartitionStateChange().
But in the replicaStateMachine.handleStateChanges(activeReplicasOnDeadBrokers,
OfflineReplica), it will remove the replica 1 in ISR.
In the removeReplicaFromIsr, controller will read the ISR from zk again, will
find Leader change to 1, then it will change
the partition's leader to -1 and ISR [2]:
[2017-06-12 21:43:19,158] DEBUG [Controller 2]: Removing replica 1 from ISR
1,3,2 for partition [__consumer_offsets,3]. (kafka.controller.KafkaController)
[2017-06-12 21:43:19,160] INFO [Controller 2]: New leader and ISR for partition
[__consumer_offsets,3] is {"leader":-1,"leader_epoch":15,"isr":[2]}
(kafka.controller.KafkaController)
So the [__consumer_offsets,3] partition's leader is -1, although replica 2 is
alive and in ISR, it will keep this a long time until another relative zk event
happen, for example, reboot one of the broker who have the replicas.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)