We have seen this several times and it's quite frustrating. It seems to happen due to the fact that the leader for a partition writes to followers ahead of committing itself, especially for a topic like __consumer_offsets that is written with acks=all. If a brief network interruption occurs (as seems to happen quite regularly for us in a virtualized environment), for a low-to-medium-throughput topic like __consumer_offsets, the follower may recover "too quickly" -- it has more messages than the leader because it was written ahead, but recovers before enough more messages are written to the leader and remaining ISR such that the leader gains a higher high water mark. So the replica halts due to a supposed unclean leader election. Usually, just waiting a minute before restarting the halted broker solves the issue: more messages have been written, the leader has a higher HW, and the replica will happily truncate itself and recover. At least, that is my theory -- it's been a journey understanding Kafka's details well enough!
This happens with 0.10.0 and occurs even with min.insync.replicas=2 (majority of 3 replicas). In fact the problem can be amplified by setting min.isr: if fewer than minimum replicas are available, then it can be impossible to write more messages to the leader as above, so the only way to recover is to delete data files from the halted follower. Similar for very low-throughput topics. At the same time, without min.insync.replicas enforcing a quorum, the risk of a true unclean leader election or data loss is increased -- a double edged sword! It seems related to https://issues.apache.org/jira/browse/KAFKA-3410 or https://issues.apache.org/jira/browse/KAFKA-3861 but happens even under normal conditions (no data loss required!) Anyone else have suggestions? **Brokers halting due to a simple network hiccup is, shall we say, not good!** -Peter > On Jun 25, 2016, at 12:28 PM, Morellato, Wanny <wanny.morell...@concur.com> > wrote: > > Hi all, > > My kafka brokers (0.9.0.1) are refusing to restart and they return the > following error > > Halting because log truncation is not allowed for topic __consumer_offsets, > Current leader 11's latest offset 79445540 is less than replica 13's latest > offset 79445565 (kafka.server.ReplicaFetcherThread) > > Deleting the topic __consumer_offsets from those servers seam to fix the > problem… > > From what I understand this should result in some duplicate delivery… > If that is the case, is it possible to configure kafka in a way that it will > automatically recover from this type of failure? > > Thanks > > Wanny > > > ________________________________ > > This e-mail message is authorized for use by the intended recipient only and > may contain information that is privileged and confidential. If you received > this message in error, please call us immediately at (425) 590-5000 and ask > to speak to the message sender. Please do not copy, disseminate, or retain > this message unless you are the intended recipient. In addition, to ensure > the security of your data, please do not send any unencrypted credit card or > personally identifiable information to this email address. Thank you.