[ 
https://issues.apache.org/jira/browse/KAFKA-588?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13486293#comment-13486293
 ] 

Jay Kreps commented on KAFKA-588:
---------------------------------

Jun the OffsetIndex.truncateTo logic looks right to me and we have a number of 
tests on this so I am wondering if that is the problem. How was this broker 
shut down (or was it shutdown at all?). I ask because the DumpSegment tool just 
dumps what is on disk. In the case of the OffsetIndex the end of file pointer 
is just in memory and truncation just moves that pointer. I wonder if  the 
truncate wasn't correct but somehow the lastOffset wasn't set correctly and as 
a result we are giving the error incorrectly. If the broker wen through clean 
shutdown this should not be possible, but if not it is possible.

One bug I think I see is that we initialize the lastOffset to be the baseOffset 
of the segment, but this means that if the first message is entered into the 
index with that same baseOffset (which is legitimate) it would produce an 
error. Normally I think we don't hit this just because our index interval isn't 
1, however after a truncate we don't reset the bytesSinceLastIndexEntry so this 
becomes possible even with a larger index interval. Do you know how many 
entries were in that index file that you checked? I am pretty sure this is a 
bug, but am not sure if it caused this error (it could cause it though).
                
> Index truncation doesn't seem to remove the last entry properly
> ---------------------------------------------------------------
>
>                 Key: KAFKA-588
>                 URL: https://issues.apache.org/jira/browse/KAFKA-588
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.8
>            Reporter: Jun Rao
>            Assignee: Jay Kreps
>            Priority: Blocker
>              Labels: bugs
>
> [2012-10-26 08:04:13,333] INFO [Kafka Log on Broker 3], Truncated log segment 
> /tmp/kafka_server_3_logs/test_1-0/00000000000000130500.log to target offset 
> 429050 (kafka.log.
> Log)
> [2012-10-26 08:04:13,333] INFO [ReplicaFetcherManager on broker 3] adding 
> fetcher on topic test_1, partion 0, initOffset 429050 to broker 2 with 
> fetcherId 0 (kafka.server.R
> eplicaFetcherManager)
> [2012-10-26 08:04:13,335] INFO Replica Manager on Broker 3: Handling leader 
> and isr request LeaderAndIsrRequest(1,,1000,Map((test_1,1) -> 
> PartitionStateInfo({ "ISR":"2,3","leader":"2","leaderEpoch":"2" },3), 
> (test_1,0) -> PartitionStateInfo({ "ISR":"2,3","leader":"2","leaderEpoch":"2" 
> },3))) (kafka.server.ReplicaManager)
> [2012-10-26 08:04:13,335] INFO Replica Manager on Broker 3: Starting the 
> follower state transition to follow leader 2 for topic test_1 partition 1 
> (kafka.server.ReplicaManager)
> [2012-10-26 08:04:13,335] INFO Partition [test_1, 1] on broker 3: Current 
> leader epoch [2] is larger or equal to the requested leader epoch [2], 
> discard the become follower request (kafka.cluster.Partition)
> [2012-10-26 08:04:13,336] INFO Replica Manager on Broker 3: Starting the 
> follower state transition to follow leader 2 for topic test_1 partition 0 
> (kafka.server.ReplicaManager)
> [2012-10-26 08:04:13,336] INFO Partition [test_1, 0] on broker 3: Current 
> leader epoch [2] is larger or equal to the requested leader epoch [2], 
> discard the become follower request (kafka.cluster.Partition)
> [2012-10-26 08:04:13,588] ERROR [ReplicaFetcherThread-2-0-on-broker-3], Error 
> due to  (kafka.server.ReplicaFetcherThread)
> java.lang.IllegalArgumentException: Attempt to append an offset (429050) no 
> larger than the last offset appended (429050).
>         at kafka.log.OffsetIndex.append(OffsetIndex.scala:180)
>         at kafka.log.LogSegment.append(LogSegment.scala:56)
>         at kafka.log.Log.append(Log.scala:273)
>         at 
> kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:51)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:116)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:99)
>         at scala.collection.immutable.Map$Map2.foreach(Map.scala:127)
>         at 
> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:99)
>         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