[ https://issues.apache.org/jira/browse/KAFKA-5634?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16100252#comment-16100252 ]
Jason Gustafson commented on KAFKA-5634: ---------------------------------------- I've been looking at this a bit more. I think the only way this can happen is if the offsets returned by the leader are inconsistent. In this case, it seems like the high watermark returned in the fetch must have been behind the log start offset. The relevant snippet in the replica fetch response handler is here: {code} val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark) val leaderLogStartOffset = partitionData.logStartOffset // for the follower replica, we do not need to keep // its segment base offset the physical position, // these values will be computed upon making the leader replica.highWatermark = new LogOffsetMetadata(followerHighWatermark) replica.maybeIncrementLogStartOffset(leaderLogStartOffset) {code} The fetched offset was 1459250, which means the follower's LEO couldn't have been lower than that. So the follower high watermark must have been taken from the fetch response. Then the call to {{maybeIncrementLogStartOffset}} fails because the leader's log start offset is ahead of the high watermark which was just set. > Replica fetcher thread crashes due to OffsetOutOfRangeException > --------------------------------------------------------------- > > Key: KAFKA-5634 > URL: https://issues.apache.org/jira/browse/KAFKA-5634 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.11.0.0 > Reporter: Jason Gustafson > Assignee: Jason Gustafson > Priority: Critical > Labels: regression, reliability > Fix For: 0.11.0.1 > > > We have seen the following exception recently: > {code} > kafka.common.KafkaException: error processing data for partition [foo,0] > offset 1459250 > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:203) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:174) > at scala.Option.foreach(Option.scala:257) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:174) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:171) > at > scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply$mcV$sp(AbstractFetcherThread.scala:171) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:171) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:171) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:213) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:169) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) > Caused by: org.apache.kafka.common.errors.OffsetOutOfRangeException: The > specified offset 1459250 is higher than the high watermark 1459032 of the > partition foo-0 > {code} > The error check was added in the patch for KIP-107: > https://github.com/apache/kafka/commit/8b05ad406d4cba6a75d1683b6d8699c3ab28f9d6. > After investigation, we found that it is possible for the log start offset > on the leader to get ahead of the high watermark on the follower after > segment deletion. The check therefore seems incorrect. The impact of this bug > is that the fetcher thread crashes on the follower and the broker must be > restarted. -- This message was sent by Atlassian JIRA (v6.4.14#64029)