Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-588

Index truncation doesn't seem to remove the last entry properly

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Blocker
    • Resolution: Fixed
    • 0.8.0
    • None
    • core

    Description

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

      Attachments

        1. KAFKA-588-v2.patch
          9 kB
          Jay Kreps
        2. KAFKA-588.patch
          9 kB
          Jay Kreps

        Activity

          People

            jkreps Jay Kreps
            junrao Jun Rao
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: