Tao Qin created KAFKA-1194:
------------------------------

             Summary: 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
             Fix For: 0.8.1


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.1.5#6160)

Reply via email to