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

Neha Narkhede updated KAFKA-654:
--------------------------------

    Attachment: kafka-654-v1.patch

Patch to fix the problem on the follower as explained above. However, I 
couldn't reproduce this issue on the leader. I guess we can file another JIRA 
if we are able to find that issue again
                
> Irrecoverable error while trying to roll a segment that already exists
> ----------------------------------------------------------------------
>
>                 Key: KAFKA-654
>                 URL: https://issues.apache.org/jira/browse/KAFKA-654
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8
>            Reporter: Neha Narkhede
>            Priority: Blocker
>         Attachments: kafka-654-v1.patch
>
>
> I tried setting up a 5 broker 0.8 cluster and sending messages to 100s of 
> topics on it. For a couple of topic partitions, the produce requests never 
> succeed since they fail on the leader with the following error - 
> [2012-12-05 22:54:05,711] WARN [Kafka Log on Broker 2], Newly rolled segment 
> file 0000000000000000000
> 0.log already exists; deleting it first (kafka.log.Log)
> [2012-12-05 22:54:05,711] WARN [Kafka Log on Broker 2], Newly rolled segment 
> file 0000000000000000000
> 0.index already exists; deleting it first (kafka.log.Log)
> [2012-12-05 22:54:05,715] ERROR [ReplicaFetcherThread-1-0-on-broker-2], Error 
> due to  (kafka.server.R
> eplicaFetcherThread)
> kafka.common.KafkaException: Trying to roll a new log segment for topic 
> partition NusWriteEvent-4 with start offset 0 while it already exsits
>         at kafka.log.Log.rollToOffset(Log.scala:456)
>         at kafka.log.Log.roll(Log.scala:434)
>         at kafka.log.Log.maybeRoll(Log.scala:423)
>         at kafka.log.Log.append(Log.scala:257)
>         at 
> kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:51)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:125)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:108)
>         at 
> scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:125)
>         at 
> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:344)
>         at 
> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:344)
>         at 
> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:108)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:50)

--
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