kafka-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Michal Turek (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-3916) Connection from controller to broker disconnects
Date Fri, 12 Aug 2016 07:43:21 GMT

    [ https://issues.apache.org/jira/browse/KAFKA-3916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15418477#comment-15418477
] 

Michal Turek commented on KAFKA-3916:
-------------------------------------

Hi Jason and others,

I have finally found the root cause for our case. I was grep-ing the controller logs for a
while to hide disturbing stack traces and noticed that all warnings from last month always
mention the same topic, which is quite strange because we have about 50 of them total. This
one is the largest one, with the highest throughput, but not strongly larger than others.
Time stamps of the warnings are also from the afternoon or early evening when Kafka receives
the highest traffic, so I believe the issue is somehow related to performance. There is also
always leader=2, ISR changes between [1,2] and [1,2,3], related partitions are always 2, 7,
11.

While working on a completely different task yesterday, I also by coincidence found that consumption
speed of events buffered in Kafka for this topic reached its limits, other topics were ok.
Producers were only little slower than consumers. So we are trying to add few more partitions
to increase parallelism both in consumers and internally in Kafka (replication). I guess it
will help.

{noformat}
# egrep 'WARN|IOException' controller.log | less -S

[2016-08-10 19:00:14,502] WARN [Controller-1-to-broker-1-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2,3],zk_version=145,replicas=[2,3,1]},{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2,3],zk_version=120,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(1, (host1), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 1 was disconnected before the response was read
[2016-08-10 19:00:14,506] WARN [Controller-1-to-broker-4-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2,3],zk_version=145,replicas=[2,3,1]},{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2,3],zk_version=120,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(4, (host4), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 4 was disconnected before the response was read
[2016-08-10 19:00:14,506] WARN [Controller-1-to-broker-2-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2,3],zk_version=145,replicas=[2,3,1]},{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2,3],zk_version=120,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(2, (host2), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 2 was disconnected before the response was read
[2016-08-10 19:00:14,502] WARN [Controller-1-to-broker-3-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2,3],zk_version=145,replicas=[2,3,1]},{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2,3],zk_version=120,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(3, (host3), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 3 was disconnected before the response was read
[2016-08-10 19:50:41,999] WARN [Controller-1-to-broker-1-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2],zk_version=129,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(1, (host1), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 1 was disconnected before the response was read
[2016-08-10 19:50:42,002] WARN [Controller-1-to-broker-3-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2],zk_version=129,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(3, (host3), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 3 was disconnected before the response was read
[2016-08-10 19:50:42,002] WARN [Controller-1-to-broker-4-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2],zk_version=129,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(4, (host4), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 4 was disconnected before the response was read
[2016-08-10 19:50:42,002] WARN [Controller-1-to-broker-2-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=11,controller_epoch=18,leader=2,leader_epoch=23,isr=[1,2],zk_version=129,replicas=[2,1,3]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(2, (host2), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 2 was disconnected before the response was read
[2016-08-10 20:10:12,044] WARN [Controller-1-to-broker-2-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=7,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=136,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(2, (host2), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 2 was disconnected before the response was read
[2016-08-10 20:10:12,044] WARN [Controller-1-to-broker-1-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=7,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=136,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(1, (host1), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 1 was disconnected before the response was read
[2016-08-10 20:10:12,044] WARN [Controller-1-to-broker-3-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=7,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=136,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(3, (host3), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 3 was disconnected before the response was read
[2016-08-10 20:10:12,044] WARN [Controller-1-to-broker-4-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=7,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=136,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(4, (host4), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 4 was disconnected before the response was read
[2016-08-10 21:04:11,998] WARN [Controller-1-to-broker-1-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=168,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(1, (host1), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 1 was disconnected before the response was read
[2016-08-10 21:04:12,001] WARN [Controller-1-to-broker-4-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=168,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(4, (host4), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 4 was disconnected before the response was read
[2016-08-10 21:04:11,998] WARN [Controller-1-to-broker-2-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=168,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(2, (host2), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 2 was disconnected before the response was read
[2016-08-10 21:04:11,998] WARN [Controller-1-to-broker-3-send-thread], Controller 1 epoch
20 fails to send request {controller_id=1,controller_epoch=20,partition_states=[{topic=(always
same topic),partition=2,controller_epoch=18,leader=2,leader_epoch=26,isr=[1,2],zk_version=168,replicas=[2,3,1]}],live_brokers=[{id=1,end_points=[{port=9092,host=(host1),security_protocol_type=0}]},{id=2,end_points=[{port=9092,host=(host2),security_protocol_type=0}]},{id=3,end_points=[{port=9092,host=(host3),security_protocol_type=0}]},{id=4,end_points=[{port=9092,host=(host4),security_protocol_type=0}]}]}
to broker Node(3, (host3), 9092). Reconnecting to broker. (kafka.controller.RequestSendThread)
java.io.IOException: Connection to 3 was disconnected before the response was read
{noformat}

> Connection from controller to broker disconnects
> ------------------------------------------------
>
>                 Key: KAFKA-3916
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3916
>             Project: Kafka
>          Issue Type: Bug
>          Components: controller
>    Affects Versions: 0.9.0.1
>            Reporter: Dave Powell
>
> We recently upgraded from 0.8.2.1 to 0.9.0.1. Since then, several times per day, the
controllers in our clusters have their connection to all brokers disconnected, and then successfully
reconnected a few hundred ms later. Each time this occurs we see a brief spike in our 99th
percentile produce and consume times, reaching several hundred ms.
> Here is an example of what we're seeing in the controller.log:
> {code}
> [2016-06-28 14:15:35,416] WARN [Controller-151-to-broker-160-send-thread], Controller
151 epoch 106 fails to send request {…} to broker Node(160, broker.160.hostname, 9092).
Reconnecting to broker. (kafka.controller.RequestSendThread)
> java.io.IOException: Connection to 160 was disconnected before the response was read
>         at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:87)
>         at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:84)
>         at scala.Option.foreach(Option.scala:236)
>         at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:84)
>         at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:80)
>         at kafka.utils.NetworkClientBlockingOps$.recurse$1(NetworkClientBlockingOps.scala:129)
>         at kafka.utils.NetworkClientBlockingOps$.kafka$utils$NetworkClientBlockingOps$$pollUntilFound$extension(NetworkClientBlockingOps.scala:139)
>         at kafka.utils.NetworkClientBlockingOps$.blockingSendAndReceive$extension(NetworkClientBlockingOps.scala:80)
>         at kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:180)
>         at kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:171)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> ... one each for all brokers (including the controller) ...
>  [2016-06-28 14:15:35,721] INFO [Controller-151-to-broker-160-send-thread], Controller
151 connected to Node(160, broker.160.hostname, 9092) for sending state change requests (kafka.controller.RequestSendThread)
> … one each for all brokers (including the controller) ...
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message