[
https://issues.apache.org/jira/browse/KAFKA-4128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15568706#comment-15568706
]
Mazhar Shaikh commented on KAFKA-4128:
--
Hi Gwen Shapira,
My concern for this bug is as below :
1. When ever a follower connects to leader, where follower has more messages
(offset) then leader, then follower truncates/Drop these msg to last
Highwatermark.
=>Here, Do we have any configuration which will avoid this dropping of msg
and instead replicate it to master ?
2. What can be the possible reason for ZookeeperSession timeout, considering
there is no issues with garbage collection.
Broker = 6
replica = 2
Total Partitions : 96,
Partition per broker : 16 (Leader) + 16 (Follower)
> Kafka broker losses messages when zookeeper session times out
> -
>
> Key: KAFKA-4128
> URL: https://issues.apache.org/jira/browse/KAFKA-4128
> Project: Kafka
> Issue Type: Bug
>Affects Versions: 0.8.2.1, 0.9.0.1
>Reporter: Mazhar Shaikh
>Priority: Critical
>
> Pumping 30k msgs/second after some 6-8 hrs of run below logs are printed and
> the messages are lost.
> [More than 5k messages are lost on every partitions]
> Below are few logs:
> [2016-09-06 05:00:42,595] INFO Client session timed out, have not heard from
> server in 20903ms for sessionid 0x256fabec47c0003, closing socket connection
> and attempting reconnect (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:42,696] INFO zookeeper state changed (Disconnected)
> (org.I0Itec.zkclient.ZkClient)
> [2016-09-06 05:00:42,753] INFO Partition [topic,62] on broker 4: Shrinking
> ISR for partition [topic,62] from 4,2 to 4 (kafka.cluster.Partition)
> [2016-09-06 05:00:43,585] INFO Opening socket connection to server
> b0/169.254.2.1:2182. Will not attempt to authenticate using SASL (unknown
> error) (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:43,586] INFO Socket connection established to
> b0/169.254.2.1:2182, initiating session (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:43,587] INFO Unable to read additional data from server
> sessionid 0x256fabec47c0003, likely server has closed socket, closing socket
> connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,644] INFO Opening socket connection to server
> b1/169.254.2.116:2181. Will not attempt to authenticate using SASL (unknown
> error) (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,651] INFO Socket connection established to
> b1/169.254.2.116:2181, initiating session (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,658] INFO zookeeper state changed (Expired)
> (org.I0Itec.zkclient.ZkClient)
> [2016-09-06 05:00:44,659] INFO Initiating client connection,
> connectString=b2.broker.com:2181,b1.broker.com:2181,zoo3.broker.com:2182
> sessionTimeout=15000 watcher=org.I0Itec.zkclient.ZkClient@37b8e86a
> (org.apache.zookeeper.ZooKeeper)
> [2016-09-06 05:00:44,659] INFO Unable to reconnect to ZooKeeper service,
> session 0x256fabec47c0003 has expired, closing socket connection
> (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,661] INFO EventThread shut down
> (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,662] INFO Opening socket connection to server
> b2/169.254.2.216:2181. Will not attempt to authenticate using SASL (unknown
> error) (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,662] INFO Socket connection established to
> b2/169.254.2.216:2181, initiating session (org.apache.zookeeper.ClientCnxn)
> [2016-09-06 05:00:44,665] ERROR Error handling event ZkEvent[New session
> event sent to
> kafka.controller.KafkaController$SessionExpirationListener@33b7dedc]
> (org.I0Itec.zkclient.ZkEventThread)
> java.lang.IllegalStateException: Kafka scheduler has not been started
> at kafka.utils.KafkaScheduler.ensureStarted(KafkaScheduler.scala:114)
> at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:86)
> at
> kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:350)
> at
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1108)
> at
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1107)
> at
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1107)
> at kafka.utils.Utils$.inLock(Utils.scala:535)
> at
> kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1107)
> at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
> at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
>