Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 27D75200CA7 for ; Wed, 14 Jun 2017 15:12:09 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 26352160BC0; Wed, 14 Jun 2017 13:12:09 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id EAC7D160BDB for ; Wed, 14 Jun 2017 15:12:07 +0200 (CEST) Received: (qmail 53357 invoked by uid 500); 14 Jun 2017 13:12:06 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 53345 invoked by uid 99); 14 Jun 2017 13:12:06 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 14 Jun 2017 13:12:06 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 1F917C047B for ; Wed, 14 Jun 2017 13:12:06 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -98.952 X-Spam-Level: X-Spam-Status: No, score=-98.952 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LOTSOFHASH=0.25, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id v_1u42byaesf for ; Wed, 14 Jun 2017 13:12:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id A11EE5F485 for ; Wed, 14 Jun 2017 13:12:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id CBC12E0A75 for ; Wed, 14 Jun 2017 13:12:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 26BD821E0D for ; Wed, 14 Jun 2017 13:12:00 +0000 (UTC) Date: Wed, 14 Jun 2017 13:12:00 +0000 (UTC) From: "Carsten Rietz (JIRA)" To: dev@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (KAFKA-5431) LogCleaner stopped due to org.apache.kafka.common.errors.CorruptRecordException MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 14 Jun 2017 13:12:09 -0000 [ https://issues.apache.org/jira/browse/KAFKA-5431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16049156#comment-16049156 ] Carsten Rietz edited comment on KAFKA-5431 at 6/14/17 1:11 PM: --------------------------------------------------------------- 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 jboss 0 Jun 14 13:31 00000000000000000000.index 4.0K -rw-r--r-- 1 jboss jboss 100M Jun 14 12:30 00000000000000000000.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 00000000000000000000.timeindex 0 -rw-r--r-- 1 jboss jboss 0 Jun 14 13:31 00000000000000000001.index 4.0K -rw-r--r-- 1 jboss jboss 122 Jun 14 12:30 00000000000000000001.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 00000000000000000001.timeindex 0 -rw-r--r-- 1 jboss jboss 0 Jun 14 13:31 00000000000000000002.index 4.0K -rw-r--r-- 1 jboss jboss 122 Jun 14 12:30 00000000000000000002.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 00000000000000000002.timeindex [user@host __consumer_offsets-10]# stat 00000000000000000000.log File: `00000000000000000000.log' Size: 104857600 Blocks: 8 IO Block: 4096 regular file Device: fd00h/64768d Inode: 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.000000000 +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 00000000000000000000.log Dumping 00000000000000000000.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 00000000000000000000.log 00000000 00 00 00 00 00 00 00 00 00 00 01 31 41 b9 78 ac |...........1A.x.| 00000010 01 00 00 00 01 5c a6 5e af 6d 00 00 00 1a 00 02 |.....\.^.m......| 00000020 00 16 63 6f 6e 73 6f 6c 65 2d 63 6f 6e 73 75 6d |..console-consum| 00000030 65 72 2d 35 39 37 37 33 00 00 01 01 00 01 00 08 |er-59773........| 00000040 63 6f 6e 73 75 6d 65 72 00 00 00 01 00 05 72 61 |consumer......ra| 00000050 6e 67 65 00 2f 63 6f 6e 73 75 6d 65 72 2d 31 2d |nge./consumer-1-| 00000060 39 39 36 33 62 31 36 36 2d 39 32 39 39 2d 34 34 |9963b166-9299-44| 00000070 30 66 2d 38 64 39 37 2d 35 62 34 38 31 33 64 35 |0f-8d97-5b4813d5| 00000080 63 39 38 34 00 00 00 01 00 2f 63 6f 6e 73 75 6d |c984...../consum| 00000090 65 72 2d 31 2d 39 39 36 33 62 31 36 36 2d 39 32 |er-1-9963b166-92| 000000a0 39 39 2d 34 34 30 66 2d 38 64 39 37 2d 35 62 34 |99-440f-8d97-5b4| 000000b0 38 31 33 64 35 63 39 38 34 00 0a 63 6f 6e 73 75 |813d5c984..consu| 000000c0 6d 65 72 2d 31 00 0e 2f 31 30 2e 32 34 38 2e 34 |mer-1../10.248.4| 000000d0 33 2e 32 33 33 00 04 93 e0 00 00 27 10 00 00 00 |3.233......'....| 000000e0 28 00 00 00 00 00 01 00 1c 74 63 2e 69 6e 2e 64 |(........tc.in.d| 000000f0 65 76 2e 73 74 61 74 69 73 74 69 63 73 2e 67 72 |ev.statistics.gr| 00000100 61 70 68 69 74 00 00 00 00 00 00 00 30 00 00 00 |aphit.......0...| 00000110 00 00 01 00 1c 74 63 2e 69 6e 2e 64 65 76 2e 73 |.....tc.in.dev.s| 00000120 74 61 74 69 73 74 69 63 73 2e 67 72 61 70 68 69 |tatistics.graphi| 00000130 74 00 00 00 01 00 00 00 00 00 00 00 00 00 00 00 |t...............| 00000140 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 |................| * 06400000 {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=10000 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_offsets-12' in 1 ms. (kafka.log.Log) [2017-06-14 12:30:48,431] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-12. (kafka.log.LogCleaner) [2017-06-14 12:30:48,432] INFO Cleaner 0: Building offset map for __consumer_offsets-12... (kafka.log.LogCleaner) [2017-06-14 12:30:48,470] INFO Cleaner 0: Building offset map for log __consumer_offsets-12 for 1 segments in offset range [0, 1). (kafka.log.LogCleaner) [2017-06-14 12:30:48,499] INFO Cleaner 0: Offset map for log __consumer_offsets-12 complete. (kafka.log.LogCleaner) [2017-06-14 12:30:48,503] INFO Cleaner 0: Cleaning log __consumer_offsets-12 (cleaning prior to Wed Jun 14 12:30:42 BST 2017, discarding tombstones prior to Thu Jan 01 01:00:00 GMT 1970)... (kafka.log.LogCleaner) [2017-06-14 12:30:48,507] INFO Cleaner 0: Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Wed Jun 14 12:30:42 BST 2017) into 0, retaining deletes. (kafka.log.LogCleaner) [2017-06-14 12:30:48,521] INFO Cleaner 0: Swapping in cleaned segment 0 for segment(s) 0 in log __consumer_offsets-12. (kafka.log.LogCleaner) Log cleaner thread 0 cleaned log __consumer_offsets-12 (dirty section = [0, 0]) [2017-06-14 12:30:52,161] INFO Rolled new log segment for '__consumer_offsets-12' in 1 ms. (kafka.log.Log) {code} Here the log file seems to have been rolled and not truncated. What tips over the logcleaner a bit later {code} [2017-06-14 12:31:03,564] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-12. (kafka.log.LogCleaner) [2017-06-14 12:31:03,564] INFO Cleaner 0: Building offset map for __consumer_offsets-12... (kafka.log.LogCleaner) [2017-06-14 12:31:03,583] INFO Cleaner 0: Building offset map for log __consumer_offsets-12 for 3 segments in offset range [1, 4). (kafka.log.LogCleaner) [2017-06-14 12:31:03,584] INFO Cleaner 0: Offset map for log __consumer_offsets-12 complete. (kafka.log.LogCleaner) [2017-06-14 12:31:03,584] INFO Cleaner 0: Cleaning log __consumer_offsets-12 (cleaning prior to Wed Jun 14 12:31:02 BST 2017, discarding tombstones prior to Tue Jun 13 12:30:42 BST 2017)... (kafka.log.LogCleaner) [2017-06-14 12:31:03,585] INFO Cleaner 0: Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Wed Jun 14 12:30:42 BST 2017) into 0, retaining deletes. (kafka.log.LogCleaner) [2017-06-14 12:31:03,589] 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) {code} edit: Its likely that i cannot have another look before monday. I hope someone has a brigth idea. Deactivating the preallocation works fine as a workaround. was (Author: crietz): 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 jboss 0 Jun 14 13:31 00000000000000000000.index 4.0K -rw-r--r-- 1 jboss jboss 100M Jun 14 12:30 00000000000000000000.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 00000000000000000000.timeindex 0 -rw-r--r-- 1 jboss jboss 0 Jun 14 13:31 00000000000000000001.index 4.0K -rw-r--r-- 1 jboss jboss 122 Jun 14 12:30 00000000000000000001.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 00000000000000000001.timeindex 0 -rw-r--r-- 1 jboss jboss 0 Jun 14 13:31 00000000000000000002.index 4.0K -rw-r--r-- 1 jboss jboss 122 Jun 14 12:30 00000000000000000002.log 4.0K -rw-r--r-- 1 jboss jboss 12 Jun 14 13:31 00000000000000000002.timeindex [user@host __consumer_offsets-10]# stat 00000000000000000000.log File: `00000000000000000000.log' Size: 104857600 Blocks: 8 IO Block: 4096 regular file Device: fd00h/64768d Inode: 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.000000000 +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 00000000000000000000.log Dumping 00000000000000000000.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 00000000000000000000.log 00000000 00 00 00 00 00 00 00 00 00 00 01 31 41 b9 78 ac |...........1A.x.| 00000010 01 00 00 00 01 5c a6 5e af 6d 00 00 00 1a 00 02 |.....\.^.m......| 00000020 00 16 63 6f 6e 73 6f 6c 65 2d 63 6f 6e 73 75 6d |..console-consum| 00000030 65 72 2d 35 39 37 37 33 00 00 01 01 00 01 00 08 |er-59773........| 00000040 63 6f 6e 73 75 6d 65 72 00 00 00 01 00 05 72 61 |consumer......ra| 00000050 6e 67 65 00 2f 63 6f 6e 73 75 6d 65 72 2d 31 2d |nge./consumer-1-| 00000060 39 39 36 33 62 31 36 36 2d 39 32 39 39 2d 34 34 |9963b166-9299-44| 00000070 30 66 2d 38 64 39 37 2d 35 62 34 38 31 33 64 35 |0f-8d97-5b4813d5| 00000080 63 39 38 34 00 00 00 01 00 2f 63 6f 6e 73 75 6d |c984...../consum| 00000090 65 72 2d 31 2d 39 39 36 33 62 31 36 36 2d 39 32 |er-1-9963b166-92| 000000a0 39 39 2d 34 34 30 66 2d 38 64 39 37 2d 35 62 34 |99-440f-8d97-5b4| 000000b0 38 31 33 64 35 63 39 38 34 00 0a 63 6f 6e 73 75 |813d5c984..consu| 000000c0 6d 65 72 2d 31 00 0e 2f 31 30 2e 32 34 38 2e 34 |mer-1../10.248.4| 000000d0 33 2e 32 33 33 00 04 93 e0 00 00 27 10 00 00 00 |3.233......'....| 000000e0 28 00 00 00 00 00 01 00 1c 74 63 2e 69 6e 2e 64 |(........tc.in.d| 000000f0 65 76 2e 73 74 61 74 69 73 74 69 63 73 2e 67 72 |ev.statistics.gr| 00000100 61 70 68 69 74 00 00 00 00 00 00 00 30 00 00 00 |aphit.......0...| 00000110 00 00 01 00 1c 74 63 2e 69 6e 2e 64 65 76 2e 73 |.....tc.in.dev.s| 00000120 74 61 74 69 73 74 69 63 73 2e 67 72 61 70 68 69 |tatistics.graphi| 00000130 74 00 00 00 01 00 00 00 00 00 00 00 00 00 00 00 |t...............| 00000140 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 |................| * 06400000 {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=10000 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_offsets-12' in 1 ms. (kafka.log.Log) [2017-06-14 12:30:48,431] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-12. (kafka.log.LogCleaner) [2017-06-14 12:30:48,432] INFO Cleaner 0: Building offset map for __consumer_offsets-12... (kafka.log.LogCleaner) [2017-06-14 12:30:48,470] INFO Cleaner 0: Building offset map for log __consumer_offsets-12 for 1 segments in offset range [0, 1). (kafka.log.LogCleaner) [2017-06-14 12:30:48,499] INFO Cleaner 0: Offset map for log __consumer_offsets-12 complete. (kafka.log.LogCleaner) [2017-06-14 12:30:48,503] INFO Cleaner 0: Cleaning log __consumer_offsets-12 (cleaning prior to Wed Jun 14 12:30:42 BST 2017, discarding tombstones prior to Thu Jan 01 01:00:00 GMT 1970)... (kafka.log.LogCleaner) [2017-06-14 12:30:48,507] INFO Cleaner 0: Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Wed Jun 14 12:30:42 BST 2017) into 0, retaining deletes. (kafka.log.LogCleaner) [2017-06-14 12:30:48,521] INFO Cleaner 0: Swapping in cleaned segment 0 for segment(s) 0 in log __consumer_offsets-12. (kafka.log.LogCleaner) Log cleaner thread 0 cleaned log __consumer_offsets-12 (dirty section = [0, 0]) [2017-06-14 12:30:52,161] INFO Rolled new log segment for '__consumer_offsets-12' in 1 ms. (kafka.log.Log) {code} Here the log file seems to have been rolled and not truncated. What tips over the logcleaner a bit later {code} [2017-06-14 12:31:03,564] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-12. (kafka.log.LogCleaner) [2017-06-14 12:31:03,564] INFO Cleaner 0: Building offset map for __consumer_offsets-12... (kafka.log.LogCleaner) [2017-06-14 12:31:03,583] INFO Cleaner 0: Building offset map for log __consumer_offsets-12 for 3 segments in offset range [1, 4). (kafka.log.LogCleaner) [2017-06-14 12:31:03,584] INFO Cleaner 0: Offset map for log __consumer_offsets-12 complete. (kafka.log.LogCleaner) [2017-06-14 12:31:03,584] INFO Cleaner 0: Cleaning log __consumer_offsets-12 (cleaning prior to Wed Jun 14 12:31:02 BST 2017, discarding tombstones prior to Tue Jun 13 12:30:42 BST 2017)... (kafka.log.LogCleaner) [2017-06-14 12:31:03,585] INFO Cleaner 0: Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Wed Jun 14 12:30:42 BST 2017) into 0, retaining deletes. (kafka.log.LogCleaner) [2017-06-14 12:31:03,589] 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) {code} > 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 00000000000000594653.log > 0 -rw-r--r-- 1 user user 100M Jun 12 11:00 00000000000000594653.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)