[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16535439#comment-16535439 ] Jason Gustafson commented on KAFKA-7130: Yeah, this looks like it was fixed by KAFKA-6292. I will pick back into 1.1 and 1.0 and close this. If there is another RC for 1.1.1, it can get in. Decreasing the segment size a little bit seems like a reasonable workaround. > 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 > Attachments: dump-001311940075.index.bz2, > dump-001311940075.log.bz2 > > > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16535102#comment-16535102 ] Dhruvil Shah commented on KAFKA-7130: - Yes, you could use a smaller segment size. I would recommend using the default of 1GB unless there is a good reason to change it. I think we should include the fix in 1.1.1. [~hachikuji], what do you think? > 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 > Attachments: dump-001311940075.index.bz2, > dump-001311940075.log.bz2 > > > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16534489#comment-16534489 ] Karsten Schnitter commented on KAFKA-7130: -- As mentioned above, we use {{segment.bytes=2147483647}}. Additionally we use lz4 compression on the topic. I agree, that this issue looks very similar to KAFKA-6292. > 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 > Attachments: dump-001311940075.index.bz2, > dump-001311940075.log.bz2 > > > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16533997#comment-16533997 ] Dhruvil Shah commented on KAFKA-7130: - Thanks for the information, [~kschnitter]. There was an issue with potential overflow in the logic to read messages from a segment which could cause us to fall outside the maximum allowable segment size. [KAFKA-6292|https://issues.apache.org/jira/browse/KAFKA-6292] fixed this issue, and I think what you are running into is very similar to that. The interesting part though is how the segment got to be this big in the first place. Is either of `log.segment.bytes` or `segment.bytes` set to a non-default value for the broker / topic? Is compaction enabled for this topic? > 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 > Attachments: dump-001311940075.index.bz2, > dump-001311940075.log.bz2 > > > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16533432#comment-16533432 ] Karsten Schnitter commented on KAFKA-7130: -- I think the most interesting parts may be the last lines of the dumps: {code:title=dump-001311940075.log} baseOffset: 1316626416 lastOffset: 1316626416 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 51 isTransactional: false position: 2147481625 CreateTime: 1530618550432 isvalid: true size: 517 magic: 2 compresscodec: LZ4 crc: 103954393 baseOffset: 1316626417 lastOffset: 1316626419 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 51 isTransactional: false position: 2147482142 CreateTime: 1530618550435 isvalid: true size: 896 magic: 2 compresscodec: LZ4 crc: 553751416 baseOffset: 1316626420 lastOffset: 1316626420 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 51 isTransactional: false position: 2147483038 CreateTime: 1530618550436 isvalid: true size: 600 magic: 2 compresscodec: LZ4 crc: 1152385286 {code} {code:title=dump-001311940075.index} offset: 1316626399 position: 2147474054 offset: 1316626407 position: 2147478722 offset: 1316626420 position: 2147483038 {code} Find both files attached to this issue. > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16533370#comment-16533370 ] Karsten Schnitter commented on KAFKA-7130: -- We run the topics showing this issue with {{segment.bytes=2147483647}} (Integer.MAX_VALUE - 1), which is the largest value Kafka allows. Can this be an issue? > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16533327#comment-16533327 ] Karsten Schnitter commented on KAFKA-7130: -- 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 001307269075.index -rw-r--r-- 1 vcap vcap 872918033 Jul 4 08:41 001307269075.log -rw-r--r-- 1 vcap vcap 9984 Jul 4 08:41 001307269075.timeindex -rw-r--r-- 1 vcap vcap 8832 Jul 4 08:41 001309227785.index -rw-r--r-- 1 vcap vcap 1157193996 Jul 4 08:41 001309227785.log -rw-r--r-- 1 vcap vcap 13248 Jul 4 08:41 001309227785.timeindex -rw-r--r-- 1 vcap vcap 16384 Jul 4 08:42 001311959479.index -rw-r--r-- 1 vcap vcap 2146435305 Jul 4 08:42 001311959479.log -rw-r--r-- 1 vcap vcap 24576 Jul 4 08:42 001311959479.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 001311940075.index,001311940075.log,001311940075.timeindex,001316626421.index,001316626421.log,001316626421.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.
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16533035#comment-16533035 ] Dhruvil Shah commented on KAFKA-7130: - [~kschnitter] it would be useful if you could save the affected partition (or at least the segment) while we investigate this. As a first step, could you please share information about all segments for this partition, including the size of each of them? `ls -l` output of the directory should be sufficient. Would you be willing to share metadata dump of the affected segment? This will tell us what the segment state is like. You could use `bin/kafka-run-class.sh kafka.tools.DumpLogSegments`. > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16532391#comment-16532391 ] Karsten Schnitter commented on KAFKA-7130: -- I saved the faulty segments from the last incident. But they contain sensitive data, that I cannot share outside my company. I tried to read the last message of the finished segment from the partition: {noformat} $ kafka-console-consumer.sh --bootstrap-server $KAFKA_SERVERS --topic ingestion --partition 67 --offset 1316626420 --max-messages 1 [2018-07-04 07:33:11,604] WARN [Consumer clientId=consumer-1, groupId=console-consumer-79022] Unknown error fetching data for topic-partition ingestion-67 (org.apache.kafka.clients.consumer.internals.Fetcher) ... ^C[2018-07-04 07:33:11,606] WARN [Consumer clientId=consumer-1, groupId=console-consumer-79022] Unknown error fetching data for topic-partition ingestion-67 (org.apache.kafka.clients.consumer.internals.Fetcher) [2018-07-04 07:33:11,612] WARN [Consumer clientId=consumer-1, groupId=console-consumer-79022] Unknown error fetching data for topic-partition ingestion-67 (org.apache.kafka.clients.consumer.internals.Fetcher) Processed a total of 0 messages {noformat} The error message is repeated until I stop the command. It is the same for other messages in the segment. Consuming the first message of the new segment works fine: {noformat} $ kafka-console-consumer.sh --bootstrap-server $KAFKA_SERVERS --topic ingestion --partition 67 --offset 1316626421 --max-messages 1 Processed a total of 1 messages {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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16532029#comment-16532029 ] Jason Gustafson commented on KAFKA-7130: Also it would be helpful if you include the full stack trace. > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16532024#comment-16532024 ] Jason Gustafson commented on KAFKA-7130: I agree the position suggests an overflow scenario. I don't suppose you saved any of the segments from the saved partition? > 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)
[jira] [Commented] (KAFKA-7130) EOFException after rolling log segment
[ https://issues.apache.org/jira/browse/KAFKA-7130?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16531213#comment-16531213 ] Karsten Schnitter commented on KAFKA-7130: -- I am suspicious about the read position from the EOFException: {{Started read from position 2147483643}}. This is very close to Integer.MAX_VALUE and has no connection to the partition offset of 971865991. > 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)