Maysam Yabandeh created KAFKA-3861:
--------------------------------------
Summary: Shrunk ISR before leader crash makes the partition
unavailable
Key: KAFKA-3861
URL: https://issues.apache.org/jira/browse/KAFKA-3861
Project: Kafka
Issue Type: Bug
Affects Versions: 0.10.0.0
Reporter: Maysam Yabandeh
We observed a case that the leader experienced a crash and lost its in-memory
data and latest HW offsets. Normally Kafka should be safe and be able to make
progress with a single node failure. However a few seconds before the crash the
leader shrunk its ISR to itself, which is safe since min-in-sync-replicas is 2
and replication factor is 3 thus the troubled leader cannot accept new produce
messages. After the crash however the controller could not name any of the of
the followers as the new leader since as far as the controller knows they are
not in ISR and could potentially be behind the last leader. Note that
unclean-leader-election is disabled in this cluster since the cluster requires
a very high degree of durability and cannot tolerate data loss.
One hackish solution would be that the admin investigates the logs, determine
that unclean-leader-election in this particular case would be safe and
temporarily enables it until new leaders are selected for affected partitions.
This manual process is however slow and error-prone and the cluster will suffer
partial unavailability in the meanwhile.
We are thinking of having the controller make an exception for this case: if
ISR size is less than min-in-sync-replicas and the new leader would be -1, then
the controller does an RPC to all the replicas and inquire of the latest
offset, and if all the replicas responded then chose the one with the largest
offset as the leader as well as the new ISR. Note that the controller cannot do
that if any of the non-leader replicas do not respond since there might be a
case that the responding replicas have not been involved the last ISR and hence
potentially behind the others (and the controller could not know that since it
does not keep track of previous ISR).
Pros would be that kafka will be safely available when such cases occur and
would not require any admin intervention. The cons however is that the
controller talking to brokers inside the leader election function would break
the existing pattern in the source code as currently the leader is elected
locally without requiring any additional RPC.
Thoughts?
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)