[ https://issues.apache.org/jira/browse/KAFKA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16577335#comment-16577335 ]
Dong Lin edited comment on KAFKA-7278 at 8/11/18 9:54 PM: ---------------------------------------------------------- [~ijuma] Yeah the latest code in trunk seems to have this issue. The following sequence of events may happen: 1) There is segment1, segment2 and segment 3 for a given partition 2) LogCleaner determines to merge segment1 and segment2 into a new segment and will call Log.replaceSegments(..., oldSegments=[segment1, segment2]) 3) Log retention is triggered and Log.deleteSegment(segment=segment1) is called and executed. This renames the files for segment1 from log directory. 4) Log.replaceSegments(oldSegments=[segment1, segment2]) is executed and Log.asyncDeleteSegment(segment1) is executed, which fails to find files for segment1 and throws IOException. was (Author: lindong): [~ijuma] Yeah the latest code in trunk seems to have this issue. The following sequence of events may happen: 1) There is segment1, segment2 and segment 3 for a given partition 2) LogCleaner determines to merge segment1 and segment2 into segment3 and will call Log.replaceSegments(..., oldSegments=[segment1, segment2]) 3) Log retention is triggered and Log.deleteSegment(segment=segment1) is called and executed. This renames the files for segment1 from log directory. 4) Log.replaceSegments(oldSegments=[segment1, segment2]) is executed and Log.asyncDeleteSegment(segment1) is executed, which fails to find files for segment1 and throws IOException. > replaceSegments() should not call asyncDeleteSegment() for segments which > have been removed from segments list > -------------------------------------------------------------------------------------------------------------- > > Key: KAFKA-7278 > URL: https://issues.apache.org/jira/browse/KAFKA-7278 > Project: Kafka > Issue Type: Improvement > Reporter: Dong Lin > Assignee: Dong Lin > Priority: Major > > Currently Log.replaceSegments() will call `asyncDeleteSegment(...)` for every > segment listed in the `oldSegments`. oldSegments should be constructed from > Log.segments and only contain segments listed in Log.segments. > However, Log.segments may be modified between the time oldSegments is > determined to the time Log.replaceSegments() is called. If there are > concurrent async deletion of the same log segment file, Log.replaceSegments() > will call asyncDeleteSegment() for a segment that does not exist and Kafka > server may shutdown the log directory due to NoSuchFileException. > This is likely the root cause of > https://issues.apache.org/jira/browse/KAFKA-6188. > Given the understanding of the problem, we should be able to fix the issue by > only deleting segment if the segment can be found in Log.segments. > > -- This message was sent by Atlassian JIRA (v7.6.3#76005)