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

Neha Narkhede updated KAFKA-860:
--------------------------------

    Attachment: kafka-860-v1.patch

The root cause is that during produce request handling, we acquire different 
locks to check if the broker is a leader and then append messages atomically. 
The fix is to move the append to Partition, so that either it is the leader and 
it finishes the append or it rejects the produce request since it is becoming a 
follower. No interleaving should happen.
                
> Replica fetcher thread errors out and dies during rolling bounce of cluster
> ---------------------------------------------------------------------------
>
>                 Key: KAFKA-860
>                 URL: https://issues.apache.org/jira/browse/KAFKA-860
>             Project: Kafka
>          Issue Type: Bug
>          Components: replication
>    Affects Versions: 0.8
>            Reporter: Neha Narkhede
>            Assignee: Neha Narkhede
>            Priority: Blocker
>              Labels: kafka-0.8, p1
>         Attachments: kafka-860-v1.patch
>
>
> 2013/04/10 20:04:32.071 ERROR [ReplicaFetcherThread] 
> [ReplicaFetcherThread-0-272] [kafka] [] [ReplicaFetcherThread-0-272], Error 
> due to 
> kafka.common.KafkaException: error processing data for topic PageViewEvent 
> partititon 3 offset 2482625623
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$4.apply(AbstractFetcherThread.scala:135)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$4.apply(AbstractFetcherThread.scala:113)
>         at scala.collection.immutable.Map$Map1.foreach(Map.scala:105)
>         at 
> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:113)
>         at 
> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:89)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)
> Caused by: java.lang.RuntimeException: Offset mismatch: fetched offset = 
> 2482625623, log end offset = 2482625631.
>         at 
> kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:49)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$4.apply(AbstractFetcherThread.scala:132)
>         ... 5 more
> This causes replica fetcher thread to shut down

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to