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

Joel Koshy commented on KAFKA-1555:
-----------------------------------

Given the current implementation, I think the documentation looks good. I do
have one comment/question on the implementation though (at the end).  I have
a couple of minor typo corrections and suggestions here:

* all (-1) _replicas_
* acknowledgement by all _replicas_
* (_which_ is achieved..)
* _provides_ the _strongest_ durability guarantee
* A message that _has been acknowledged_ by all in-sync replicas will not be 
lost as long as at least one of those in-sync replicas _remains available_.
* The sentence that follows ("Note, however...") contains details that seem 
redundant to what has already been said in parantheses. So we can remove one or 
the other.
* Instead of "To avoid this _unfortunately_ condition" -> _"Although this 
ensures maximum availability of the partition, this behavior may be undesirable 
to some users who prefer durability over availability.  Therefore, we provide 
two topic-level configurations ..."_
* if all replicas _become unavailable_, _then_ the partition will remain 
unavailable until the _most recent_ leader becomes available again. _This 
effectively prefers unavailability over the risk of message loss. See _the_ 
previous section on unclean leader election for more details.
* _Specify_ a minimum ISR size: ... above a certain minimum, _in order_ to 
prevent _the_ loss of messages... just a single replica, which _subsequently_ 
becomes unavailable... guarantees that the message will be _acknowledged at 
least this many in-sync replicas_.
* "The trade-off here" - appears to be in a separate paragraph altogether, but 
it seems it should belong under the second point on min.isr
* Also, perhaps we can rephrase it a bit: _"This setting offers a trade-off 
between consistency and availability. A higher setting for minimum ISR size 
guarantees better consistency since the message is guaranteed to be written to 
more replicas which reduces the probability that it will be lost. However, it 
reduces  availability since the partition will be unavailable for writes if the 
number of in-sync replicas drops below the minimum threshold._

----

My only remaining concern about the current implementation is that min.isr
is a broker-topic config and not explicitly a producer config. However, it
currently takes effect only if {{acks == -1}}. That seems slightly odd to me.
i.e., we could just as well have it take effect even if {{acks == 0/1}} -
i.e., reject the append if the current {{|ISR| < min.isr}} (with the caveat of
NotEnoughReplicasAfterAppend) regardless of ack setting. Do you think this
is uninituitive for users?


> 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-DOCS.0.patch, KAFKA-1555-DOCS.1.patch, 
> KAFKA-1555-DOCS.2.patch, KAFKA-1555.0.patch, KAFKA-1555.1.patch, 
> KAFKA-1555.2.patch, KAFKA-1555.3.patch, KAFKA-1555.4.patch, 
> KAFKA-1555.5.patch, KAFKA-1555.5.patch, KAFKA-1555.6.patch, 
> KAFKA-1555.8.patch, KAFKA-1555.9.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