Hi Jiang, One thing you can try is to set acks=-1, and set the replica.lag.max.messages properly such that it will not kicks all follower replicas immediately under your produce load. Then if one of the follower replica is lagging and the other is not, this one will be dropped out of ISR and when the leader fails, only the one that is still in sync will be chosen.
Guozhang On Fri, Jul 18, 2014 at 5:30 PM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -) <jwu...@bloomberg.net> wrote: > Hi Jun, > > I had some discussion with Guozhang about this; just realize it's not post > to the maillist, so I post them again. > > Setting replica.lag.max.messages=10^12 is to ensure a follower will not be > removed from ISR because it's behind the leader, so larger value is safer. > 10^12 here is used as > "infinite". > > we use replica.lag.max.messages="infinite" together with acks=-1. In this > setting, if all brokers are in sync initially, and only one broker is down > afterwards,then there is no message loss, and producers and consumers will > not be blocked. > > The above is the basic requirment to a fault tolerant system. In more > complicated > scenarios, for example, two brokers fail out of ISR in sequence one at a > time, > then message loss still happens. But we have not figured out any other > settings > that can satisfy the basic requirment. They either block the producer > (acks=3), or > message loss happens when a single broker is down (acks=2 for any > replica.lag.time.max.ms and replica.lag.max.messages values). > > If I missed some features in kafka, or some future updates that will solve > the problem, please let us know. I think some other users are also trying > to figure this out for data-loss 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. You can > observe the max lag in jmx and set the value to be a bit higher than that. > > Thanks, > > Jun > > > On Fri, Jul 18, 2014 at 11:20 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 > LEX -) <jwu...@bloomberg.net> wrote: > > > We tested ack=-1 with replica.lag.max.messages=1000000000000. In this > > config no message loss was found. > > > > This is the only config we found to satisfy 1. no message loss and 2. > > service keeps available when 1 single broker is down. Are there other > > configs that can achieve the same, or stronger 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 since it requires the controller to keep track of this > > information in sync. An alternative solution to your issue would either > be > > using ack=-1 or reduce replica.lag.time.max.ms so that followers not > > keeping up closely will be dropped out of ISR more quickly. > > > > Guozhang > > > > > > On Wed, Jul 16, 2014 at 5:44 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 > > LEX -) <jwu...@bloomberg.net> wrote: > > > > > Guozhong, > > > > > > So this is the cause of message loss in my test where acks=2 and > > > replicas=3: > > > At one moment all 3 replicas, leader L, followers F1 and F2 are in > ISR. A > > > publisher sends a message m to L. F1 fetches m. Both L and F1 > > acknowledge m > > > so the send() is successful. Before F2 fetches m, L is killed and > leader > > > election takes place, and F2 is selected as the new leader. After F2 > > > becomes the leader, it doesn't replicate m from F1, so consumers won't > > > receive the message m. > > > > > > It seems to me that the election here is an unclean leader election > that > > > can be avoided. For example, instead of just choosing the first live > > broker > > > in the ISR as the new leader, choosing the one fetched more messages as > > the > > > new leader may avoid the message loss in the above scenario. Is this a > > > feasible fix? > > > > > > Thanks, > > > Jiang > > > > > > ----- Original Message ----- > > > From: wangg...@gmail.com > > > To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -) > > > At: Jul 15 2014 16:30:56 > > > > > > That is true: when broker becomes a new leader it will stop replicating > > > data from others. However, what you may want to do is tune the > following > > > configs so that replicas will not be easily dropping out of ISR under > > high > > > produce load: > > > > > > 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, it always > happens > > > > that only the leader remains in ISR and shutting down the leader will > > > cause > > > > message loss. > > > > > > > > The leader election code shows that the new leader will be the first > > > alive > > > > broker in the ISR list. So it's possible the new leader 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@kafka.apache.org > > > > At: Jul 15 2014 16:11:17 > > > > > > > > That could be the cause, and it can be verified by changing the acks > to > > > -1 > > > > and checking the data loss ratio then. > > > > > > > > Guozhang > > > > > > > > > > > > On Tue, Jul 15, 2014 at 12:49 PM, Jiang Wu (Pricehistory) (BLOOMBERG/ > > 731 > > > > LEX -) <jwu...@bloomberg.net> wrote: > > > > > > > > > Guozhang,My coworker came up with an explaination: at one moment > the > > > > > leader L, and two followers F1, F2 are all in ISR. The producer > > sends a > > > > > message m1 and receives acks from L and F1. Before the messge is > > > > replicated > > > > > to F2, L is down. In the following leader election, F2, instead of > > F1, > > > > > becomes the leader, and loses m1 somehow. > > > > > Could that be the root cause? > > > > > Thanks, > > > > > Jiang > > > > > > > > > > From: users@kafka.apache.org At: Jul 15 2014 15:05:25 > > > > > To: users@kafka.apache.org > > > > > Subject: Re: message loss for sync producer, acks=2, topic > replicas=3 > > > > > > > > > > Guozhang, > > > > > > > > > > Please find the config below: > > > > > > > > > > Producer: > > > > > > > > > > props.put("producer.type", "sync"); > > > > > > > > > > props.put("request.required.acks", 2); > > > > > > > > > > props.put("serializer.class", "kafka.serializer.StringEncoder"); > > > > > > > > > > props.put("partitioner.class", > > "kafka.producer.DefaultPartitioner"); > > > > > > > > > > props.put("message.send.max.retries", "60"); > > > > > > > > > > props.put("retry.backoff.ms", "300"); > > > > > > > > > > Consumer: > > > > > > > > > > props.put("zookeeper.session.timeout.ms", "400"); > > > > > > > > > > props.put("zookeeper.sync.time.ms", "200"); > > > > > > > > > > props.put("auto.commit.interval.ms", "1000"); > > > > > > > > > > Broker: > > > > > num.network.threads=2 > > > > > num.io.threads=8 > > > > > socket.send.buffer.bytes=1048576 > > > > > socket.receive.buffer.bytes=1048576 > > > > > socket.request.max.bytes=104857600 > > > > > num.partitions=2 > > > > > log.retention.hours=168 > > > > > log.retention.bytes=20000000 > > > > > log.segment.bytes=536870912 > > > > > log.retention.check.interval.ms=60000 > > > > > log.cleaner.enable=false > > > > > zookeeper.connection.timeout.ms=1000000 > > > > > > > > > > Topic: > > > > > Topic:p1r3 PartitionCount:1 ReplicationFactor:3 > > > > > Configs:retention.bytes=10000000000 > > > > > > > > > > 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, acks=2, topic > replicas=3 > > > > > > > > > > What config property values did you use on > producer/consumer/broker? > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > On Tue, Jul 15, 2014 at 10:32 AM, Jiang Wu (Pricehistory) > (BLOOMBERG/ > > > 731 > > > > > LEX -) <jwu...@bloomberg.net> wrote: > > > > > > > > > > > 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 version of Kafka are you using, and did you kill the broker > > > with > > > > > -9? > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > > > > On Tue, Jul 15, 2014 at 9:23 AM, Jiang Wu (Pricehistory) > > (BLOOMBERG/ > > > > 731 > > > > > > LEX -) <jwu...@bloomberg.net> wrote: > > > > > > > > > > > > > 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 > > > > > > > consumer continue to run for a while. After the producer stops, > > the > > > > > > > consumer reports if all messages are received. > > > > > > > > > > > > > > The test was repeated multiple rounds; message loss happened in > > > about > > > > > 10% > > > > > > > of the tests. A typical scenario is as follows: before the > leader > > > is > > > > > > > killed, all 3 replicas are in ISR. After the leader is killed, > > one > > > > > > follower > > > > > > > becomes the leader, and 2 replicas (including the new leader) > are > > > in > > > > > ISR. > > > > > > > Both the producer and consumer pause for several seconds during > > > that > > > > > > time, > > > > > > > and then continue. Message loss happens after the leader is > > killed. > > > > > > > > > > > > > > Because the new leader is in ISR before the old leader is > killed, > > > > > unclean > > > > > > > leader election doesn't explain the message loss. > > > > > > > > > > > > > > I'm wondering if anyone else also observed such message loss? > Is > > > > there > > > > > > any > > > > > > > known issue that may cause the message loss in the above > > scenario? > > > > > > > > > > > > > > Thanks, > > > > > > > Jiang > > > > > > > > > > > > > > > > > > -- > > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > -- Guozhang > > > > > > > > > > > > > > > > > -- > > > -- Guozhang > > > > > > > > > > > > -- > > -- Guozhang > > > > > > > -- -- Guozhang