Great, thanks for the information! So it is definitely acks=all we want to go for. Unfortunately we run into an blocking issue in our production like test environment which we have not been able to find a solution for. So here it is, ANY idea on how we could possibly find a solution is very much appreciated!
Environment: Kafka version: kafka_2.11-0.8.2.1 5 kafka brokers and 5 ZK on spread out on 5 hosts Using new producer (async) Topic: partitions=10 replication-factor=4 min.insync.replicas=2 Default property values used for broker configs and producer. Scenario and problem: Incoming diameter data (10k TPS) is sent to 5 topics via 5 producers which is working great until we start another 5 producers sending to another 5 topics with the same rate (10k). What happens then is that the producers sending to 2 of the topics fills up the buffer and the throughput becomes very low, with BufferExhaustedExceptions for most of the messages. When checking the latency for the problematic topics it becomes really high (around 150ms). Stopping the 5 producers that were started in the second round, the latency goes down to about 1 ms again and the buffer will go back to normal. The load is not that high, about 10MB/s, it is not even near disk bound. So the questions right now are, why do we get such high latency to specifically two topics when starting more producers, even though cpu and disk load looks unproblematic? And why two topics specifically, is there an order of what topics to prfioritize when things get clogged for some reason? Sorry for the quite messy description, we are all kind of new at kafka here! BR Andreas > On 28 Nov 2015, at 09:26, Prabhjot Bharaj <prabhbha...@gmail.com> wrote: > > Hi, > > This should help :) > > During my benchmarks, I noticed that if 5 node kafka cluster running 1 > topic is given a continuous injection of 50GB in one shot (using a modified > producer performance script, which writes my custom data to kafka), the > last replica can sometimes lag and it used to catch up at a speed of 1GB in > 20-25 seconds. This lag increases if producer performance injects 200GB in > one shot. > > I'm not sure how it will behave with multiple topics. it could have an > impact on the overall throughput (because more partitions will be alive on > the same broker thereby dividing the network usage), but I have to test it > in staging environment > > Regards, > Prabhjot > > On Sat, Nov 28, 2015 at 12:10 PM, Gwen Shapira <g...@confluent.io> wrote: > >> Hi, >> >> min.insync.replica is alive and well in 0.9 :) >> >> Normally, you will have 4 our of 4 replicas in sync. However if one of the >> replicas will fall behind, you will have 3 out of 4 in sync. >> If you set min.insync.replica = 3, produce requests will fail if the number >> on in-sync replicas fall below 3. >> >> I hope this helps. >> >> Gwen >> >> On Fri, Nov 27, 2015 at 9:43 PM, Prabhjot Bharaj <prabhbha...@gmail.com> >> wrote: >> >>> Hi Gwen, >>> >>> How about min.isr.replicas property? >>> Is it still valid in the new version 0.9 ? >>> >>> We could get 3 out of 4 replicas in sync if we set it's value to 3. >>> Correct? >>> >>> Thanks, >>> Prabhjot >>> On Nov 28, 2015 10:20 AM, "Gwen Shapira" <g...@confluent.io> wrote: >>> >>>> In your scenario, you are receiving acks from 3 replicas while it is >>>> possible to have 4 in the ISR. This means that one replica can be up to >>>> 4000 messages (by default) behind others. If a leader crashes, there is >>> 33% >>>> chance this replica will become the new leader, thereby losing up to >> 4000 >>>> messages. >>>> >>>> acks = all requires all ISR to ack as long as they are in the ISR, >>>> protecting you from this scenario (but leading to high latency if a >>> replica >>>> is hanging and is just about to drop out of the ISR). >>>> >>>> Also, note that in future versions acks > 1 was deprecated, to protect >>>> against such subtle mistakes. >>>> >>>> Gwen >>>> >>>> On Fri, Nov 27, 2015 at 12:28 AM, Andreas Flinck < >>>> andreas.fli...@digitalroute.com> wrote: >>>> >>>>> Hi all >>>>> >>>>> The reason why I need to know is that we have seen an issue when >> using >>>>> acks=all, forcing us to quickly find an alternative. I leave the >> issue >>>> out >>>>> of this post, but will probably come back to that! >>>>> >>>>> My question is about acks=all and min.insync.replicas property. Since >>> we >>>>> have found a workaround for an issue by using acks>1 instead of all >>>>> (absolutely no clue why at this moment), I would like to know what >>>> benefit >>>>> you get from e.g. acks=all and min.insync.replicas=3 instead of using >>>>> acks=3 in a 5 broker cluster and replication-factor of 4. To my >>>>> understanding you would get the exact level of durability and >> security >>>> from >>>>> using either of those settings. However, I suspect this is not quite >>> the >>>>> case from finding hints without proper explanation that acks=all is >>>>> preferred. >>>>> >>>>> >>>>> Regards >>>>> Andreas >>>> >>> >> > > > > -- > --------------------------------------------------------- > "There are only 10 types of people in the world: Those who understand > binary, and those who don't"