[ 
https://issues.apache.org/jira/browse/KAFKA-2165?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jun Rao updated KAFKA-2165:
---------------------------
    Status: In Progress  (was: Patch Available)

Thanks for reporting this. A couple of comments.

1. Currently, if we get any exception other than OffsetOutOfRangeException, we 
won't move the fetchOffset and won't append the data to the log. So, the 
fetchOffset and replica.logEndOffset should still be consistent, right? Also, 
do you know what kind of exception you got?
2. I am not sure if the check added in the patch is necessary. When the replica 
fetcher thread gets an OffsetOutOfRangeException, it either overflows or 
underflows. The additional test that you added tests for underflow in the 
branch that handles underflow.


> ReplicaFetcherThread: data loss on unknown exception
> ----------------------------------------------------
>
>                 Key: KAFKA-2165
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2165
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2.1
>            Reporter: Alexey Ozeritskiy
>         Attachments: KAFKA-2165.patch
>
>
> Sometimes in our cluster some replica gets out of the isr. Then broker 
> redownloads the partition from the beginning. We got the following messages 
> in logs:
> {code}
> # The leader:
> [2015-03-25 11:11:07,796] ERROR [Replica Manager on Broker 21]: Error when 
> processing fetch request for partition [topic,11] offset 54369274 from 
> follower with correlation id 2634499. Possible cause: Request for offset 
> 54369274 but we only have log segments in the range 49322124 to 54369273. 
> (kafka.server.ReplicaManager)
> {code}
> {code}
> # The follower:
> [2015-03-25 11:11:08,816] WARN [ReplicaFetcherThread-0-21], Replica 31 for 
> partition [topic,11] reset its fetch offset from 49322124 to current leader 
> 21's start offset 49322124 (kafka.server.ReplicaFetcherThread)
> [2015-03-25 11:11:08,816] ERROR [ReplicaFetcherThread-0-21], Current offset 
> 54369274 for partition [topic,11] out of range; reset offset to 49322124 
> (kafka.server.ReplicaFetcherThread)
> {code}
> This occures because we update fetchOffset 
> [here|https://github.com/apache/kafka/blob/0.8.2/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L124]
>  and then try to process message. 
> If any exception except OffsetOutOfRangeCode occures we get unsynchronized 
> fetchOffset and replica.logEndOffset.
> On next fetch iteration we can get 
> fetchOffset>replica.logEndOffset==leaderEndOffset and OffsetOutOfRangeCode.



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

Reply via email to