[ https://issues.apache.org/jira/browse/KAFKA-6832?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16463689#comment-16463689 ]
Ciprian Pascu commented on KAFKA-6832: -------------------------------------- Hi, For us, the problem is easily reproducible e.g by killing all brokers, at short time one after another. In this case, broker-0 was not able to follow broker-2 due to the below situation (decoded using kafka.tools.DumpLogSegments): broker-0: *- baseOffset: 10712 lastOffset: 10712* baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: 0 *partitionLeaderEpoch: 0* isTransactional: false position: 3476142 CreateTime: 1525424946243 isvalid: true size: 316 magic: 2 compresscodec: NONE crc: 1743233761 *- baseOffset: 10713 lastOffset: 10713* baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: 0 *partitionLeaderEpoch: 0* isTransactional: false position: 3476458 CreateTime: 1525424947245 isvalid: true size: 316 magic: 2 compresscodec: NONE crc: 634274826 *- baseOffset: 10714 lastOffset: 10714* baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: 0 *partitionLeaderEpoch: 0* isTransactional: false position: 3476774 CreateTime: 1525424948248 isvalid: true size: 316 magic: 2 compresscodec: NONE crc: 4225911256 broker-2: *- baseOffset: 10712 lastOffset: 10712* baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: 0 *partitionLeaderEpoch: 0* isTransactional: false position: 3476142 Create Time: 1525424946243 isvalid: true size: 316 magic: 2 compresscodec: NONE crc: 1743233761 *- baseOffset: 10713 lastOffset: 10720* baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: 0 *partitionLeaderEpoch: 1* isTransactional: false position: 3476458 CreateTime: 1525424957261 isvalid: true size: 2108 magic: 2 compresscodec: NONE crc: 3444932391 *- baseOffset: 10721 lastOffset: 10721* baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: 0 *partitionLeaderEpoch: 1* isTransactional: false position: 3478566 CreateTime: 1525424958263 isvalid: true size: 316 magic: 2 compresscodec: NONE crc: 1708198842 Will KAFKA-6361 address this issue as well? Because, as we see, here the logs have diverged by several batches; broker-0 should go 2 batches backwards, in order to start following again. Ciprian. > Wrong start position in the log file on the leader, on fetch request. > --------------------------------------------------------------------- > > Key: KAFKA-6832 > URL: https://issues.apache.org/jira/browse/KAFKA-6832 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 1.1.0 > Reporter: Ciprian Pascu > Priority: Major > > Hi, > We have an environment with 3 Kafka brokers; after hard reboot all brokers > (by hard rebooting the VMs on which they are located), we experience drop in > the ISR, for the topics that have replication factor greater than 1; it is > caused by the death of some of the replica threads with the following > exception: > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: > *kafka.common.KafkaException: Error processing data for partition > __consumer_offsets-39 offset 308060* > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$ > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$ > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > scala.Option.foreach(Option.scala:257) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(Abs > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(Abs > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply$mcV$sp(AbstractFetcherThrea > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.utils.CoreUtils$.inLock(CoreUtils.scala:217) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:167) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: *Caused by: > java.lang.IllegalArgumentException: Out of order offsets found in > List(308059, 308060)* > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.log.Log$$anonfun$append$2.apply(Log.scala:683) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.log.Log$$anonfun$append$2.apply(Log.scala:624) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.log.Log.maybeHandleIOException(Log.scala:1679) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.log.Log.append(Log.scala:624) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.log.Log.appendAsFollower(Log.scala:607) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:102) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:41) > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$ > Apr 27 08:46:24 hostname kafka-server-start.sh[11215]: ... 13 more > > The replica requests for offset *308060, but it gets a message set containing > (**308059, 308060), which makes the replica thread crash, due to the above > exception. The reason why the leader sends a message set with a smaller > offset than requested seems to be in the implementation of 'read' method from > 'LogSegment'; according to the comment, this method should '*Read a message > set from this segment beginning with the first offset >= startOffset', but > actually it is using 'translateOffset' method, which uses 'lookup' method > which, according to comment, 'Find the largest offset less than or equal to > the given targetOffset'; the code confirms this; so, it seems we have a > contradiction here. > > Ciprian. > -- This message was sent by Atlassian JIRA (v7.6.3#76005)