[jira] [Commented] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16049229#comment-16049229 ] Jun Rao commented on KAFKA-5431: [~crietz], thanks for the info. I guess the preallocated size is 100MB? It's not clear why .log has the preallocated size. Normally, we shrink the file size to its actual size after rolling. In fact, the size for other segments like 0001.log does look normal. It's also interesting that each segment has only 1 message in it. Did you set a really small log segment size? > LogCleaner stopped due to > org.apache.kafka.common.errors.CorruptRecordException > --- > > Key: KAFKA-5431 > URL: https://issues.apache.org/jira/browse/KAFKA-5431 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.10.2.1 >Reporter: Carsten Rietz >Priority: Critical > Labels: reliability > Fix For: 0.11.0.1 > > > Hey all, > i have a strange problem with our uat cluster of 3 kafka brokers. > the __consumer_offsets topic was replicated to two instances and our disks > ran full due to a wrong configuration of the log cleaner. We fixed the > configuration and updated from 0.10.1.1 to 0.10.2.1 . > Today i increased the replication of the __consumer_offsets topic to 3 and > triggered replication to the third cluster via kafka-reassign-partitions.sh. > That went well but i get many errors like > {code} > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,18] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,24] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > {code} > Which i think are due to the full disk event. > The log cleaner threads died on these wrong messages: > {code} > [2017-06-12 09:59:50,722] ERROR [kafka-log-cleaner-thread-0], Error due to > (kafka.log.LogCleaner) > org.apache.kafka.common.errors.CorruptRecordException: Record size is less > than the minimum record overhead (14) > [2017-06-12 09:59:50,722] INFO [kafka-log-cleaner-thread-0], Stopped > (kafka.log.LogCleaner) > {code} > Looking at the file is see that some are truncated and some are jsut empty: > $ ls -lsh 00594653.log > 0 -rw-r--r-- 1 user user 100M Jun 12 11:00 00594653.log > Sadly i do not have the logs any more from the disk full event itsself. > I have three questions: > * What is the best way to clean this up? Deleting the old log files and > restarting the brokers? > * Why did kafka not handle the disk full event well? Is this only affecting > the cleanup or may we also loose data? > * Is this maybe caused by the combination of upgrade and disk full? > And last but not least: Keep up the good work. Kafka is really performing > well while being easy to administer and has good documentation! -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16049156#comment-16049156 ] Carsten Rietz commented on KAFKA-5431: -- I still have no idea what exactly to do to trigger this behaviour. Maybe its some strange behaviour of our producers. I have been able to find one instance on our dev system with a low roll interval, a one broker cluster and some restarts. Here is the corrupt log file: {code} [user@host __consumer_offsets-12# ll -hs | head total 2.7M 0 -rw-r--r-- 1 jboss jboss0 Jun 14 13:31 .index 4.0K -rw-r--r-- 1 jboss jboss 100M Jun 14 12:30 .log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 .timeindex 0 -rw-r--r-- 1 jboss jboss0 Jun 14 13:31 0001.index 4.0K -rw-r--r-- 1 jboss jboss 122 Jun 14 12:30 0001.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 0001.timeindex 0 -rw-r--r-- 1 jboss jboss0 Jun 14 13:31 0002.index 4.0K -rw-r--r-- 1 jboss jboss 122 Jun 14 12:30 0002.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 0002.timeindex [user@host __consumer_offsets-10]# stat .log File: `.log' Size: 104857600 Blocks: 8 IO Block: 4096 regular file Device: fd00h/64768dInode: 4718956 Links: 1 Access: (0644/-rw-r--r--) Uid: ( 494/ jboss) Gid: ( 488/ jboss) Access: 2017-06-14 12:31:03.585079439 +0100 Modify: 2017-06-14 12:30:42.0 +0100 Change: 2017-06-14 12:30:48.520886205 +0100 {code} {code} [user@host __consumer_offsets-10]# kafka-run-class.sh kafka.tools.DumpLogSegments --files .log Dumping .log Starting offset: 0 offset: 0 position: 0 CreateTime: 1497439842157 isvalid: true payloadsize: 257 magic: 1 compresscodec: NONE crc: 1102674092 keysize: 26 Exception in thread "main" org.apache.kafka.common.errors.CorruptRecordException: Record size is smaller than minimum record overhead (14). {code} hexdumping the file also shows that its still a sparse file with trailing zero content {code} [user@host __consumer_offsets-10]# hexdump -C .log 00 00 00 00 00 00 00 00 00 00 01 31 41 b9 78 ac |...1A.x.| 0010 01 00 00 00 01 5c a6 5e af 6d 00 00 00 1a 00 02 |.\.^.m..| 0020 00 16 63 6f 6e 73 6f 6c 65 2d 63 6f 6e 73 75 6d |..console-consum| 0030 65 72 2d 35 39 37 37 33 00 00 01 01 00 01 00 08 |er-59773| 0040 63 6f 6e 73 75 6d 65 72 00 00 00 01 00 05 72 61 |consumer..ra| 0050 6e 67 65 00 2f 63 6f 6e 73 75 6d 65 72 2d 31 2d |nge./consumer-1-| 0060 39 39 36 33 62 31 36 36 2d 39 32 39 39 2d 34 34 |9963b166-9299-44| 0070 30 66 2d 38 64 39 37 2d 35 62 34 38 31 33 64 35 |0f-8d97-5b4813d5| 0080 63 39 38 34 00 00 00 01 00 2f 63 6f 6e 73 75 6d |c984./consum| 0090 65 72 2d 31 2d 39 39 36 33 62 31 36 36 2d 39 32 |er-1-9963b166-92| 00a0 39 39 2d 34 34 30 66 2d 38 64 39 37 2d 35 62 34 |99-440f-8d97-5b4| 00b0 38 31 33 64 35 63 39 38 34 00 0a 63 6f 6e 73 75 |813d5c984..consu| 00c0 6d 65 72 2d 31 00 0e 2f 31 30 2e 32 34 38 2e 34 |mer-1../10.248.4| 00d0 33 2e 32 33 33 00 04 93 e0 00 00 27 10 00 00 00 |3.233..'| 00e0 28 00 00 00 00 00 01 00 1c 74 63 2e 69 6e 2e 64 |(tc.in.d| 00f0 65 76 2e 73 74 61 74 69 73 74 69 63 73 2e 67 72 |ev.statistics.gr| 0100 61 70 68 69 74 00 00 00 00 00 00 00 30 00 00 00 |aphit...0...| 0110 00 00 01 00 1c 74 63 2e 69 6e 2e 64 65 76 2e 73 |.tc.in.dev.s| 0120 74 61 74 69 73 74 69 63 73 2e 67 72 61 70 68 69 |tatistics.graphi| 0130 74 00 00 00 01 00 00 00 00 00 00 00 00 00 00 00 |t...| 0140 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 || * 0640 {code} We are running kafka_2.12-0.10.2.1 on jdk1.8.0_121 on Red Hat Enterprise Linux Server release 6.9. Relevant parameters: {code} log.segment.bytes=10485760 log.retention.check.interval.ms=1000 log.roll.ms=1 log.preallocate=true log.retention.bytes=10485760 {code} And finally the log files. I removed all __consumer_offsets and restarted the kafka broker due to full disk before. {code} [2017-06-14 12:28:46,198] INFO Started Kafka Graphite metrics reporter with polling period 1 seconds (com.criteo.kafka.KafkaGraphiteMetricsReporter) ... [2017-06-14 12:28:51,170] INFO [Group Metadata Manager on Broker 1]: Loading offsets and group metadata from __consumer_offsets-12 (kafka.coordinator.GroupMetadataManager) [2017-06-14 12:28:51,189] INFO [Group Metadata Manager on Broker 1]: Finished loading offsets from __consumer_offsets-12 in 19 milliseconds. (kafka.coordinator.GroupMetadataManager) [2017-06-14 12:30:47,164] INFO Rolled new log segment for '__consumer_offset
[jira] [Commented] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16047871#comment-16047871 ] Carsten Rietz commented on KAFKA-5431: -- Thanks for the fast reponse. We did some more digging today and it seems related to log.preallocate=true. The log files which tip over the LogCleaner are not compacted even if they were rolled. Here is a example with 1706631 being faulty. {code} [user@host ~]$ ls -lsh data/__consumer_offsets-26/*.log 328K -rw-r--r-- 1 jboss jboss 328K Jun 13 09:29 01701717.log 332K -rw-r--r-- 1 jboss jboss 330K Jun 13 09:29 01704168.log 32K -rw-r--r-- 1 jboss jboss 100M Jun 13 09:29 01706631.log {code} In the kafka log we see the normal {code} [2017-06-13 09:29:09,345] INFO Rolled new log segment for '__consumer_offsets-26' in 1 ms. (kafka.log.Log) {code} As i understand the code this should not be possible :) We worked around for now by setting log.preallocate=false, deleting all __consumer_offsets on one broker and restarting it. Now eerything seems to run stable. I will try to find another occurrence on our test environment and check for corrupted records. > LogCleaner stopped due to > org.apache.kafka.common.errors.CorruptRecordException > --- > > Key: KAFKA-5431 > URL: https://issues.apache.org/jira/browse/KAFKA-5431 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.10.2.1 >Reporter: Carsten Rietz > Labels: reliability > > Hey all, > i have a strange problem with our uat cluster of 3 kafka brokers. > the __consumer_offsets topic was replicated to two instances and our disks > ran full due to a wrong configuration of the log cleaner. We fixed the > configuration and updated from 0.10.1.1 to 0.10.2.1 . > Today i increased the replication of the __consumer_offsets topic to 3 and > triggered replication to the third cluster via kafka-reassign-partitions.sh. > That went well but i get many errors like > {code} > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,18] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,24] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > {code} > Which i think are due to the full disk event. > The log cleaner threads died on these wrong messages: > {code} > [2017-06-12 09:59:50,722] ERROR [kafka-log-cleaner-thread-0], Error due to > (kafka.log.LogCleaner) > org.apache.kafka.common.errors.CorruptRecordException: Record size is less > than the minimum record overhead (14) > [2017-06-12 09:59:50,722] INFO [kafka-log-cleaner-thread-0], Stopped > (kafka.log.LogCleaner) > {code} > Looking at the file is see that some are truncated and some are jsut empty: > $ ls -lsh 00594653.log > 0 -rw-r--r-- 1 user user 100M Jun 12 11:00 00594653.log > Sadly i do not have the logs any more from the disk full event itsself. > I have three questions: > * What is the best way to clean this up? Deleting the old log files and > restarting the brokers? > * Why did kafka not handle the disk full event well? Is this only affecting > the cleanup or may we also loose data? > * Is this maybe caused by the combination of upgrade and disk full? > And last but not least: Keep up the good work. Kafka is really performing > well while being easy to administer and has good documentation! -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16047637#comment-16047637 ] huxihx commented on KAFKA-5431: --- Could you run command below to see whether there exists a corrupt record for `__consumer_offsets` topic? {noformat} bin/kafka-simple-consumer-shell.sh --topic __consumer_offsets --partition 18 --broker-list *** --formatter "kafka.coordinator.GroupMetadataManager\$OffsetsMessageFormatter" bin/kafka-simple-consumer-shell.sh --topic __consumer_offsets --partition 24 --broker-list *** --formatter "kafka.coordinator.GroupMetadataManager\$OffsetsMessageFormatter" {noformat} > LogCleaner stopped due to > org.apache.kafka.common.errors.CorruptRecordException > --- > > Key: KAFKA-5431 > URL: https://issues.apache.org/jira/browse/KAFKA-5431 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.10.2.1 >Reporter: Carsten Rietz > Labels: reliability > > Hey all, > i have a strange problem with our uat cluster of 3 kafka brokers. > the __consumer_offsets topic was replicated to two instances and our disks > ran full due to a wrong configuration of the log cleaner. We fixed the > configuration and updated from 0.10.1.1 to 0.10.2.1 . > Today i increased the replication of the __consumer_offsets topic to 3 and > triggered replication to the third cluster via kafka-reassign-partitions.sh. > That went well but i get many errors like > {code} > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,18] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,24] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > {code} > Which i think are due to the full disk event. > The log cleaner threads died on these wrong messages: > {code} > [2017-06-12 09:59:50,722] ERROR [kafka-log-cleaner-thread-0], Error due to > (kafka.log.LogCleaner) > org.apache.kafka.common.errors.CorruptRecordException: Record size is less > than the minimum record overhead (14) > [2017-06-12 09:59:50,722] INFO [kafka-log-cleaner-thread-0], Stopped > (kafka.log.LogCleaner) > {code} > Looking at the file is see that some are truncated and some are jsut empty: > $ ls -lsh 00594653.log > 0 -rw-r--r-- 1 user user 100M Jun 12 11:00 00594653.log > Sadly i do not have the logs any more from the disk full event itsself. > I have three questions: > * What is the best way to clean this up? Deleting the old log files and > restarting the brokers? > * Why did kafka not handle the disk full event well? Is this only affecting > the cleanup or may we also loose data? > * Is this maybe caused by the combination of upgrade and disk full? > And last but not least: Keep up the good work. Kafka is really performing > well while being easy to administer and has good documentation! -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16046650#comment-16046650 ] Jun Rao commented on KAFKA-5431: Interesting. First, in general, if there is an IOException during writing to the log, the broker will shut down immediately. Second, if the cleaner hits an IOException, currently, we just abort the current cleaning job. The existing log should be intact. The "invalid message" from the fetch follower seems a bit weird. That suggests that the log segment at offset 0 is corrupted somehow. Could you use the DumpLogSegment tool (https://cwiki.apache.org/confluence/display/KAFKA/System+Tools#SystemTools-DumpLogSegment) on that segment in the leader to see if there is any log corruption? > LogCleaner stopped due to > org.apache.kafka.common.errors.CorruptRecordException > --- > > Key: KAFKA-5431 > URL: https://issues.apache.org/jira/browse/KAFKA-5431 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.10.2.1 >Reporter: Carsten Rietz > Labels: reliability > > Hey all, > i have a strange problem with our uat cluster of 3 kafka brokers. > the __consumer_offsets topic was replicated to two instances and our disks > ran full due to a wrong configuration of the log cleaner. We fixed the > configuration and updated from 0.10.1.1 to 0.10.2.1 . > Today i increased the replication of the __consumer_offsets topic to 3 and > triggered replication to the third cluster via kafka-reassign-partitions.sh. > That went well but i get many errors like > {code} > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,18] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,24] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > {code} > Which i think are due to the full disk event. > The log cleaner threads died on these wrong messages: > {code} > [2017-06-12 09:59:50,722] ERROR [kafka-log-cleaner-thread-0], Error due to > (kafka.log.LogCleaner) > org.apache.kafka.common.errors.CorruptRecordException: Record size is less > than the minimum record overhead (14) > [2017-06-12 09:59:50,722] INFO [kafka-log-cleaner-thread-0], Stopped > (kafka.log.LogCleaner) > {code} > Looking at the file is see that some are truncated and some are jsut empty: > $ ls -lsh 00594653.log > 0 -rw-r--r-- 1 user user 100M Jun 12 11:00 00594653.log > Sadly i do not have the logs any more from the disk full event itsself. > I have three questions: > * What is the best way to clean this up? Deleting the old log files and > restarting the brokers? > * Why did kafka not handle the disk full event well? Is this only affecting > the cleanup or may we also loose data? > * Is this maybe caused by the combination of upgrade and disk full? > And last but not least: Keep up the good work. Kafka is really performing > well while being easy to administer and has good documentation! -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16046480#comment-16046480 ] Ismael Juma commented on KAFKA-5431: `LogSegment.recover` discards invalid bytes from the end of the segment. Not sure why that didn't happen here. cc [~junrao] > LogCleaner stopped due to > org.apache.kafka.common.errors.CorruptRecordException > --- > > Key: KAFKA-5431 > URL: https://issues.apache.org/jira/browse/KAFKA-5431 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.10.2.1 >Reporter: Carsten Rietz > Labels: reliability > > Hey all, > i have a strange problem with our uat cluster of 3 kafka brokers. > the __consumer_offsets topic was replicated to two instances and our disks > ran full due to a wrong configuration of the log cleaner. We fixed the > configuration and updated from 0.10.1.1 to 0.10.2.1 . > Today i increased the replication of the __consumer_offsets topic to 3 and > triggered replication to the third cluster via kafka-reassign-partitions.sh. > That went well but i get many errors like > {code} > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,18] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > [2017-06-12 09:59:50,342] ERROR Found invalid messages during fetch for > partition [__consumer_offsets,24] offset 0 error Record size is less than the > minimum record overhead (14) (kafka.server.ReplicaFetcherThread) > {code} > Which i think are due to the full disk event. > The log cleaner threads died on these wrong messages: > {code} > [2017-06-12 09:59:50,722] ERROR [kafka-log-cleaner-thread-0], Error due to > (kafka.log.LogCleaner) > org.apache.kafka.common.errors.CorruptRecordException: Record size is less > than the minimum record overhead (14) > [2017-06-12 09:59:50,722] INFO [kafka-log-cleaner-thread-0], Stopped > (kafka.log.LogCleaner) > {code} > Looking at the file is see that some are truncated and some are jsut empty: > $ ls -lsh 00594653.log > 0 -rw-r--r-- 1 user user 100M Jun 12 11:00 00594653.log > Sadly i do not have the logs any more from the disk full event itsself. > I have three questions: > * What is the best way to clean this up? Deleting the old log files and > restarting the brokers? > * Why did kafka not handle the disk full event well? Is this only affecting > the cleanup or may we also loose data? > * Is this maybe caused by the combination of upgrade and disk full? > And last but not least: Keep up the good work. Kafka is really performing > well while being easy to administer and has good documentation! -- This message was sent by Atlassian JIRA (v6.4.14#64029)