[
https://issues.apache.org/jira/browse/FLUME-2307?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14963850#comment-14963850
]
tzachi commented on FLUME-2307:
---
Hi guys, this is still happening with Flume 1.5.0-cdh5.4.2. As mentioned above
it is also not consistent. It works for few days as expected (which means
deleting old files after 2 checkpoints), and then from some unknown reason it
stops deleting old files, until the disk gets full and the logs start shouting
"Usable space exhausted". I am using 2 different file channels (with 2
different sinks) and both data directories experience the same issue (having
lots of old files).
> Remove Log writetimeout
> ---
>
> Key: FLUME-2307
> URL: https://issues.apache.org/jira/browse/FLUME-2307
> Project: Flume
> Issue Type: Bug
> Components: Channel
>Affects Versions: v1.4.0
>Reporter: Steve Zesch
>Assignee: Hari Shreedharan
> Fix For: v1.5.0
>
> Attachments: FLUME-2307-1.patch, FLUME-2307.patch
>
>
> I've observed Flume failing to clean up old log data in FileChannels. The
> amount of old log data can range anywhere from tens to hundreds of GB. I was
> able to confirm that the channels were in fact empty. This behavior always
> occurs after lock timeouts when attempting to put, take, rollback, or commit
> to a FileChannel. Once the timeout occurs, Flume stops cleaning up the old
> files. I was able to confirm that the Log's writeCheckpoint method was still
> being called and successfully obtaining a lock from tryLockExclusive(), but I
> was not able to confirm removeOldLogs being called. The application log did
> not include "Removing old file: log-xyz" for the old files which the Log
> class would output if they were correctly being removed. I suspect the lock
> timeouts were due to high I/O load at the time.
> Some stack traces:
> {code}
> org.apache.flume.ChannelException: Failed to obtain lock for writing to the
> log. Try increasing the log write timeout value. [channel=fileChannel]
> at
> org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doPut(FileChannel.java:478)
> at
> org.apache.flume.channel.BasicTransactionSemantics.put(BasicTransactionSemantics.java:93)
> at
> org.apache.flume.channel.BasicChannelSemantics.put(BasicChannelSemantics.java:80)
> at
> org.apache.flume.channel.ChannelProcessor.processEventBatch(ChannelProcessor.java:189)
> org.apache.flume.ChannelException: Failed to obtain lock for writing to the
> log. Try increasing the log write timeout value. [channel=fileChannel]
> at
> org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doCommit(FileChannel.java:594)
> at
> org.apache.flume.channel.BasicTransactionSemantics.commit(BasicTransactionSemantics.java:151)
> at
> dataxu.flume.plugins.avro.AsyncAvroSink.process(AsyncAvroSink.java:548)
> at
> dataxu.flume.plugins.ClassLoaderFlumeSink.process(ClassLoaderFlumeSink.java:33)
> at
> org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:68)
> at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
> at java.lang.Thread.run(Thread.java:619)
> org.apache.flume.ChannelException: Failed to obtain lock for writing to the
> log. Try increasing the log write timeout value. [channel=fileChannel]
> at
> org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doRollback(FileChannel.java:621)
> at
> org.apache.flume.channel.BasicTransactionSemantics.rollback(BasicTransactionSemantics.java:168)
> at
> org.apache.flume.channel.ChannelProcessor.processEventBatch(ChannelProcessor.java:194)
> at
> dataxu.flume.plugins.avro.AvroSource.appendBatch(AvroSource.java:209)
> at sun.reflect.GeneratedMethodAccessor19.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> at java.lang.reflect.Method.invoke(Method.java:597)
> at
> org.apache.avro.ipc.specific.SpecificResponder.respond(SpecificResponder.java:91)
> at org.apache.avro.ipc.Responder.respond(Responder.java:151)
> at
> org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.messageReceived(NettyServer.java:188)
> at
> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:75)
> at
> org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.handleUpstream(NettyServer.java:173)
> at
> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
> at
> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:792)
> at
>