I am not sure if receiving acks from all replicas makes sense though. It
means that none of the replicas can fail. However, the purpose of having
multiple replicas is to be able to tolerate failures.

Thanks,

Jun


On Fri, Jul 11, 2014 at 11:49 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731
LEX -) <jwu...@bloomberg.net> wrote:

> Hi,
> I'm doing stress and failover tests on a 3 node 0.8.1.1 kafka cluster and
> have the following observations.
>
> A topic is created with 1 partition and 3 replications.
> request.required.acks is set to -1 for a sync producer. When the publishing
> speed is high (3M messages, each 2000 bytes, published in lists of size
> 2000), the two followers will fail out of sync. Only the leader remains in
> ISR. But the producer can keep sending. If the leader is killed with CTR_C,
> one follower will become leader, but message loss will happen because of
> the unclean leader election.
>
> In the same test, request.required.acks=3 gives the desired result.
> Followers will fail out of sync, but the producer will be blocked untill
> all followers back to ISR. No data loss is observed in this case.
>
> From the code, this turns out to be how it's designed:
> if ((requiredAcks < 0 && numAcks >= inSyncReplicas.size) ||
> (requiredAcks > 0 && numAcks >= requiredAcks)) {
> /*
> * requiredAcks < 0 means acknowledge after all replicas in ISR
> * are fully caught up to the (local) leader's offset
> * corresponding to this produce request.
> */
> (true, ErrorMapping.NoError)
> }
>
> I'm wondering if it's more reasonable to let request.required.acks=-1 mean
> "receive acks from all replicas" instead of "receive acks from replicas in
> ISR"? As in the above test, follower will fail out sync under high
> publishing volume; that makes request.required.acks=-1 equivalent to
> request.required.acks=1. Since the kafka document states
> request.required.acks=-1 provides the best durability, one would expect it
> is equivalent to request.required.acks=number_of_replications.
>
> Regards,
> Jiang

Reply via email to