[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15746393#comment-15746393 ]
Jun Rao commented on KAFKA-4477: -------------------------------- [~tdevoe], from the controller log, starting from 19:59:13, the controller keeps failing connecting to broker 1002. What 1002 up at that point? [2016-11-28 19:59:13,140] WARN [Controller-1003-to-broker-1002-send-thread], Controller 1003 epoch 23 fails to send request {controller_id=1003,controller_epoch=23,partition_states=[{topic=__consumer_offsets,partition=18,controller_epoch=22,leader=1002,leader_epoch=25,isr=[1002],zk_version=73,replicas=[1002,1001,1003]},{topic=__consumer_offsets,partition=45,controller_epoch=22,leader=1002,leader_epoch=25,isr=[1002],zk_version=68,replicas=[1002,1003,1001]},{topic=topic_23,partition=0,controller_epoch=22,leader=1002,leader_epoch=10,isr=[1002],zk_version=30,replicas=[1002,1003,1001]},{topic=topic_22,partition=2,controller_epoch=22,leader=1002,leader_epoch=10,isr=[1002],zk_version=26,replicas=[1002,1003,1001]},{topic=topic_3,partition=0,controller_epoch=22,leader=1002,leader_epoch=12,isr=[1002],zk_version=33,replicas=[1002,1001,1003]},{topic=__consumer_offsets,partition=36,controller_epoch=22,leader=1002,leader_epoch=25,isr=[1002],zk_version=72,replicas=[1002,1001,1003]},{topic=connect-offsets,partition=13,controller_epoch=22,leader=1002,leader_epoch=12,isr=[1002],zk_version=37,replicas=[1002,1001,1003]}],live_brokers=[{id=1003,end_points=[{port=9092,host=node_1003,security_protocol_type=2}],rack=null},{id=1002,end_points=[{port=9092,host=node_1002,security_protocol_type=2}],rack=null},{id=1001,end_points=[{port=9092,host=node_1001,security_protocol_type=2}],rack=null}]} to broker node_1002:9092 (id: 1002 rack: null). Reconnecting to broker. (kafka.controller.RequestSendThread) java.io.IOException: Connection to 1002 was disconnected before the response was read at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:115) at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:112) at scala.Option.foreach(Option.scala:257) at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:112) at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:108) at kafka.utils.NetworkClientBlockingOps$.recursivePoll$1(NetworkClientBlockingOps.scala:137) at kafka.utils.NetworkClientBlockingOps$.kafka$utils$NetworkClientBlockingOps$$pollContinuously$extension(NetworkClientBlockingOps.scala:143) at kafka.utils.NetworkClientBlockingOps$.blockingSendAndReceive$extension(NetworkClientBlockingOps.scala:108) at kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:190) at kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:181) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) > Node reduces its ISR to itself, and doesn't recover. Other nodes do not take > leadership, cluster remains sick until node is restarted. > -------------------------------------------------------------------------------------------------------------------------------------- > > Key: KAFKA-4477 > URL: https://issues.apache.org/jira/browse/KAFKA-4477 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 0.10.1.0 > Environment: RHEL7 > java version "1.8.0_66" > Java(TM) SE Runtime Environment (build 1.8.0_66-b17) > Java HotSpot(TM) 64-Bit Server VM (build 25.66-b17, mixed mode) > Reporter: Michael Andre Pearce (IG) > Assignee: Apurva Mehta > Priority: Critical > Labels: reliability > Attachments: issue_node_1001.log, issue_node_1001_ext.log, > issue_node_1002.log, issue_node_1002_ext.log, issue_node_1003.log, > issue_node_1003_ext.log, kafka.jstack, state_change_controller.tar.gz > > > We have encountered a critical issue that has re-occured in different > physical environments. We haven't worked out what is going on. We do though > have a nasty work around to keep service alive. > We do have not had this issue on clusters still running 0.9.01. > We have noticed a node randomly shrinking for the partitions it owns the > ISR's down to itself, moments later we see other nodes having disconnects, > followed by finally app issues, where producing to these partitions is > blocked. > It seems only by restarting the kafka instance java process resolves the > issues. > We have had this occur multiple times and from all network and machine > monitoring the machine never left the network, or had any other glitches. > Below are seen logs from the issue. > Node 7: > [2016-12-01 07:01:28,112] INFO Partition > [com_ig_trade_v1_position_event--demo--compacted,10] on broker 7: Shrinking > ISR for partition [com_ig_trade_v1_position_event--demo--compacted,10] from > 1,2,7 to 7 (kafka.cluster.Partition) > All other nodes: > [2016-12-01 07:01:38,172] WARN [ReplicaFetcherThread-0-7], Error in fetch > kafka.server.ReplicaFetcherThread$FetchRequest@5aae6d42 > (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 7 was disconnected before the response was > read > All clients: > java.util.concurrent.ExecutionException: > org.apache.kafka.common.errors.NetworkException: The server disconnected > before a response was received. > After this occurs, we then suddenly see on the sick machine an increasing > amount of close_waits and file descriptors. > As a work around to keep service we are currently putting in an automated > process that tails and regex's for: and where new_partitions hit just itself > we restart the node. > "\[(?P<time>.+)\] INFO Partition \[.*\] on broker .* Shrinking ISR for > partition \[.*\] from (?P<old_partitions>.+) to (?P<new_partitions>.+) > \(kafka.cluster.Partition\)" -- This message was sent by Atlassian JIRA (v6.3.4#6332)