You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jun Rao (JIRA)" <ji...@apache.org> on 2015/06/13 03:52:00 UTC

[jira] [Updated] (KAFKA-2164) ReplicaFetcherThread: suspicious log message on reset offset

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

Jun Rao updated KAFKA-2164:
---------------------------
       Resolution: Fixed
    Fix Version/s: 0.8.3
           Status: Resolved  (was: Patch Available)

Thanks for the patch. +1 and committed to trunk.

[~aozeritsky], somehow I can't add you to Kafka contributor's list. Not sure why. 

> ReplicaFetcherThread: suspicious log message on reset offset
> ------------------------------------------------------------
>
>                 Key: KAFKA-2164
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2164
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2.1
>            Reporter: Alexey Ozeritskiy
>             Fix For: 0.8.3
>
>         Attachments: KAFKA-2164.patch
>
>
> If log.logEndOffset < leaderStartOffset the follower resets its offset and prints the following:
> {code}
> [2015-03-25 11:11:08,975] WARN [ReplicaFetcherThread-0-21], Replica 30 for partition [topic,11] reset its fetch offset from 49322124 to current leader 21's start offset 49322124 (kafka.server.ReplicaFetcherThread)
> [2015-03-25 11:11:08,976] ERROR [ReplicaFetcherThread-0-21], Current offset 54369274 for partition [topic,11] out of range; reset offset to 49322124 (kafka.server.ReplicaFetcherThread)
> {code}
> I think the right message should be:
> {code}
> [2015-03-25 11:11:08,975] WARN [ReplicaFetcherThread-0-21], Replica 30 for partition [topic,11] reset its fetch offset from 54369274 to current leader 21's start offset 49322124 (kafka.server.ReplicaFetcherThread)
> [2015-03-25 11:11:08,976] ERROR [ReplicaFetcherThread-0-21], Current offset 54369274 for partition [topic,11] out of range; reset offset to 49322124 (kafka.server.ReplicaFetcherThread)
> {code}
> This occurs because ReplicaFetcherThread resets the offset and then print log message.
> Posible solution:
> {code}
> diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/
> index b31b432..181cbc1 100644
> --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
> +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
> @@ -111,9 +111,9 @@ class ReplicaFetcherThread(name:String,
>         * Roll out a new log at the follower with the start offset equal to the
>         */
>        val leaderStartOffset = simpleConsumer.earliestOrLatestOffset(topicAndPar
> -      replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderSt
>        warn("Replica %d for partition %s reset its fetch offset from %d to curre
>          .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.
> +      replicaMgr.logManager.truncateFullyAndStartAt(topicAndPartition, leaderSt
>        leaderStartOffset
>      }
>    }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)