[
https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16533327#comment-16533327
]
Karsten Schnitter edited comment on KAFKA-7130 at 7/5/18 7:10 AM:
------------------------------------------------------------------
I saved the two segments for further investigation. As this is a production
system the segments are deleted after approx. 2 days. Fortunately the affected
segments are still there, not only in the backup.
The output of {{ls -l}} shows the following with the affected segment on top
and the newly created on second position.
{noformat}
-rw-r--r-- 1 vcap vcap 6656 Jul 4 08:41 00000000001307269075.index
-rw-r--r-- 1 vcap vcap 872918033 Jul 4 08:41 00000000001307269075.log
-rw-r--r-- 1 vcap vcap 9984 Jul 4 08:41 00000000001307269075.timeindex
-rw-r--r-- 1 vcap vcap 8832 Jul 4 08:41 00000000001309227785.index
-rw-r--r-- 1 vcap vcap 1157193996 Jul 4 08:41 00000000001309227785.log
-rw-r--r-- 1 vcap vcap 13248 Jul 4 08:41 00000000001309227785.timeindex
-rw-r--r-- 1 vcap vcap 16384 Jul 4 08:42 00000000001311959479.index
-rw-r--r-- 1 vcap vcap 2146435305 Jul 4 08:42 00000000001311959479.log
-rw-r--r-- 1 vcap vcap 24576 Jul 4 08:42 00000000001311959479.timeindex
{noformat}
I ran {{bin/kafka-run-class.sh kafka.tools.DumpLogSegments}} with the following
command:
{code:sh}
$ kafka-run-class.sh kafka.tools.DumpLogSegments --files
00000000001311940075.index,00000000001311940075.log,00000000001311940075.timeindex,00000000001316626421.index,00000000001316626421.log,00000000001316626421.timeindex
| gzip > dump.gz
{code}
When running the command it only processed the first index and log file, where
it crashed with the following stacktrace:
{noformat}
Exception in thread "main" org.apache.kafka.common.KafkaException:
java.io.EOFException: Failed to read `log header` from file channel
`sun.nio.ch.FileChannelImpl@15bb6bea`. 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
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:39)
at scala.collection.Iterator.foreach(Iterator.scala:929)
at scala.collection.Iterator.foreach$(Iterator.scala:929)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1417)
at scala.collection.IterableLike.foreach(IterableLike.scala:71)
at scala.collection.IterableLike.foreach$(IterableLike.scala:70)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at kafka.tools.DumpLogSegments$.dumpLog(DumpLogSegments.scala:381)
at
kafka.tools.DumpLogSegments$.$anonfun$main$1(DumpLogSegments.scala:112)
at
kafka.tools.DumpLogSegments$.$anonfun$main$1$adapted(DumpLogSegments.scala:104)
at
scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:32)
at
scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:29)
at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:191)
at kafka.tools.DumpLogSegments$.main(DumpLogSegments.scala:104)
at kafka.tools.DumpLogSegments.main(DumpLogSegments.scala)
Caused by: java.io.EOFException: Failed to read `log header` from file channel
`sun.nio.ch.FileChannelImpl@15bb6bea`. 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)
... 18 more
{noformat}
This seems to be the same problem. I need to run the command again separately
for the files, as the result is too large to publish here (76MB).
was (Author: kschnitter):
I saved the two segments for further investigation. As this is a production
system the segments are deleted after approx. 2 days. Fortunately the affected
segments are still there, not only in the backup.
The output of {{ls -l}} shows the following with the affected segment on top
and the newly created on second position.
{noformat}
-rw-r--r-- 1 vcap vcap 6656 Jul 4 08:41 00000000001307269075.index
-rw-r--r-- 1 vcap vcap 872918033 Jul 4 08:41 00000000001307269075.log
-rw-r--r-- 1 vcap vcap 9984 Jul 4 08:41 00000000001307269075.timeindex
-rw-r--r-- 1 vcap vcap 8832 Jul 4 08:41 00000000001309227785.index
-rw-r--r-- 1 vcap vcap 1157193996 Jul 4 08:41 00000000001309227785.log
-rw-r--r-- 1 vcap vcap 13248 Jul 4 08:41 00000000001309227785.timeindex
-rw-r--r-- 1 vcap vcap 16384 Jul 4 08:42 00000000001311959479.index
-rw-r--r-- 1 vcap vcap 2146435305 Jul 4 08:42 00000000001311959479.log
-rw-r--r-- 1 vcap vcap 24576 Jul 4 08:42 00000000001311959479.timeindex
{noformat}
I ran {{bin/kafka-run-class.sh kafka.tools.DumpLogSegments}} with the following
command:
{code:sh}
$ kafka-run-class.sh kafka.tools.DumpLogSegments --files
00000000001311940075.index,00000000001311940075.log,00000000001311940075.timeindex,00000000001316626421.index,00000000001316626421.log,00000000001316626421.timeindex
| gzip > dump.gz
{code}
You can find the file attached to this issue. When running the command it only
processed the first index and log file, where it crashed with the following
stacktrace:
{noformat}
Exception in thread "main" org.apache.kafka.common.KafkaException:
java.io.EOFException: Failed to read `log header` from file channel
`sun.nio.ch.FileChannelImpl@15bb6bea`. 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
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:39)
at scala.collection.Iterator.foreach(Iterator.scala:929)
at scala.collection.Iterator.foreach$(Iterator.scala:929)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1417)
at scala.collection.IterableLike.foreach(IterableLike.scala:71)
at scala.collection.IterableLike.foreach$(IterableLike.scala:70)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at kafka.tools.DumpLogSegments$.dumpLog(DumpLogSegments.scala:381)
at
kafka.tools.DumpLogSegments$.$anonfun$main$1(DumpLogSegments.scala:112)
at
kafka.tools.DumpLogSegments$.$anonfun$main$1$adapted(DumpLogSegments.scala:104)
at
scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:32)
at
scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:29)
at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:191)
at kafka.tools.DumpLogSegments$.main(DumpLogSegments.scala:104)
at kafka.tools.DumpLogSegments.main(DumpLogSegments.scala)
Caused by: java.io.EOFException: Failed to read `log header` from file channel
`sun.nio.ch.FileChannelImpl@15bb6bea`. 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)
... 18 more
{noformat}
This seems to be the same problem.
> 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)