[ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15708281#comment-15708281 ]
Harald Kirsch commented on KAFKA-1194: -------------------------------------- The report of success was slightly exaggerted :-( After several hours of flawless operation, cleaner threads manage to trip over their own feed, it seems, with a variety of exceptions. I have seen this now with a single cleaner thread as well as with 4 threads. With 4 threads, I got variant 1 below 2 times then variant 2, then variant three but all within 10 seconds. I wonder in what kind of mood (mode) the system is that this all happens within a few seconds. Let me know if I can help with more information, details of the setup, configuration details or experiments, debug logging. Variant 1: {noformat} [2016-11-30 08:26:26,576] ERROR [kafka-log-cleaner-thread-1], Error due to (kafka.log.LogCleaner) kafka.common.KafkaStorageException: Failed to change the log file suffix from to .deleted for log segment 58972 at kafka.log.LogSegment.kafkaStorageException$1(LogSegment.scala:327) at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:329) at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:956) at kafka.log.Log$$anonfun$replaceSegments$1.apply(Log.scala:1002) at kafka.log.Log$$anonfun$replaceSegments$1.apply(Log.scala:997) at scala.collection.immutable.List.foreach(List.scala:318) at kafka.log.Log.replaceSegments(Log.scala:997) at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:425) at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:364) at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363) at scala.collection.immutable.List.foreach(List.scala:318) at kafka.log.Cleaner.clean(LogCleaner.scala:363) at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:239) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:218) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) Caused by: java.nio.file.FileAlreadyExistsException: d:\Search\kafka\fileshare-10\00000000000000058972.log -> d:\Search\kafka\fileshare-10\00000000000000058972.log.deleted at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:81) at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:387) at sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287) at java.nio.file.Files.move(Files.java:1395) at org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:670) at kafka.log.FileMessageSet.renameTo(FileMessageSet.scala:431) ... 14 more Suppressed: java.nio.file.AccessDeniedException: d:\Search\kafka\fileshare-10\00000000000000058972.log -> d:\Search\kafka\fileshare-10\00000000000000058972.log.deleted at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83) at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301) at sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287) at java.nio.file.Files.move(Files.java:1395) at org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:667) ... 15 more {noformat} Variant 2: {noformat} [2016-11-30 08:26:30,467] ERROR [kafka-log-cleaner-thread-3], Error due to (kafka.log.LogCleaner) kafka.common.InvalidOffsetException: Attempt to append an offset (59264) to position 61 no larger than the last offset appended (66994) to d:\Search\kafka\fileshare-10\00000000000000000000.index.swap.cleaned. at kafka.log.OffsetIndex$$anonfun$append$1.apply$mcV$sp(OffsetIndex.scala:132) at kafka.log.OffsetIndex$$anonfun$append$1.apply(OffsetIndex.scala:122) at kafka.log.OffsetIndex$$anonfun$append$1.apply(OffsetIndex.scala:122) at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234) at kafka.log.OffsetIndex.append(OffsetIndex.scala:122) at kafka.log.LogSegment.append(LogSegment.scala:105) at kafka.log.Cleaner.cleanInto(LogCleaner.scala:504) at kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:404) at kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:400) at scala.collection.immutable.List.foreach(List.scala:318) at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:400) at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:364) at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363) at scala.collection.immutable.List.foreach(List.scala:318) at kafka.log.Cleaner.clean(LogCleaner.scala:363) at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:239) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:218) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) {noformat} Variant 3: {noformat} [2016-11-30 08:26:35,967] ERROR [kafka-log-cleaner-thread-0], Error due to (kafka.log.LogCleaner) java.nio.file.AccessDeniedException: d:\Search\kafka\fileshare-10\00000000000000000000.log.swap.cleaned at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83) at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102) at sun.nio.fs.WindowsFileSystemProvider.newFileChannel(WindowsFileSystemProvider.java:115) at java.nio.channels.FileChannel.open(FileChannel.java:287) at java.nio.channels.FileChannel.open(FileChannel.java:335) at kafka.log.FileMessageSet$.openChannel(FileMessageSet.scala:461) at kafka.log.FileMessageSet.<init>(FileMessageSet.scala:86) at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:393) at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:364) at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363) at scala.collection.immutable.List.foreach(List.scala:318) at kafka.log.Cleaner.clean(LogCleaner.scala:363) at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:239) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:218) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) {noformat} > The kafka broker cannot delete the old log files after the configured time > -------------------------------------------------------------------------- > > Key: KAFKA-1194 > URL: https://issues.apache.org/jira/browse/KAFKA-1194 > Project: Kafka > Issue Type: Bug > Components: log > Affects Versions: 0.8.1 > Environment: window > Reporter: Tao Qin > Assignee: Jay Kreps > Labels: features, patch > Fix For: 0.10.2.0 > > Attachments: KAFKA-1194.patch, Untitled.jpg, kafka-1194-v1.patch, > kafka-1194-v2.patch > > Original Estimate: 72h > Remaining Estimate: 72h > > We tested it in windows environment, and set the log.retention.hours to 24 > hours. > # The minimum age of a log file to be eligible for deletion > log.retention.hours=24 > After several days, the kafka broker still cannot delete the old log file. > And we get the following exceptions: > [2013-12-19 01:57:38,528] ERROR Uncaught exception in scheduled task > 'kafka-log-retention' (kafka.utils.KafkaScheduler) > kafka.common.KafkaStorageException: Failed to change the log file suffix from > to .deleted for log segment 1516723 > at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:249) > at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:638) > at kafka.log.Log.kafka$log$Log$$deleteSegment(Log.scala:629) > at kafka.log.Log$$anonfun$deleteOldSegments$1.apply(Log.scala:418) > at kafka.log.Log$$anonfun$deleteOldSegments$1.apply(Log.scala:418) > at > scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:59) > at scala.collection.immutable.List.foreach(List.scala:76) > at kafka.log.Log.deleteOldSegments(Log.scala:418) > at > kafka.log.LogManager.kafka$log$LogManager$$cleanupExpiredSegments(LogManager.scala:284) > at > kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:316) > at > kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:314) > at > scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:743) > at scala.collection.Iterator$class.foreach(Iterator.scala:772) > at > scala.collection.JavaConversions$JIteratorWrapper.foreach(JavaConversions.scala:573) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:73) > at > scala.collection.JavaConversions$JListWrapper.foreach(JavaConversions.scala:615) > at > scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:742) > at kafka.log.LogManager.cleanupLogs(LogManager.scala:314) > at > kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(LogManager.scala:143) > at kafka.utils.KafkaScheduler$$anon$1.run(KafkaScheduler.scala:100) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:724) > I think this error happens because kafka tries to rename the log file when it > is still opened. So we should close the file first before rename. > The index file uses a special data structure, the MappedByteBuffer. Javadoc > describes it as: > A mapped byte buffer and the file mapping that it represents remain valid > until the buffer itself is garbage-collected. > Fortunately, I find a forceUnmap function in kafka code, and perhaps it can > be used to free the MappedByteBuffer. -- This message was sent by Atlassian JIRA (v6.3.4#6332)