[ 
https://issues.apache.org/jira/browse/KAFKA-13773?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17514245#comment-17514245
 ] 

Jun Rao commented on KAFKA-13773:
---------------------------------

[~Timelad] : Thanks for filing the jira. I ran the following tool on one the 
log segments. The log file seems corrupted.  

 
{code:java}
bin/kafka-dump-log.sh  --files 
kafka-2-retention/audit-trail-0/00000000000000000000.log
baseOffset: 30432 lastOffset: 30432 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false 
isControl: false position: 250051313 CreateTime: 1648460938666 size: 8216 
magic: 2 compresscodec: none crc: 2462031276 isvalid: true
baseOffset: 30433 lastOffset: 30433 count: 1 baseSequence: -1 lastSequence: -1 
producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false 
isControl: false position: 250059529 CreateTime: 1648460938942 size: 8219 
magic: 2 compresscodec: none crc: 563640287 isvalid: true
Found 5340 invalid bytes at the end of 00000000000000000000.log
{code}
 

If the file is used as it is to determine the max timestamp, it could lead to 
invalid timestamp. 

Normally, if the broker dies because of no disk space, on restarting, the 
broker will go through log recovery to check the validity of the data. However, 
from the log4j file, it doesn't seem there was log recovery. So, I am wondering 
if the broker crashed before that log segment was rolled and flushed, or after.

 

 

> Data loss after recovery from crash due to full hard disk
> ---------------------------------------------------------
>
>                 Key: KAFKA-13773
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13773
>             Project: Kafka
>          Issue Type: Bug
>          Components: log cleaner
>    Affects Versions: 3.1.0, 2.8.1
>            Reporter: Tm Alkemade
>            Priority: Major
>         Attachments: kafka-.zip, kafka-logfiles.zip
>
>
> While doing some testing of Kafka on Kubernetes, the data disk for kafka 
> filled up, which led to all 3 nodes crashing. I increased the disk size for 
> all three nodes and started up kafka again (one by one, waiting for the 
> previous node to become available before starting the next one). After a 
> little while two out of three nodes had no data anymore.
> According to the logs, the log cleaner kicked in and decided that the latest 
> timestamp on those partitions was '0' (i.e. 1970-01-01), and that is older 
> than the 2 week limit specified on the topic.
>  
> {code:java}
> 2022-03-28 12:17:19,740 INFO [LocalLog partition=audit-trail-0, 
> dir=/var/lib/kafka/data-0/kafka-log1] Deleting segment files 
> LogSegment(baseOffset=0, size=249689733, lastModifiedTime=1648460888636, 
> largestRecordTimestamp=Some(0)) (kafka.log.LocalLog$) [kafka-scheduler-0]
> 2022-03-28 12:17:19,753 INFO Deleted log 
> /var/lib/kafka/data-0/kafka-log1/audit-trail-0/00000000000000000000.log.deleted.
>  (kafka.log.LogSegment) [kafka-scheduler-0]
> 2022-03-28 12:17:19,754 INFO Deleted offset index 
> /var/lib/kafka/data-0/kafka-log1/audit-trail-0/00000000000000000000.index.deleted.
>  (kafka.log.LogSegment) [kafka-scheduler-0]
> 2022-03-28 12:17:19,754 INFO Deleted time index 
> /var/lib/kafka/data-0/kafka-log1/audit-trail-0/00000000000000000000.timeindex.deleted.
>  (kafka.log.LogSegment) [kafka-scheduler-0]{code}
> Using kafka-dump-log.sh I was able to determine that the greatest timestamp 
> in that file (before deletion) was actually 1648460888636 ( 2022-03-28, 
> 09:48:08 UTC, which is today). However since this segment was the 
> 'latest/current' segment much of the file is empty. The code that determines 
> the last entry (TimeIndex.lastEntryFromIndexFile)  doesn't seem to know this 
> and just read the last position in the file, the file being mostly empty 
> causes it to read 0 for that position.
> The cleaner code seems to take this into account since 
> UnifiedLog.deleteOldSegments is never supposed to delete the current segment, 
> judging by the scaladoc, however in this case the check doesn't seem to do 
> its job. Perhaps the detected highWatermark is wrong?
> I've attached the logs and the zipped data directories (data files are over 
> 3Gb in size when unzipped)
>  
> I've encountered this problem with both kafka 2.8.1 and 3.1.0.
> I've also tried changing min.insync.replicas to 2: The issue still occurs.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to