I have now moved my brokers to local disks (though not dedicated disks). Running the tests again, although there is a significant improvement in max latency during steady state operation, still when I take down the kafka node that has the zookeeper leadership, I get 10-15 second max latencies. I noticed that when this happens, the producer is having to refresh the meta data twice in quick succession, and looking at the metrics link, this happened here too - if you look at 09:29:20 there is a drop in the metadata age, indicating that a partition leadership change must have happened, but then another one happens at 09:29:40. Whey would the reassignment happen twice, as this looks like it is causing a problem.
Tom > On 27 Jun 2017, at 10:17, Tom Dearman <tom.dear...@gmail.com> wrote: > > Hi, > I have a problem with the latency of my kafka producer under some > circumstances. We are running three kafka brokers on version 0.10.2.0 and 3 > zookeepers on version 3.4.8. The server properties are below, the main > producer property that we change is that we require acks=all, so at least 2 > will acknowledge our producer requests as we have min.insync.replicas=2. We > have It all runs on our own servers but in an OpenShift environment. The > zookeeper pods write to local storage, but the kafka broker pods write to > Ceph storage in such a way that a kafka brokers data is kept and re-assigned > to the same broker on restart. I am including a link of kafka producer > metrics that highlights the problem (link is only valid for next 7 days): > > > https://snapshot.raintank.io/dashboard/snapshot/fjfBMC09aBQcWWzj54uCunqMYhNt4ggO > > <https://snapshot.raintank.io/dashboard/snapshot/fjfBMC09aBQcWWzj54uCunqMYhNt4ggO> > > This link has quite a lot of metrics, but the top two are about latency, > request latency and queue time (I assume that the request latency does not > include the time spent in the queue). > @09:29:20, a kafka pod was restarted, the pod was the one which was the > overall zookeeper leader elector. This caused very large latency times for > our messages - average is high, but we are particularly interested in the max > latency, there was also very high queue time which is just as important to us. > @09:31:00 I had to restart our test client which is causing the load to go to > the producer as all 14 threads had stopped since they had waited more than 5 > seconds for a producer send. > @09:34:40 I ran a manual rebalance - this hardly causes a blip in the latency. > @09:38:20 a kafka pod was restored, but this time not the one which was the > overall zookeeper leader elector. This caused a large latency for the > requests and queue time. > @09:40:30 I ran a manual rebalance - again it hardly caused a blip. > > > What I find strange about this is that the rebalance itself seems fine, with > a controlled shut down, the broker is supposed to do a rebalance before > shutting down, so I would have thought everything would be off the closing > broker and the latency of a controlled shut down would be no worse than when > I do a manual rebalance. > > > Please can someone help. > > Tom > > > Our server.properties is: > > broker.id=-1 > > listeners=PLAINTEXT://:9092 <PLAINTEXT://:9092> > > num.network.threads=3 > > num.io.threads=8 > > socket.send.buffer.bytes=102400 > > socket.receive.buffer.bytes=102400 > > socket.request.max.bytes=104857600 > > log.dirs=/mnt/data/logs > > num.partitions=20 > > num.recovery.threads.per.data.dir=1 > > log.retention.hours=168 > > log.segment.bytes=1073741824 > > log.retention.check.interval.ms=300000 > > zookeeper.connect=zookeeper-0:2181,zookeeper-1:2181,zookeeper-2:2181 > > zookeeper.connection.timeout.ms=6000 > > advertised.listeners=PLAINTEXT://kafka-0:9092 <plaintext://kafka-0:9092> > > default.replication.factor=3 > > compression.type=gzip > > delete.topic.enable=true > > offsets.retention.minutes=10080 > > unclean.leader.election.enable=false > > min.insync.replicas=2 > auto.leader.rebalance.enable=false > > leader.imbalance.check.interval.seconds=300 > > leader.imbalance.per.broker.percentage=10 > > > > > > inter.broker.protocol.version=0.10.2 > > > > > > log.message.format.version=0.10.2