[
https://issues.apache.org/jira/browse/KAFKA-4128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15568706#comment-15568706
]
Mazhar Shaikh edited comment on KAFKA-4128 at 10/13/16 2:18 PM:
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)
I see this specific error is not handled in kafka code.
/**
* Handle a partition whose offset is out of range and return a new fetch
offset.
*/
def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = {
val replica = replicaMgr.getReplica(topicAndPartition.topic,
topicAndPartition.partition).get
/**
* Unclean leader election: A follower goes down, in the meanwhile the
leader keeps appending messages. The follower comes back up
* and before it has completely caught up with the leader's logs, all
replicas in the ISR go down. The follower is now uncleanly
* elected as the new leader, and it starts appending messages from the
client. The old leader comes back up, becomes a follower
* and it may discover that the current leader's end offset is behind its
own end offset.
*
* In such a case, truncate the current follower's log to the current
leader's end offset and continue fetching.
*
* There is a potential for a mismatch between the logs of the two replicas
here. We don't fix this mismatch as of now.
*/
There is a potential for a mismatch between the logs of the two replicas here.
We don't fix this mismatch as of now.
was (Author: mazhar.shaikh.in):
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]