[ 
https://issues.apache.org/jira/browse/KAFKA-1555?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14143486#comment-14143486
 ] 

Joe Stein commented on KAFKA-1555:
----------------------------------

[~gwenshap] thanks for the patch! few initial comments/can you please:

1) update review board for others to assist in reviewing/feedback a la 
https://cwiki.apache.org/confluence/display/KAFKA/Patch+submission+and+review 

2) we need to add an extra test in Partition.checkEnoughReplicasReachOffset() 
please (also maybe in ProducerFailureHandlingTest too would make sense please)

3) (nit pick) in one case you use InSync and others Insync could you pick one 
(preferable InSync since that is more consistent with existing code) just for 
consistency.

4) do we want some validation if someone tries to set 0 or -1 or something for 
the min.isr value?

5) * note * once this is committed we should also update 
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes

6) we should also update the public enums in the java producer too 
clients/src/main/java/org/apache/kafka/common/protocol/Errors.java

7) We should have a warning message I think would make sense in KafkaApis 
appendToLocalLog on server side, no?

I should be able to give this a spin in testing later tonight/tomorrow 
(exciting!!!) on a cluster with topics and such.

I like this patch/approach too because people could apply it to 0.8.1.1 easily 
if they wanted (not saying (yet) it should be in 0.8.1.2) and it handles all 
the use cases folks have held stake on this (so far) without major changes or 
confusion in how it works.

> 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: Gwen Shapira
>             Fix For: 0.8.2
>
>         Attachments: KAFKA-1555.0.patch, KAFKA-1555.1.patch
>
>
> 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 three 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.
> 3. ISR can contains only 1 broker, therefore acknowledged messages may be 
> stored in only 1 broker.
> 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. B and C restart and are removed from ISR. Producer sends a 
> message m to A, and receives an acknowledgement. Disk failure happens in A 
> before B and C replicate m. Message m is lost.
> In summary, any existing configuration cannot satisfy the requirements.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to