Kafka
  1. Kafka
  2. KAFKA-3916

Connection from controller to broker disconnects

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.9.0.1
    • Fix Version/s: 0.10.1.0
    • Component/s: controller
    • Labels:
      None

      Description

      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:

      [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) ...
      

        Issue Links

          Activity

          Hide
          Andrey Konyaev added a comment -

          I have this problem with 0.10 vers.

          Show
          Andrey Konyaev added a comment - I have this problem with 0.10 vers.
          Hide
          Michal Turek added a comment -

          We saw this issue yesterday, I don't know if this helps, but it may be useful while debugging.

          • Kafka 0.9.0.1 and 0.9.0.1 clients.
          • There is ISR shrink and immediate ISR expand visible in graphs based on JMX of Kafka brokers.
          • Consumers were unable to commit offsets at that time.
          2016-08-02 14:25:29.589 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-7]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.560 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-2]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.562 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-0]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.563 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-5]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.570 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-6]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.570 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-3]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.570 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-4]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.572 INFO  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-1]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry
          2016-08-02 14:25:52.572 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-7]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.573 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-2]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.573 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-0]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.574 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-5]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.575 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-6]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.575 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-3]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.575 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-4]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.576 INFO  o.a.k.c.c.internals.AbstractCoordinator       [Consumer-1]: Marking the coordinator 2147483646 dead.
          2016-08-02 14:25:52.576 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-7]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.577 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-2]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.577 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-0]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.577 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-5]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.578 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-6]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.578 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-3]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.578 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-4]: Auto offset commit failed: The request timed out.
          2016-08-02 14:25:52.579 WARN  o.a.k.c.c.internals.ConsumerCoordinator       [Consumer-1]: Auto offset commit failed: The request timed out.
          
          Show
          Michal Turek added a comment - We saw this issue yesterday, I don't know if this helps, but it may be useful while debugging. Kafka 0.9.0.1 and 0.9.0.1 clients. There is ISR shrink and immediate ISR expand visible in graphs based on JMX of Kafka brokers. Consumers were unable to commit offsets at that time. 2016-08-02 14:25:29.589 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-7]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.560 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-2]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.562 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-0]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.563 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-5]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.570 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-6]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.570 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-3]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.570 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-4]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.572 INFO o.a.k.c.c.internals.ConsumerCoordinator [Consumer-1]: Offset commit for group ... failed due to REQUEST_TIMED_OUT, will find new coordinator and retry 2016-08-02 14:25:52.572 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-7]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.573 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-2]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.573 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-0]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.574 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-5]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.575 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-6]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.575 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-3]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.575 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-4]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.576 INFO o.a.k.c.c.internals.AbstractCoordinator [Consumer-1]: Marking the coordinator 2147483646 dead. 2016-08-02 14:25:52.576 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-7]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.577 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-2]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.577 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-0]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.577 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-5]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.578 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-6]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.578 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-3]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.578 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-4]: Auto offset commit failed: The request timed out. 2016-08-02 14:25:52.579 WARN o.a.k.c.c.internals.ConsumerCoordinator [Consumer-1]: Auto offset commit failed: The request timed out.
          Hide
          Michal Turek added a comment - - edited

          Production of messages was failing for few seconds this time. I found no issue in logs of consumers. Exceptions in controller.log are the same as in description. Multiple ISR shrinks and expands happened during 6:30 pm to 9:30 pm based on our monitoring and graphs, then the issue disappeared.

          2016-08-10 20:01:40.611 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          2016-08-10 20:01:40.683 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          ...
          2016-08-10 20:01:43.714 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          2016-08-10 20:01:43.714 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          2016-08-10 20:01:43.838 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-3, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.840 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.840 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.840 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-8, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.841 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-4, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.841 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.841 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-10, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.842 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.842 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-6, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.842 WARN  o.a.kafka.clients.producer.internals.Sender   [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-6, retrying (0 attempts left). Error: NETWORK_EXCEPTION
          2016-08-10 20:01:43.919 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          2016-08-10 20:01:43.920 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          ...
          2016-08-10 20:01:43.971 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          2016-08-10 20:01:43.972 WARN  (application class)  [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          
          Show
          Michal Turek added a comment - - edited Production of messages was failing for few seconds this time. I found no issue in logs of consumers. Exceptions in controller.log are the same as in description. Multiple ISR shrinks and expands happened during 6:30 pm to 9:30 pm based on our monitoring and graphs, then the issue disappeared. 2016-08-10 20:01:40.611 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired 2016-08-10 20:01:40.683 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired ... 2016-08-10 20:01:43.714 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired 2016-08-10 20:01:43.714 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired 2016-08-10 20:01:43.838 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-3, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.840 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.840 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.840 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-8, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.841 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-4, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.841 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.841 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-10, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.842 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-0, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.842 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-6, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.842 WARN o.a.kafka.clients.producer.internals.Sender [kafka-producer-network-thread | (consumer group)]: Got error produce response with correlation id 113676841 on topic-partition (topic)-6, retrying (0 attempts left). Error: NETWORK_EXCEPTION 2016-08-10 20:01:43.919 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired 2016-08-10 20:01:43.920 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired ... 2016-08-10 20:01:43.971 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired 2016-08-10 20:01:43.972 WARN (application class) [kafka-producer-network-thread | (consumer group)]: Write to Kafka failed: (topic), org.apache.kafka.common.errors.TimeoutException: Batch Expired
          Hide
          Jason Gustafson added a comment -

          Could we be hitting the idle connection timeout? The default setting for connections.max.idle.ms is 10 minutes. When the cluster is stable, I'm not sure there's much traffic from the controller, so we may proactively disconnect the connection. Perhaps someone can try increasing it?

          Show
          Jason Gustafson added a comment - Could we be hitting the idle connection timeout? The default setting for connections.max.idle.ms is 10 minutes. When the cluster is stable, I'm not sure there's much traffic from the controller, so we may proactively disconnect the connection. Perhaps someone can try increasing it?
          Hide
          Michal Turek added a comment -

          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.

          # 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
          
          Show
          Michal Turek added a comment - 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. # 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
          Hide
          ASF GitHub Bot added a comment -

          GitHub user hachikuji opened a pull request:

          https://github.com/apache/kafka/pull/1734

          KAFKA-3916: Check for disconnects properly before sending from the controller

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/hachikuji/kafka KAFKA-3916

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/kafka/pull/1734.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #1734


          commit 1cc38623c047128d9c83309c66f37491726c8082
          Author: Jason Gustafson <jason@confluent.io>
          Date: 2016-08-12T00:40:32Z

          KAFKA-3916: Check for disconnects properly before sending from the controller


          Show
          ASF GitHub Bot added a comment - GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1734 KAFKA-3916 : Check for disconnects properly before sending from the controller You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-3916 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/1734.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1734 commit 1cc38623c047128d9c83309c66f37491726c8082 Author: Jason Gustafson <jason@confluent.io> Date: 2016-08-12T00:40:32Z KAFKA-3916 : Check for disconnects properly before sending from the controller
          Hide
          Ismael Juma added a comment -

          Issue resolved by pull request 1734
          https://github.com/apache/kafka/pull/1734

          Show
          Ismael Juma added a comment - Issue resolved by pull request 1734 https://github.com/apache/kafka/pull/1734
          Hide
          ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/kafka/pull/1734

          Show
          ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1734

            People

            • Assignee:
              Jason Gustafson
              Reporter:
              Dave Powell
            • Votes:
              11 Vote for this issue
              Watchers:
              22 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development