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

Jay Kreps updated KAFKA-588:
----------------------------

    Attachment: KAFKA-588.patch

I can't reproduce this issue, but this patch expands the tests and adds a 
little more logging. Specifically:
1. Expanded OffsetIndex to check we can still append after each truncate
2. Expanded LogSegmentTest to sequentially append 2 messages, truncate one of 
them off, and repeat this 30 times.
3. Removed obsolete references to "logical offset" (we changed terminology--now 
offsets are always logical and positions are always physical so this makes no 
sense.

Here is my reasoning: this problem doesn't occur every time so it must be 
triggered by something non-deterministic. The possibilities I can think of:
1. The OffsetIndex is not properly synchronized
2. The truncate call is not properly setting the lastOffset and so the error is 
spurious.
3. The truncate call has an off-by-one error.
4. We are shutting down uncleanly and the recovery process is somehow not 
kicking in or not working correctly.

Here is how I looked into these:
1. It does not look like a synchronization problem since both truncateTo and 
append are synchronized on the same lock.
2. I suspected a problem in setting the lastOffset, but actually I am handling 
this case--the check already special cases an empty index. So this is not it.
3. This does not appear to be the case, truncateTo(X) results in an index in 
which the last entry is <= X-1 which is what we want, we have  a number of 
tests on this.
4. I don't see this, we are going through the same append path in recovery we 
go through normally.
                
> 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
>         Attachments: KAFKA-588.patch
>
>
> [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