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

Jun Rao edited comment on KAFKA-13773 at 3/31/22 7:25 PM:
----------------------------------------------------------

[~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}
 

The issue with skipping recovery is that some of the preallocated timeindex 
file won't be shrunk to the right size and we will pick up some garbage as the 
timestamp.


was (Author: junrao):
[~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 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