[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16532366#comment-16532366 ]
Karsten Schnitter commented on KAFKA-7130: ------------------------------------------ The stack trace of the original system was lost. Fortunately for this issue, but unfortunately for us, the issue reoccured on a different cluster. I extracted the stack trace from there: {noformat} [2018-07-04 07:10:53,633] ERROR [ReplicaManager broker=4] Error processing fetch operation on partition ingestion-67, offset 1316626420 (kafka.server.ReplicaManager) org.apache.kafka.common.KafkaException: java.io.EOFException: Failed to read `log header` from file channel `sun.nio.ch.FileChannelImpl@44f0380f`. Expected to read 17 bytes, but reached end of file after reading 0 bytes. Started read from position 2147483638. at org.apache.kafka.common.record.RecordBatchIterator.makeNext(RecordBatchIterator.java:40) at org.apache.kafka.common.record.RecordBatchIterator.makeNext(RecordBatchIterator.java:24) at org.apache.kafka.common.utils.AbstractIterator.maybeComputeNext(AbstractIterator.java:79) at org.apache.kafka.common.utils.AbstractIterator.hasNext(AbstractIterator.java:45) at org.apache.kafka.common.record.FileRecords.searchForOffsetWithSize(FileRecords.java:287) at kafka.log.LogSegment.translateOffset(LogSegment.scala:190) at kafka.log.LogSegment.read(LogSegment.scala:242) at kafka.log.Log.$anonfun$read$2(Log.scala:1020) at kafka.log.Log.maybeHandleIOException(Log.scala:1678) at kafka.log.Log.read(Log.scala:976) at kafka.server.ReplicaManager.read$1(ReplicaManager.scala:920) at kafka.server.ReplicaManager.$anonfun$readFromLocalLog$6(ReplicaManager.scala:982) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:59) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:52) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at kafka.server.ReplicaManager.readFromLocalLog(ReplicaManager.scala:981) at kafka.server.ReplicaManager.readFromLog$1(ReplicaManager.scala:818) at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:823) at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:631) at kafka.server.KafkaApis.handle(KafkaApis.scala:105) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:69) at java.lang.Thread.run(Thread.java:748) Caused by: java.io.EOFException: Failed to read `log header` from file channel `sun.nio.ch.FileChannelImpl@44f0380f`. Expected to read 17 bytes, but reached end of file after reading 0 bytes. Started read from position 2147483638. at org.apache.kafka.common.utils.Utils.readFullyOrFail(Utils.java:806) at org.apache.kafka.common.record.FileLogInputStream.nextBatch(FileLogInputStream.java:66) at org.apache.kafka.common.record.FileLogInputStream.nextBatch(FileLogInputStream.java:40) at org.apache.kafka.common.record.RecordBatchIterator.makeNext(RecordBatchIterator.java:35) ... 21 more {noformat} > EOFException after rolling log segment > -------------------------------------- > > Key: KAFKA-7130 > URL: https://issues.apache.org/jira/browse/KAFKA-7130 > Project: Kafka > Issue Type: Bug > Components: replication > Affects Versions: 1.1.0 > Reporter: Karsten Schnitter > Priority: Major > > When rolling a log segment one of our Kafka cluster got an immediate read > error on the same partition. This lead to a flood of log messages containing > the corresponding stacktraces. Data was still appended to the partition but > consumers were unable to read from that partition. Reason for the exception > is unclear. > {noformat} > [2018-07-02 23:53:32,732] INFO [Log partition=ingestion-3, > dir=/var/vcap/store/kafka] Rolled new log segment at offset 971865991 in 1 > ms. (kafka.log.Log) > [2018-07-02 23:53:32,739] INFO [ProducerStateManager partition=ingestion-3] > Writing producer snapshot at offset 971865991 (kafka.log.ProducerStateManager) > [2018-07-02 23:53:32,739] INFO [Log partition=ingestion-3, > dir=/var/vcap/store/kafka] Rolled new log segment at offset 971865991 in 1 > ms. (kafka.log.Log) > [2018-07-02 23:53:32,750] ERROR [ReplicaManager broker=1] Error processing > fetch operation on partition ingestion-3, offset 971865977 > (kafka.server.ReplicaManager) > Caused by: java.io.EOFException: Failed to read `log header` from file > channel `sun.nio.ch.FileChannelImpl@2e0e8810`. Expected to read 17 bytes, but > reached end of file after reading 0 bytes. Started read from position > 2147483643. > {noformat} > We mitigated the issue by stopping the affected node and deleting the > corresponding directory. Once the partition was recreated for the replica (we > use replication-factor 2) the other replica experienced the same problem. We > mitigated likewise. > To us it is unclear, what caused this issue. Can you help us in finding the > root cause of this problem? > > -- This message was sent by Atlassian JIRA (v7.6.3#76005)