Re: how to ensure strong consistency with reasonable availabilit

2014-07-23 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
first fetch the 5 newly committed messages before being added to ISR. Are you observing data loss in this case? Thanks, Jun On Tue, Jul 22, 2014 at 6:21 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -) jwu...@bloomberg.net wrote: kafka-1028 addressed another unclean leader election problem

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-21 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
at 5:02 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -) jwu...@bloomberg.net wrote: Guozhang, You mean a replica.lag.max.messages value that ensures only one follower can fail out of ISR? I think for any value, if one follower fails out of ISR, then the other may fail too. So this doesn't

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-18 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
consistency while keep the availability level? Thanks, Jiang - Original Message - From: wangg...@gmail.com To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org At: Jul 16 2014 11:34:13 Selecting replicas in ISR based on their fetched messages would be quite complicated

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-18 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
sensitive applications. Thanks, Jiang From: users@kafka.apache.org At: Jul 18 2014 18:41:00 To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org Subject: Re: message loss for sync producer, acks=2, topic replicas=3 You probably don't need to set replica.lag.max.messages that high

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-16 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
: replica.lag.max.messages replica.lag.time.max.ms You can get their description here: http://kafka.apache.org/documentation.html#brokerconfigs Guozhang On Tue, Jul 15, 2014 at 1:25 PM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -) jwu...@bloomberg.net wrote: When ack=-1 and the publisher thread number is high

message loss for sync producer, acks=2, topic replicas=3

2014-07-15 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
Hi, I observed some unexpected message loss in kafka fault tolerant test. In the test, a topic with 3 replicas is created. A sync producer with acks=2 publishes to the topic. A consumer consumes from the topic and tracks message ids. During the test, the leader is killed. Both producer and

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-15 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
Guozhang, I'm testing on 0.8.1.1; just kill pid, no -9. Regards, Jiang From: users@kafka.apache.org At: Jul 15 2014 13:27:50 To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org Subject: Re: message loss for sync producer, acks=2, topic replicas=3 Hello Jiang, Which

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-15 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
log.cleaner.enable=false zookeeper.connection.timeout.ms=100 Topic: Topic:p1r3 PartitionCount:1ReplicationFactor:3 Configs:retention.bytes=100 Thanks, Jiang From: users@kafka.apache.org At: Jul 15 2014 13:59:03 To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-15 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
Topic: Topic:p1r3 PartitionCount:1ReplicationFactor:3 Configs:retention.bytes=100 Thanks, Jiang From: users@kafka.apache.org At: Jul 15 2014 13:59:03 To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org Subject: Re: message loss for sync producer

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-15 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
will be behind the followers. It seems that after a broker becomes a leader, it stops replicating from others even when it hasn't received all available messages? Regards, Jiang - Original Message - From: wangg...@gmail.com To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users

Re: request.required.acks=-1 under high data volume

2014-07-14 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
. Regards, Jiang From: users@kafka.apache.org At: Jul 11 2014 18:27:46 To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org Cc: wangg...@gmail.com Subject: Re: request.required.acks=-1 under high data volume I think the root problem is that replicas are falling behind and hence

Re: request.required.acks=-1 under high data volume

2014-07-11 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
another ack, -2 for instance, to specify receive acks from all replicas as a favor of consistency. Since you already did this much investigation would you like to file a JIRA and submit a patch for this? Guozhang On Fri, Jul 11, 2014 at 11:49 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -) jwu