Re: Achieving Consistency and Durability

2014-10-24 Thread Kyle Banker
Looks great, Gwen. I've added a few comments to the ticket. On Mon, Oct 20, 2014 at 2:32 PM, Gwen Shapira wrote: > Hi Kyle, > > I added new documentation, which will hopefully help. Please take a look > here: > https://issues.apache.org/jira/browse/KAFKA-1555 > > I've heard rumors that you are v

Re: Achieving Consistency and Durability

2014-10-20 Thread Gwen Shapira
Hi Kyle, I added new documentation, which will hopefully help. Please take a look here: https://issues.apache.org/jira/browse/KAFKA-1555 I've heard rumors that you are very very good at documenting, so I'm looking forward to your comments. Note that I'm completely ignoring the acks>1 case since

Re: Achieving Consistency and Durability

2014-10-15 Thread Kyle Banker
Thanks very much for these clarifications, Gwen. I'd recommend modifying the following phrase describing "acks=-1": "This option provides the best durability, we guarantee that no messages will be lost as long as at least one in sync replica remains." The "as long as at least one in sync replica

Re: Achieving Consistency and Durability

2014-10-14 Thread Gwen Shapira
ack = 2 *will* throw an exception when there's only one node in ISR. The problem with ack=2 is that if you have 3 replicas and you got acks from 2 of them, the one replica which did not get the message can still be in ISR and get elected as leader, leading for a loss of the message. If you specify

Re: Achieving Consistency and Durability

2014-10-14 Thread Scott Reynolds
A question about 0.8.1.1 and acks. I was under the impression that setting acks to 2 will not throw an exception when there is only one node in ISR. Am I incorrect ? Thus the need for min_isr. On Tue, Oct 14, 2014 at 11:50 AM, Kyle Banker wrote: > It's quite difficult to infer from the docs the

Achieving Consistency and Durability

2014-10-14 Thread Kyle Banker
It's quite difficult to infer from the docs the exact techniques required to ensure consistency and durability in Kafka. I propose that we add a doc section detailing these techniques. I would be happy to help with this. The basic question is this: assuming that I can afford to temporarily halt pr