[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-05-08 Thread Luke Chen (Jira)


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

Luke Chen commented on KAFKA-13773:
---

PR: https://github.com/apache/kafka/pull/12136

> 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
>Affects Versions: 2.8.0, 3.1.0, 2.8.1
>Reporter: Tim Alkemade
>Assignee: Luke Chen
>Priority: Critical
> Attachments: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.zip, kafka-logfiles.zip, kafka-start-to-finish.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/.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/.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/.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.7#820007)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-31 Thread Jun Rao (Jira)


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

Jun Rao commented on KAFKA-13773:
-

[~Timelad] : Thanks for the log. This does seem to be a real issue. What 
happened is the following. The broker hit IOException in log loading during the 
initial startup.

 
{code:java}
2022-03-31 09:28:43,407 ERROR Error while writing to checkpoint file 
/var/lib/kafka/data-0/kafka-log0/__consumer_offsets-12/leader-epoch-checkpoint 
(kafka.server.LogDirFailureChannel) 
[log-recovery-/var/lib/kafka/data-0/kafka-log0]
java.io.IOException: No space left on device
    at java.base/java.io.FileOutputStream.writeBytes(Native Method)
    at java.base/java.io.FileOutputStream.write(FileOutputStream.java:354)
    at java.base/sun.nio.cs.StreamEncoder.writeBytes(StreamEncoder.java:233)
    at 
java.base/sun.nio.cs.StreamEncoder.implFlushBuffer(StreamEncoder.java:312)
    at java.base/sun.nio.cs.StreamEncoder.implFlush(StreamEncoder.java:316)
    at java.base/sun.nio.cs.StreamEncoder.flush(StreamEncoder.java:153)
    at java.base/java.io.OutputStreamWriter.flush(OutputStreamWriter.java:251)
    at java.base/java.io.BufferedWriter.flush(BufferedWriter.java:257)
    at 
org.apache.kafka.server.common.CheckpointFile.write(CheckpointFile.java:94)
    at 
kafka.server.checkpoints.CheckpointFileWithFailureHandler.write(CheckpointFileWithFailureHandler.scala:37)
    at 
kafka.server.checkpoints.LeaderEpochCheckpointFile.write(LeaderEpochCheckpointFile.scala:71)
    at 
kafka.server.epoch.LeaderEpochFileCache.flush(LeaderEpochFileCache.scala:291)
    at 
kafka.server.epoch.LeaderEpochFileCache.$anonfun$truncateFromEnd$1(LeaderEpochFileCache.scala:237)
    at 
kafka.server.epoch.LeaderEpochFileCache.truncateFromEnd(LeaderEpochFileCache.scala:234)
    at kafka.log.LogLoader$.$anonfun$load$12(LogLoader.scala:188)
    at kafka.log.LogLoader$.$anonfun$load$12$adapted(LogLoader.scala:188)
    at scala.Option.foreach(Option.scala:437)
    at kafka.log.LogLoader$.load(LogLoader.scala:188)
    at kafka.log.UnifiedLog$.apply(UnifiedLog.scala:1785)
    at kafka.log.LogManager.loadLog(LogManager.scala:282)
    at kafka.log.LogManager.$anonfun$loadLogs$13(LogManager.scala:368)
    at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
    at java.base/java.lang.Thread.run(Thread.java:829){code}
 

Normally, this will cause the broker to have a hard shutdown, but only after 
the ReplicaManager is started, which happens after log loading. Then the 
IOException is also propagated to KafkaServer, which causes it to exit 
normally. As part of the normal exit, a clean shutdown file will be written. 
This causes the next broker restart to skip log recovery.

 
{code:java}
2022-03-31 09:28:43,411 ERROR There was an error in one of the threads during 
logs loading: org.apache.kafka.common.errors.KafkaStorageException: Error while 
writing to checkpoint file 
/var/lib/kafka/data-0/kafka-log0/__consumer_offsets-12/leader-epoch-checkpoint 
(kafka.log.LogManager) [main]
2022-03-31 09:28:43,414 ERROR [KafkaServer id=0] Fatal error during KafkaServer 
startup. Prepare to shutdown (kafka.server.KafkaServer) [main]
2022-03-31 09:28:43,415 INFO [KafkaServer id=0] shutting down 
(kafka.server.KafkaServer) [main]
2022-03-31 09:28:43,418 INFO Shutting down. (kafka.log.LogManager) 
[main]2022-03-31 09:28:43,466 INFO Shutdown complete. (kafka.log.LogManager) 
[main]{code}

> 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: 2.8.0, 3.1.0, 2.8.1
>Reporter: Tm Alkemade
>Priority: Major
> Attachments: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.zip, kafka-logfiles.zip, kafka-start-to-finish.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 

[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-31 Thread Tm Alkemade (Jira)


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

Tm Alkemade commented on KAFKA-13773:
-

[~junrao] I've added a new zip with the log files from start to finish for the 
test including initial data load, restarts and final restart after resizing the 
disks.

> 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: 2.8.0, 3.1.0, 2.8.1
>Reporter: Tm Alkemade
>Priority: Major
> Attachments: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.zip, kafka-logfiles.zip, kafka-start-to-finish.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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-30 Thread Tm Alkemade (Jira)


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

Tm Alkemade commented on KAFKA-13773:
-

[~junrao] Yes, thats true, it was a new different test run, (it had the same 
behavior though). I'll try to get a consistent run with logs in the right order 
tomorrow, unfortunately getting all the logs is a bit tricky sometimes in 
Kubernetes when pods are restarting.

> 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: 2.8.0, 3.1.0, 2.8.1
>Reporter: Tm Alkemade
>Priority: Major
> Attachments: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-30 Thread Jun Rao (Jira)


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

Jun Rao commented on KAFKA-13773:
-

[~Timelad] : I checked kafka-0-2.8.0-before-fail.log 
kafka-1-2.8.0-before-fail.log in kafka-2.8.0-crash.zip. They both seem to have 
timestamp after 13:23:00,077, which is the time when the recovery is skipped. 

> 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: 2.8.0, 3.1.0, 2.8.1
>Reporter: Tm Alkemade
>Priority: Major
> Attachments: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-30 Thread Tm Alkemade (Jira)


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

Tm Alkemade commented on KAFKA-13773:
-

{quote}'but it's not clear if the previous shutdown was clean or not'
{quote}
The previous shutdown should not have been clean, since there wasn't enough 
disk space, I've attached the log for the time of the crash in 
kafka-2.8.0-crash.zip. I'll try to see if I can get the log for the first 
restart after the first disk space crash tomorrow, it might give some more 
insight.
{quote}After that, if the broker is restarted again, it should go through log 
recovery, did that happen?
{quote}
Kafka skipped log recovery once it had enough disk space. See 
kafka-2.7.0vs2.8.0.zip, it contains the log4j logs of the first start after 
resizing the disk.

> 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: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-30 Thread Jun Rao (Jira)


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

Jun Rao commented on KAFKA-13773:
-

[~Timelad] : Thanks for the additional logs. I took a quick look at 
kafka-2-2.8.0-before-resize.log. The broker did skip recovery, but it's not 
clear if the previous shutdown was clean or not. Do you have the log before 
that? During loading, the broker shut down abruptly due to the no space issue. 
After that, if the broker is restarted again, it should go through log 
recovery, did that happen?

 
{code:java}
2022-03-30 13:23:00,077 INFO Skipping recovery for all logs in 
/var/lib/kafka/data-0/kafka-log2 since clean shutdown file was found 
(kafka.log.LogManager) [main]
{code}
 

 

> 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: DiskAndOffsets.png, kafka-.zip, kafka-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-30 Thread Tm Alkemade (Jira)


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

Tm Alkemade commented on KAFKA-13773:
-

[~showuon] I ran additional tests on 2.6.0 2.7.0 and 2.8.0. I was not able to 
reproduce it on 2.6.0 and 2.7.0, however 2.8.0 does have the issue.

[~junrao] You're right, log recovery seems to be skipped in 2.8.0, I've 
attached a new zip file with the logs from 2.7.0 and 2.8.0 at the first startup 
after the disk resize. The 2.7.0 logs have a log recovery step, however the 
2.8.0 logs contain: 
{code:java}
2022-03-30 13:25:52,577 INFO Skipping recovery for all logs in 
/var/lib/kafka/data-0/kafka-log2 since clean shutdown file was found 
(kafka.log.LogManager) [main] {code}
It should be mentioned that since this is on Kubernetes, there are several 
attempts to restart the Kafka pod where Kafka tries, and fails, to start. 
Perhaps the clean shutdown file is created in one of these restarts. The new 
zip also contains the log of one of the failed restarts. Unfortunately its hard 
for me to get the log for the first failed restart. I've also noticed that the 
clean shutdown file is contained in the 'kafka-.zip' file I attached, even 
though those files were taken before resizing the data disk.

I've also added a zipfile of the actual crash in case its useful.

 

> 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-2.7.0vs2.8.0.zip, 
> kafka-2.8.0-crash.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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-29 Thread Jun Rao (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-13773?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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/.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 .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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-29 Thread Luke Chen (Jira)


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

Luke Chen commented on KAFKA-13773:
---

[~junrao] , do you have any thoughts about this issue? Thanks.

> 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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-29 Thread Luke Chen (Jira)


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

Luke Chen commented on KAFKA-13773:
---

Thanks [~Timelad] , testing v2.7.0, and v2.6.0 should be good enough IMO. 
Thanks.

> 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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-29 Thread Tm Alkemade (Jira)


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

Tm Alkemade commented on KAFKA-13773:
-

[~showuon] I haven't tested it on other versions, I might have some time 
tomorrow to do a sample of other versions. Are there any particular versions 
you're interested in?

> 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/.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/.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/.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)


[jira] [Commented] (KAFKA-13773) Data loss after recovery from crash due to full hard disk

2022-03-29 Thread Luke Chen (Jira)


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

Luke Chen commented on KAFKA-13773:
---

[~Timelad] , thanks for reporting this issue. You mentioned you encountered 
this issue in kafka 2.8.1 and 3.1.0. Did it happen before v2.8.1? I'd like to 
know if this is a regression issue, or a long existing issue. Thanks.

> 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/.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/.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/.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)