[ https://issues.apache.org/jira/browse/KAFKA-9211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16989369#comment-16989369 ]
li xiangyuan commented on KAFKA-9211: ------------------------------------- After some test, I believe we have successfully reproduced this problem in test environment. below shows what I did: 1.I start 6 8C16G brokers(kafka 2.3.0 with ) with config: {code:java} KAFKA_HEAP_OPTS=-Xmx6g -Xms6g -XX:MetaspaceSize=196m -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80{code} in server.properties: {code:java} #inter.broker.protocol.version=0.10.0.1 log.message.format.version=0.10.0.1 {code} and then create 1 topic(called large300 below) with 300 partition, 220+ topics that has 40 partition(called a1,a2,and so on), all topics have 2 replicas so whole cluster has totally more than 1W8 partitions, each broker has 3000 partition and half of them (1500) works as leader partition. then this cluster's broker & partition num are similar to our production environment. 2.I run 30 2C4G clients(kafka-clients 0.10.0.1),each one do: * one thread pool has 100 thread,each one execute producerA.send("large300",[1024]random bytes) & producerA.flush() once per second, these threads need execute semaphore.acquire() (in my test code Semaphore semaphore = new Semaphore(44)) before producerA.send() and semaphore.release() after producerA.flush(). consider to network lag .etc, generally 30 clients will produce message to topic large300 6000/sec . * one thread execute code below: {code:java} while (true) { Thread.sleep(5); a++; int loop = a % 10 +1; for(int topicId =1;topicId<=loop;topicId++){ String t = "a"+topicId; String k = String.valueOf(System.currentTimeMillis()); producerB.send(t, data); producerB.flush(); } }{code} generally 30 clients will produce meesage to topic a1(1000message/sec) ~ a10(100message/sec) * start 21 consumer-groups, each group start 30 threads consume message from topic "large300". after do all , each client will has about 1000 connections to all 6 brokers. then I shutdown broker id6 gracefully and restart ,after a auto Preferred Replica Election (or excute manually), we can observe random topic produce speed drop down. when this happened,producer jmx metrics show request latency to node 6 become very high(producer.node.latency.png in attachment).but server jmx(kafka.server.TotalTimeMs.99thPercentile,type=RequestMetrics,name=TotalTimeMs,request=Produce) show produce request handle time didn't increase. finally I use tcpdump catch producer-server connection, I find something doubtful.you can see detail info in attachment: nodelay.txt show before Preferred Replica Election triggered,producer's request send very fast. ackdelay.txt show after Preferred Replica Election triggered, some producer's produce request will split to multi tcp package to send to broker, producer didn't send whole produce request in once but wait until get tcp ack pack , but broker somtimes send back ack package wait about 40ms, this status won't recover even after 12 hours. and execute ss can know tcp rtt become very high: {code:java} ss -i -t -e -m | grep -A 1 "24132"ss -i -t -e -m | grep -A 1 "24132"ESTAB 0 34593 [::ffff:172.30.5.144]:24132 [::ffff:172.30.3.36]:XmlIpcRegSvc timer:(on,208ms,0) uid:1080 ino:15632227 sk:ffff933b84b13180 <-> skmem:(r0,rb65536,t0,tb262144,f7936,w57600,o0,bl0,d0) ts sack cubic wscale:2,0 rto:234 rtt:33.854/9.621 ato:40 mss:1412 rcvmss:536 advmss:1448 cwnd:10 ssthresh:7 bytes_acked:64001581 bytes_received:1366855 segs_out:48417 segs_in:48297 send 3.3Mbps lastsnd:26 lastrcv:150 lastack:26 pacing_rate 6.7Mbps unacked:1 rcv_rtt:26361 rcv_space:34696{code} so if a produce request split into 4 tcp packages,it will cost most 120ms in network. and the worst situation in my test, the tcp connection has windows only 51 bytes with wscale 2, it only allow producer send one tcp package less than 200 bytes,a produce request splited to 18 tcp pack, and broker send back ack would delay 200ms, it will cost 3500ms to send the whole produce request! I believe something happened after Preferred Replica Election(almost the update that combine all partitions leaderandisr requests to one request,in my situation it will ask broker 6 to change partitions status leader & stop fetch as slave for 1500 partitions) ,it trigger tcp Congestion Control,and will not recover forever unless I reinit the producer so it will build another health tcp connection. I dont exactly know whether it's a tcp Congestion Control,and why it couldn't recover, and whether it has any os options to avoid this?I have tried some linux os config (net.ipv4.tcp_low_latency=1,net.ipv4.tcp_adv_win_scale=10 .etc),all of them no use. hope any give me some advice to resolve the problem,thx.. > kafka upgrade 2.3.0 cause produce speed decrease > ------------------------------------------------ > > Key: KAFKA-9211 > URL: https://issues.apache.org/jira/browse/KAFKA-9211 > Project: Kafka > Issue Type: Bug > Components: controller, producer > Affects Versions: 2.3.0 > Reporter: li xiangyuan > Priority: Critical > Attachments: ackdelay.txt, broker-jstack.txt, nodelay.txt, > producer-jstack.txt, producer.node.latency.png > > > Recently we try upgrade kafka from 0.10.0.1 to 2.3.0. > we have 15 clusters in production env, each one has 3~6 brokers. > we know kafka upgrade should: > 1.replcae code to 2.3.0.jar and restart all brokers one by one > 2.unset inter.broker.protocol.version=0.10.0.1 and restart all brokers > one by one > 3.unset log.message.format.version=0.10.0.1 and restart all brokers one > by one > > for now we have already done step 1 & 2 in 12 clusters.but when we try to > upgrade left clusters (already done step 1) in step 2, we found some topics > drop produce speed badly. > we have research this issue for long time, since we couldn't test it in > production environment and we couldn't reproduce in test environment, we > couldn't find the root cause. > now we only could describe the situation in detail as i know, hope anyone > could help us. > > 1.because bug KAFKA-8653, i add code below in KafkaApis.scala > handleJoinGroupRequest function: > {code:java} > if (rebalanceTimeoutMs <= 0) { > rebalanceTimeoutMs = joinGroupRequest.data.sessionTimeoutMs > }{code} > 2.one cluster upgrade failed has 6 8C16G brokers, about 200 topics with 2 > replicas,every broker keep 3000+ partitions and 1500+ leader partition, but > most of them has very low produce message speed,about less than > 50messages/sec, only one topic with 300 partitions has more than 2500 > message/sec with more than 20 consumer groups consume message from it. > so this whole cluster produce 4K messages/sec , 11m Bytes in /sec,240m Bytes > out /sec.and more than 90% traffic made by that topic has 2500messages/sec. > when we unset 5 or 6 servers' inter.broker.protocol.version=0.10.0.1 and > restart, this topic produce message drop to about 200messages/sec, i don't > know whether the way we use could tirgger any problem. > 3.we use kafka wrapped by spring-kafka and set kafkatemplate's > autoFlush=true, so each producer.send execution will execute producer.flush > immediately too.i know flush method will decrease produce performance > dramaticlly, but at least it seems nothing wrong before upgrade step 2. but > i doubt whether it's a problem now after upgrade. > 4.I noticed when produce speed decrease, some consumer group has large > message lag still consume message without any consume speed change or > decrease, so I guess only producerequest speed will drop down,but > fetchrequest not. > 5.we haven't set any throttle configuration, and all producers' acks=1(so > it's not broker replica fetch slow), and when this problem triggered, both > sever & producers cpu usage down, and servers' ioutil keep less than 30% ,so > it shuldn't be a hardware problem. > 6.this event triggered often(almost 100%) most brokers has done upgrade step > 2,then after a auto leader replica election executed, then we can observe > produce speed drop down,and we have to downgrade brokers(set > inter.broker.protocol.version=0.10.0.1)and restart brokers one by one,then it > could be normal. some cluster have to downgrade all brokers,but some cluster > could left 1 or 2 brokers without downgrade, i notice that the broker not > need downgrade is the controller. > 7.I have print jstack for producer & servers. although I do this not the same > cluster, but we can notice that their thread seems really in idle stat. > 8.both 0.10.0.1 & 2.3.0 kafka-client will trigger this problem too. > 8.unless the largest one topic will drop produce speed certainly, other topic > will drop produce speed randomly. maybe topicA will drop speed in first > upgrade attempt but next not, and topicB not drop speed in first attemp but > dropped when do another attempt. > 9.in fact, the largest cluster, has the same topic & group usage scenario > mentioned above, but the largest topic has 1w2 messages/sec,will upgrade fail > in step 1(just use 2.3.0.jar) > any help would be grateful, thx, i'm very sad now... -- This message was sent by Atlassian Jira (v8.3.4#803005)