Re: Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Stevo Slavić
Great feedback, thank you very much to both! Kind regards, Stevo Slavic. On Tue, Jul 7, 2015 at 7:33 PM, Jiangjie Qin wrote: > The replica lag definition now is time based, so as long as a replica can > catch up with leader in replica.lag.time.max.ms, it is in ISR, no matter > how many messages

Re: Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Jiangjie Qin
The replica lag definition now is time based, so as long as a replica can catch up with leader in replica.lag.time.max.ms, it is in ISR, no matter how many messages it is behind. And yes, your understanding is correct - ACK is sent back either when all replica in ISR got the message or the request

Re: Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Stevo Slavić
Thanks for heads up and code reference! Traced back required offset to https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L303 Have to investigate more, but from initial check was expecting to see there reference to "replica.lag.max.messages" (so even

Re: Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Gwen Shapira
Ah, I think I see the confusion: Replicas don't actually ACK at all. What happens is that the replica manager waits for enough ISR replicas to reach the correct offset Partition.checkEnoughReplicasReachOffset(...) has this logic. A replica can't reach offset of second batch, without first having wr

Re: Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Stevo Slavić
Hello Gwen, Thanks for fast response! Btw, congrats on officially becoming a Kafka committer and thanks, among other things, for great "Intro to Kafka" video http://shop.oreilly.com/product/0636920038603.do ! Have to read more docs and/or source. I thought this scenario is possible because repli

Re: Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Gwen Shapira
I am not sure "different replica" can ACK the second back of messages while not having the first - from what I can see, it will need to be up-to-date on the latest messages (i.e. correct HWM) in order to ACK. On Tue, Jul 7, 2015 at 7:13 AM, Stevo Slavić wrote: > Hello Apache Kafka community, > >

Kafka settings for (more) reliable/durable messaging

2015-07-07 Thread Stevo Slavić
Hello Apache Kafka community, Documentation for min.insync.replicas in http://kafka.apache.org/documentation.html#brokerconfigs states: "When used together, min.insync.replicas and request.required.acks allow you to enforce greater durability guarantees. A typical scenario would be to create a to