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

Harald Kirsch commented on KAFKA-1194:
--------------------------------------

Just stumbled over yet another problem instance. During startup, Kafka notices 
a corrupt log/index file and tries to repair it. Here is the stack trace:

{noformat}
[2016-08-03 13:56:17,467] INFO Found log file 
d:\Search\kafka\fileshare-1\00000000000000000000.log.swap from interrupted swap 
operation, repairing. (kafka.log.Log)
[2016-08-03 13:56:18,436] ERROR There was an error in one of the threads during 
logs loading: kafka.common.KafkaStorageException: Failed to change the index 
file suffix from .swap to  for log segment 0 (kafka.log.LogManager)
[2016-08-03 13:56:18,436] FATAL Fatal error during KafkaServer startup. Prepare 
to shutdown (kafka.server.KafkaServer)
kafka.common.KafkaStorageException: Failed to change the index file suffix from 
.swap to  for log segment 0
        at kafka.log.LogSegment.kafkaStorageException$1(LogSegment.scala:268)
        at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:274)
        at kafka.log.Log.replaceSegments(Log.scala:886)
        at kafka.log.Log$$anonfun$loadSegments$5.apply(Log.scala:230)
        at kafka.log.Log$$anonfun$loadSegments$5.apply(Log.scala:214)
        at scala.collection.immutable.Set$Set1.foreach(Set.scala:74)
        at kafka.log.Log.loadSegments(Log.scala:214)
        at kafka.log.Log.<init>(Log.scala:101)
        at 
kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$3$$anonfun$apply$10$$anonfun$apply$1.apply$mcV$sp(LogManager.scala:151)
        at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:56)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.nio.file.FileSystemException: 
d:\Search\kafka\fileshare-1\00000000000000000000.index.swap -> 
d:\Search\kafka\fileshare-1\00000000000000000000.index: The process cannot 
access the file because it is being used by another process.

        at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
        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.OffsetIndex.renameTo(OffsetIndex.scala:365)
        ... 14 more
        Suppressed: java.nio.file.FileSystemException: 
d:\Search\kafka\fileshare-1\00000000000000000000.index.swap -> 
d:\Search\kafka\fileshare-1\00000000000000000000.index: The process cannot 
access the file because it is being used by another process.

                at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
                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
[2016-08-03 13:56:18,451] INFO shutting down (kafka.server.KafkaServer)
[2016-08-03 13:56:18,467] INFO shut down completed (kafka.server.KafkaServer)
{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.1.0
>
>         Attachments: KAFKA-1194.patch, 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)

Reply via email to