For supporting more durability at the expense of availability, we have a
JIRA that we will fix on trunk. This will allow you to configure the
default as well as per topic durability vs availability behavior  -
https://issues.apache.org/jira/browse/KAFKA-1028

Thanks,
Neha


On Fri, Nov 1, 2013 at 1:43 PM, Joel Koshy <jjkosh...@gmail.com> wrote:

> >>>>
> >>>> Unclean shutdown could result in data loss - since you are moving
> >>>> leadership to a replica that has fallen out of ISR. i.e., it's log end
> >>>> offset is behind the last committed message to this partition.
> >>>>
> >>>>
> >>> But if data is written with 'request.required.acks=-1', no data should
> be
> >>> lost, no?  Or will partitions be truncated wholesale after an unclean
> >>> shutdown?
>
> Sorry about the delayed reply to this, but it is an important point -
> data can be lost even in this case. -1 means ack after all replicas in
> the current ISR have received the message. So for example:
> - assigned replicas for some partition = 0,1
> - ISR = 0, leader = 0; (so 1 is out of the ISR - say if broker 0 is
> slow (but up))
> - messages with acks=-1 will get committed (i.e., the producer will
> receive the ack)
> - shutdown broker 0. So this will result in an unclean leader election
> since there are no other replicas in ISR; and if the broker 1 is up it
> will become the new leader
> - however it only has messages until the earlier watermark
> - when broker 0 comes back up it will become a follower to 1. Its
> initial fetch will be for its last checkpointed high watermark (which
> is larger than the current leader's hw), but the leader will piggy
> back the current hw in the fetch response to the follower which will
> then truncate its log to the leader's current hw. At least this is my
> recollection of how it should be - but I'm staring at the code and
> think there may be an issue with it. Will update this thread if this
> is incorrect.
>
> Thanks,
>
> Joel
>

Reply via email to