You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jiang Wu (JIRA)" <ji...@apache.org> on 2014/07/22 20:46:40 UTC

[jira] [Created] (KAFKA-1555) provide strong consistency with reasonable availability

Jiang Wu created KAFKA-1555:
-------------------------------

             Summary: provide strong consistency with reasonable availability
                 Key: KAFKA-1555
                 URL: https://issues.apache.org/jira/browse/KAFKA-1555
             Project: Kafka
          Issue Type: Improvement
          Components: controller
    Affects Versions: 0.8.1.1
            Reporter: Jiang Wu
            Assignee: Neha Narkhede


In a mission critical application, we expect a kafka cluster with 3 brokers can satisfy two requirements:
1. When 1 broker is down, no message loss or service blocking happens.
2. In worse cases such as two brokers are down, service can be blocked, but no message loss happens.

We found that current kafka versoin (0.8.1.1) cannot achieve the requirements due to its two behaviors:
1. when choosing a new leader from 2 followers in ISR, the one with less messages may be chosen as the leader.
2. even when replica.lag.max.messages=0, a follower can stay in ISR when it has less messages than the leader.

The following is an analytical proof. 
We consider a cluster with 3 brokers and a topic with 3 replicas, and assume that at the beginning, all 3 replicas, leader A, followers B and C, are in sync, i.e., they have the same messages and are all in ISR.
According to the value of request.required.acks (acks for short), there are the following cases.
1. acks=0, 1, 3. Obviously these settings do not satisfy the requirement.
2. acks=2. Producer sends a message m. It's acknowledged by A and B. At this time, although C hasn't received m, C is still in ISR. If A is killed, C can be elected as the new leader, and consumers will miss m.
3. acks=-1. Suppose replica.lag.max.messages=M. There are two sub-cases:
3.1 M>0. Suppose C be killed. C will be out of ISR after replica.lag.time.max.ms. Then the producer publishes M messages to A and B. C restarts. C will join in ISR since it is M messages behind A and B. Before C replicates all messages, A is killed, and C becomes leader, then message loss happens.
3.2 M=0. In this case, when the producer publishes at a high speed, B and C will fail out of ISR; only A keeps receiving messages. Then A is killed. Either message loss or service blocking will happen, depending on whether unclean leader election is disabled (a new feature will be in 0.8.2, see jira KAFKA-1028).

In summary, any existing configuration cannot satisfy the requirements.



--
This message was sent by Atlassian JIRA
(v6.2#6252)