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

Haruki Okada edited comment on KAFKA-13403 at 4/26/22 7:35 AM:
---------------------------------------------------------------

[~showuon] Hi, could you help reviewing the PR 
[https://github.com/apache/kafka/pull/11438] ?

 

-There seems to be another ticket likely due to the same cause: 
https://issues.apache.org/jira/browse/KAFKA-13855-


After took another look at 13855, seems currently there's no clue to conclude 
it is the same cause.


was (Author: ocadaruma):
[~showuon] Hi, could you help reviewing the PR 
[https://github.com/apache/kafka/pull/11438] ?

 

There seems to be another ticket likely due to the same cause: 
https://issues.apache.org/jira/browse/KAFKA-13855

> KafkaServer crashes when deleting topics due to the race in log deletion
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-13403
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13403
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 2.4.1
>            Reporter: Haruki Okada
>            Assignee: Haruki Okada
>            Priority: Major
>
> h2. Environment
>  * OS: CentOS Linux release 7.6
>  * Kafka version: 2.4.1
>  * 
>  ** But as far as I checked the code, I think same phenomenon could happen 
> even on trunk
>  * Kafka log directory: RAID1+0 (i.e. not using JBOD so only single log.dirs 
> is set)
>  * Java version: AdoptOpenJDK 1.8.0_282
> h2. Phenomenon
> When we were in the middle of deleting several topics by `kafka-topics.sh 
> --delete --topic blah-blah`, one broker in our cluster crashed due to 
> following exception:
>  
> {code:java}
> [2021-10-21 18:19:19,122] ERROR Shutdown broker because all log dirs in 
> /data/kafka have failed (kafka.log.LogManager)
> {code}
>  
>  
> We also found NoSuchFileException was thrown right before the crash when 
> LogManager tried to delete logs for some partitions.
>  
> {code:java}
> [2021-10-21 18:19:18,849] ERROR Error while deleting log for foo-bar-topic-5 
> in dir /data/kafka (kafka.server.LogDirFailureChannel)
> java.nio.file.NoSuchFileException: 
> /data/kafka/foo-bar-topic-5.df3626d2d9eb41a2aeb0b8d55d7942bd-delete/00000000000003877066.timeindex.deleted
>         at 
> sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)
>         at 
> sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
>         at 
> sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
>         at 
> sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55)
>         at 
> sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:144)
>         at 
> sun.nio.fs.LinuxFileSystemProvider.readAttributes(LinuxFileSystemProvider.java:99)
>         at java.nio.file.Files.readAttributes(Files.java:1737)
>         at java.nio.file.FileTreeWalker.getAttributes(FileTreeWalker.java:219)
>         at java.nio.file.FileTreeWalker.visit(FileTreeWalker.java:276)
>         at java.nio.file.FileTreeWalker.next(FileTreeWalker.java:372)
>         at java.nio.file.Files.walkFileTree(Files.java:2706)
>         at java.nio.file.Files.walkFileTree(Files.java:2742)
>         at org.apache.kafka.common.utils.Utils.delete(Utils.java:732)
>         at kafka.log.Log.$anonfun$delete$2(Log.scala:2036)
>         at 
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
>         at kafka.log.Log.maybeHandleIOException(Log.scala:2343)
>         at kafka.log.Log.delete(Log.scala:2030)
>         at kafka.log.LogManager.deleteLogs(LogManager.scala:826)
>         at kafka.log.LogManager.$anonfun$deleteLogs$6(LogManager.scala:840)
>         at 
> kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:116)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:65)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> {code}
> So, the log-dir was marked as offline and ended up with KafkaServer crash 
> because the broker has only single log-dir.
> h2. Cause
> We also found below logs right before the NoSuchFileException.
>  
> {code:java}
> [2021-10-21 18:18:17,829] INFO Log for partition foo-bar-5 is renamed to 
> /data/kafka/foo-bar-5.df3626d2d9eb41a2aeb0b8d55d7942bd-delete and is 
> scheduled for deletion (kafka.log.LogManager)
> [2021-10-21 18:18:17,900] INFO [Log partition=foo-bar-5, dir=/data/kafka] 
> Found deletable segments with base offsets [3877066] due to retention time 
> 172800000ms breach (kafka.log.Log)[2021-10-21 18:18:17,901] INFO [Log 
> partition=foo-bar-5, dir=/data/kafka] Scheduling segments for deletion 
> List(LogSegment(baseOffset=3877066, size=90316366, 
> lastModifiedTime=1634634956000, largestTime=1634634955854)) (kafka.log.Log)
> {code}
> After checking through Kafka code, we concluded that there was a race between 
> "kafka-log-retention" and "kafka-delete-logs" scheduler threads.
>  
> Detailed timeline was like below:
>  - Precondition: there was two log segments (3877066, 4271262) for the 
> partition foo-bar-5
>  
> ||time||thread||event||files under partition dir||
> |2021-10-21 18:18:17,901|kafka-log-retention|Scheduled deletion for segment 
> 3877066 due to retention|3877066, 4271262|
> |2021-10-21 18:19:17,830|kafka-delete-logs|Starting to execute Log.delete() 
> (which was scheduled by topic deletion)
>   
>  Deleted all log segments 
> ([https://github.com/apache/kafka/blob/2.4.1/core/src/main/scala/kafka/log/Log.scala#L2031])
>  However, in the meantim, 3877066 was scheduled for deletion due to retention 
> and already removed from segment-list, 3877066 was not listed up as deletion 
> target here.|3877066|
> |2021-10-21 18:19:17,830 + X|kafka-delete-logs|Start deleting entire 
> partition directory by calling Utils.delete() 
> ([https://github.com/apache/kafka/blob/2.4.1/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L729])
>   
>  Files.walkFileTree starts to walk inside the directory, and found 
> 3877066|3877066|
> |2021-10-21 18:19:17,830 + X|kafka-log-retention|Delete segment 3877066 as 
> scheduled|empty|
> |2021-10-21 18:19:18,849|kafka-delete-logs|Files.walkFileTree tried to read 
> attributes of 3877066 but it failed with NoSuchFileException|empty|
> h2. Suggested solution
>  * I would like to suggest to fix Utils.delete to swallow NoSuchFileException 
> and just returns FileVisitResult.CONTINUE here
>  ** 
> [https://github.com/apache/kafka/blob/2.4.1/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L736]
>  * Also, we should use Files.deleteIfExists here
>  ** 
> [https://github.com/apache/kafka/blob/2.4.1/clients/src/main/java/org/apache/kafka/common/utils/Utils.java#L743]
>  ** Though it was not the case for our situation, this could lead 
> NoSuchFileException too because the file might be deleted by another thread 
> by same flow.
>  * What Utils.delete does is just deleting children recursively, so 
> NoSuchFileException just means we don't have to delete it, so ignoring it and 
> continuing should be safe.
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to