Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: v1.4.0
    • Fix Version/s: v1.5.0
    • Component/s: Channel
    • Labels:
      None

      Description

      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:

      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 org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:321)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:303)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:220)
              at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:75)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
              at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
              at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
              at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
              at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:94)
              at org.jboss.netty.channel.socket.nio.AbstractNioWorker.processSelectedKeys(AbstractNioWorker.java:364)
              at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:238)
              at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
              at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
              at java.lang.Thread.run(Thread.java:619)
      

      Channel Config:

      agent.channels.fileChannel.type = file
      agent.channels.fileChannel.checkpointDir = /var/log/flume-ng/channels/fileChannel/checkpoint
      agent.channels.fileChannel.dataDirs = /var/log/flume-ng/channels/fileChannel/data
      agent.channels.fileChannel.capacity = 100000000
      agent.channels.fileChannel.transactionCapacity = 100000000
      agent.channels.fileChannel.maxFileSize = 104857600 
      
      1. FLUME-2307.patch
        10 kB
        Hari Shreedharan
      2. FLUME-2307-1.patch
        18 kB
        Hari Shreedharan

        Issue Links

          Activity

          Hide
          hshreedharan Hari Shreedharan added a comment -

          Do you have backup checkpoint enabled?

          Show
          hshreedharan Hari Shreedharan added a comment - Do you have backup checkpoint enabled?
          Hide
          szesch Steve Zesch added a comment -

          We do not.

          Show
          szesch Steve Zesch added a comment - We do not.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Also can you post ls -l of your checkpoint directory? When was your checkpoint directory's contents last updated? Even if the writeCheckpoint method is called the checkpoint may not actually have been written out (since it tries to grab an exclusive lock on the channel).

          Show
          hshreedharan Hari Shreedharan added a comment - Also can you post ls -l of your checkpoint directory? When was your checkpoint directory's contents last updated? Even if the writeCheckpoint method is called the checkpoint may not actually have been written out (since it tries to grab an exclusive lock on the channel).
          Hide
          szesch Steve Zesch added a comment -

          We wiped the checkpoint and data directories, but I did check beforehand and the contents were still being updated. The oldest log file was from the 29th and the checkpoint directories contents were all updated today.

          Show
          szesch Steve Zesch added a comment - We wiped the checkpoint and data directories, but I did check beforehand and the contents were still being updated. The oldest log file was from the 29th and the checkpoint directories contents were all updated today.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          On reading the code, I think I know what is happening here:

          • A bunch of takes happen from the queue -> the events end up in the inflightTakes file, which means that they are in the inflightTakes and log-X won't be deleted until the transaction gets committed or rolled back.
          • Commit or rollback - first thing we do is grab the lock - which fails, so the rollback never happens, causing the transaction to be stuck in limbo - never committed or rolled back.
          • This means log-X would never get deleted, so any log file with Y > X would never get deleted.

          I don't see a way around it, since we need to make sure that the rollback is completed so the events are put back into the queue. Else we would essentially lose data (on restart since they are in the inflights, they get put back into the channel - so the data becomes "available" again).

          Show
          hshreedharan Hari Shreedharan added a comment - On reading the code, I think I know what is happening here: A bunch of takes happen from the queue -> the events end up in the inflightTakes file, which means that they are in the inflightTakes and log-X won't be deleted until the transaction gets committed or rolled back. Commit or rollback - first thing we do is grab the lock - which fails, so the rollback never happens, causing the transaction to be stuck in limbo - never committed or rolled back. This means log-X would never get deleted, so any log file with Y > X would never get deleted. I don't see a way around it, since we need to make sure that the rollback is completed so the events are put back into the queue. Else we would essentially lose data (on restart since they are in the inflights, they get put back into the channel - so the data becomes "available" again).
          Hide
          szesch Steve Zesch added a comment -

          Would this cause inflightTakes to grow very large as well? We did not see that file size inflate.

          Show
          szesch Steve Zesch added a comment - Would this cause inflightTakes to grow very large as well? We did not see that file size inflate.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          No, inflightTakes and inflightPuts contain only a bunch of longs, basically file id + offsets to the actual data. So when a transaction is neither committed not rolled back, the list of event offsets remain there and when we look for files to delete, we keep all files newer than the files represented in the queue (checkpoint) and the inflightTakes and inflightPuts.

          Show
          hshreedharan Hari Shreedharan added a comment - No, inflightTakes and inflightPuts contain only a bunch of longs, basically file id + offsets to the actual data. So when a transaction is neither committed not rolled back, the list of event offsets remain there and when we look for files to delete, we keep all files newer than the files represented in the queue (checkpoint) and the inflightTakes and inflightPuts.
          Hide
          brocknoland Brock Noland added a comment -

          Can we eliminate the write timeout? I have wondered for some time what purpose is it actually serving?

          Show
          brocknoland Brock Noland added a comment - Can we eliminate the write timeout? I have wondered for some time what purpose is it actually serving?
          Hide
          szesch Steve Zesch added a comment -

          Thinking on this some more, if a transaction were stuck in limbo like you said, wouldn't this result in channel backup? When we ran into the problem of these logs never being cleaned up, channel size was always either zero or a very small number of events. Also, after the initial lock timeouts, the source and sink were still able to put and take from it. It seemed like the original timeout rendered the channel unable to clean up after itself.

          Show
          szesch Steve Zesch added a comment - Thinking on this some more, if a transaction were stuck in limbo like you said, wouldn't this result in channel backup? When we ran into the problem of these logs never being cleaned up, channel size was always either zero or a very small number of events. Also, after the initial lock timeouts, the source and sink were still able to put and take from it. It seemed like the original timeout rendered the channel unable to clean up after itself.
          Hide
          brocknoland Brock Noland added a comment -

          I have wondered for some time what purpose is it actually serving?

          To add to my own question...as soon as the timeout occurs the client is going to move on to the next action be it put or take sit waiting in the lock again. That is the timeout doesn't improve the speed at which the lock requiring action occurs.

          Show
          brocknoland Brock Noland added a comment - I have wondered for some time what purpose is it actually serving? To add to my own question...as soon as the timeout occurs the client is going to move on to the next action be it put or take sit waiting in the lock again. That is the timeout doesn't improve the speed at which the lock requiring action occurs.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Steve Zesch - No, since the events are actually removed from the queue and put into a temporary structure. So the channel would not backlog. The events would be re-inserted into the queue if rollback happens, else only on restart the events get reinserted.

          Brock Noland - I agree. I am a +1 on removing the write timeout.

          Show
          hshreedharan Hari Shreedharan added a comment - Steve Zesch - No, since the events are actually removed from the queue and put into a temporary structure. So the channel would not backlog. The events would be re-inserted into the queue if rollback happens, else only on restart the events get reinserted. Brock Noland - I agree. I am a +1 on removing the write timeout.
          Hide
          iijima_satoshi Satoshi Iijima added a comment -

          I faced the same issue before, too.
          I added below to flume.conf as workaround. After that this issue have not happened.

          agent.channels.ch1.write-timeout = 30

          Show
          iijima_satoshi Satoshi Iijima added a comment - I faced the same issue before, too. I added below to flume.conf as workaround. After that this issue have not happened. agent.channels.ch1.write-timeout = 30
          Hide
          hshreedharan Hari Shreedharan added a comment -

          I will submit a patch later today removing the writeTimeout.

          Show
          hshreedharan Hari Shreedharan added a comment - I will submit a patch later today removing the writeTimeout.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Removing Log write timeout. Leaving checkpoint write timeout as is - since the checkpoint thread will just try again later. I don't really see much value in removing that one.

          Show
          hshreedharan Hari Shreedharan added a comment - Removing Log write timeout. Leaving checkpoint write timeout as is - since the checkpoint thread will just try again later. I don't really see much value in removing that one.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          On second thought, the checkpoint write timeout can also be removed as the thread is scheduled with fixed delay, so once the checkpoint is written the next checkpoint is not attempted for at least checkpoint interval. Also updated the docs removing those two parameters.

          Show
          hshreedharan Hari Shreedharan added a comment - On second thought, the checkpoint write timeout can also be removed as the thread is scheduled with fixed delay, so once the checkpoint is written the next checkpoint is not attempted for at least checkpoint interval. Also updated the docs removing those two parameters.
          Hide
          brocknoland Brock Noland added a comment -

          +1

          I cannot test/commit as I am on an airplane.

          Show
          brocknoland Brock Noland added a comment - +1 I cannot test/commit as I am on an airplane.
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit b4ddd5829897f758f869a5fc3b08dcbf4b55156a in branch refs/heads/trunk from Jarek Jarcec Cecho
          [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=b4ddd58 ]

          FLUME-2307. Remove Log writetimeout

          (Hari Shreedharan via Jarek Jarcec Cecho)

          Show
          jira-bot ASF subversion and git services added a comment - Commit b4ddd5829897f758f869a5fc3b08dcbf4b55156a in branch refs/heads/trunk from Jarek Jarcec Cecho [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=b4ddd58 ] FLUME-2307 . Remove Log writetimeout (Hari Shreedharan via Jarek Jarcec Cecho)
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 0cb6ce69140f137af947de4e0828ff73a623f042 in branch refs/heads/flume-1.5 from Jarek Jarcec Cecho
          [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=0cb6ce6 ]

          FLUME-2307. Remove Log writetimeout

          (Hari Shreedharan via Jarek Jarcec Cecho)

          Show
          jira-bot ASF subversion and git services added a comment - Commit 0cb6ce69140f137af947de4e0828ff73a623f042 in branch refs/heads/flume-1.5 from Jarek Jarcec Cecho [ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=0cb6ce6 ] FLUME-2307 . Remove Log writetimeout (Hari Shreedharan via Jarek Jarcec Cecho)
          Hide
          jarcec Jarek Jarcec Cecho added a comment -

          Thank you for your contribution Hari Shreedharan!

          Show
          jarcec Jarek Jarcec Cecho added a comment - Thank you for your contribution Hari Shreedharan !
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in flume-trunk #549 (See https://builds.apache.org/job/flume-trunk/549/)
          FLUME-2307. Remove Log writetimeout (jarcec: http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git&a=commit&h=b4ddd5829897f758f869a5fc3b08dcbf4b55156a)

          • flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java
          • flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java
          • flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java
          • flume-ng-doc/sphinx/FlumeUserGuide.rst
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in flume-trunk #549 (See https://builds.apache.org/job/flume-trunk/549/ ) FLUME-2307 . Remove Log writetimeout (jarcec: http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git&a=commit&h=b4ddd5829897f758f869a5fc3b08dcbf4b55156a ) flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java flume-ng-doc/sphinx/FlumeUserGuide.rst
          Hide
          nino4a Nina Safonova added a comment -

          Hi guys, recently we migrated to flume 1.5 and we are experiencing the similar issue: at some point flume stopped to remove old files, but it was creating a new ones so at the end we ran out of disk space. I log I see many messages for the same log-686 (it's exactly the one at which flume stopped to remove old logs:

          12 Aug 2014 05:24:49,956 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-686

          I see no "Removing old file: /local/flume-ng/data/channel1/log-686" message while for all the previous (and removed) logs I see:

          12 Aug 2014 05:09:49,607 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.Log.removeOldLogs:1060) - Removing old file: /local/flume-ng/data/channel1/log-685
          12 Aug 2014 05:09:49,715 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.Log.removeOldLogs:1060) - Removing old file: /local/flume-ng/data/channel1/log-685.meta

          Our configuration is:

          tracer.channels.channel1.type = FILE
          tracer.channels.channel1.checkpointDir = /local/flume-ng/checkpoints/channel1
          tracer.channels.channel1.dataDirs = /local/flume-ng/data/channel1
          tracer.channels.channel1.transactionCapacity = 5000
          tracer.channels.channel1.checkpointInterval = 100000
          tracer.channels.channel1.maxFileSize = 2097152000
          tracer.channels.channel1.capacity = 16000000
          tracer.channels.channel1.write-timeout = 60

          This happend twice during last 2 days.
          I wa able to debug it once and find out that in org.apache.flume.channel.file.Log.removeOldLogs(SortedSet<Integer> fileIDs) for fileIDs passed just log-686 and the latest file (which is increasing). Nothing in pendingDeletes. 26 entries in idLogFileMap, but none of them is deleted because minFileID is 686 and other files have greater ID. Why this is happening?

          Thanks

          Show
          nino4a Nina Safonova added a comment - Hi guys, recently we migrated to flume 1.5 and we are experiencing the similar issue: at some point flume stopped to remove old files, but it was creating a new ones so at the end we ran out of disk space. I log I see many messages for the same log-686 (it's exactly the one at which flume stopped to remove old logs: 12 Aug 2014 05:24:49,956 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-686 I see no "Removing old file: /local/flume-ng/data/channel1/log-686" message while for all the previous (and removed) logs I see: 12 Aug 2014 05:09:49,607 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.Log.removeOldLogs:1060) - Removing old file: /local/flume-ng/data/channel1/log-685 12 Aug 2014 05:09:49,715 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.Log.removeOldLogs:1060) - Removing old file: /local/flume-ng/data/channel1/log-685.meta Our configuration is: tracer.channels.channel1.type = FILE tracer.channels.channel1.checkpointDir = /local/flume-ng/checkpoints/channel1 tracer.channels.channel1.dataDirs = /local/flume-ng/data/channel1 tracer.channels.channel1.transactionCapacity = 5000 tracer.channels.channel1.checkpointInterval = 100000 tracer.channels.channel1.maxFileSize = 2097152000 tracer.channels.channel1.capacity = 16000000 tracer.channels.channel1.write-timeout = 60 This happend twice during last 2 days. I wa able to debug it once and find out that in org.apache.flume.channel.file.Log.removeOldLogs(SortedSet<Integer> fileIDs) for fileIDs passed just log-686 and the latest file (which is increasing). Nothing in pendingDeletes. 26 entries in idLogFileMap, but none of them is deleted because minFileID is 686 and other files have greater ID. Why this is happening? Thanks
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Can you find out how many events are still in the channel using the metrics

          Show
          hshreedharan Hari Shreedharan added a comment - Can you find out how many events are still in the channel using the metrics
          Hide
          nino4a Nina Safonova added a comment - - edited

          Unfortunately I already restarted flume. But channel was operating normally, sinks were reading from it, sources were writing to it. After restart no old logs was deleted so I did it manually. Here is the log of restart with channel1 related info:

          12 Aug 2014 21:36:38,022 INFO  [conf-file-poller-0] (org.apache.flume.channel.DefaultChannelFactory.create:40)  - Creating instance of channel channel1 type FILE
          12 Aug 2014 21:36:38,035 INFO  [conf-file-poller-0] (org.apache.flume.node.AbstractConfigurationProvider.loadChannels:205)  - Created channel channel1
          12 Aug 2014 21:36:38,205 INFO  [conf-file-poller-0] (org.apache.flume.node.AbstractConfigurationProvider.getConfiguration:119)  - Channel channel1 connected to [es-sink1, es-sink4, es-sink3, es-sink2]
          12 Aug 2014 21:36:38,221 INFO  [conf-file-poller-0] (org.apache.flume.node.Application.startAllComponents:139)  - Starting new configuration:{ sourceRunners:{} sinkRunners:{google-BQ-perf-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@6eb5845c counterGroup:{ name:null counters:{} } }, es-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4f04eccc counterGroup:{ name:null counters:{} } }, es-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4c566d9b counterGroup:{ name:null counters:{} } }, es-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@3e360244 counterGroup:{ name:null counters:{} } }, es-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4bcede44 counterGroup:{ name:null counters:{} } }, google-BQ-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@7a62693d counterGroup:{ name:null counters:{} } }, google-BQ-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@52eb6290 counterGroup:{ name:null counters:{} } }, google-BQ-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@5b940677 counterGroup:{ name:null counters:{} } }, google-BQ-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@53349d99 counterGroup:{ name:null counters:{} } }} channels:{channel1=FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }, channel2=FileChannel channel2 { dataDirs: [/local/flume-ng/data/channel2] }, channel3=org.apache.flume.channel.MemoryChannel{name: channel3}} }
          12 Aug 2014 21:36:38,222 INFO  [conf-file-poller-0] (org.apache.flume.node.Application.startAllComponents:146)  - Starting Channel channel1
          12 Aug 2014 21:36:38,222 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.FileChannel.start:259)  - Starting FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }...
          12 Aug 2014 21:36:38,270 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.replay:385)  - Found NextFileID 714, from [/local/flume-ng/data/channel1/log-691, /local/flume-ng/data/channel1/log-707, /local/flume-ng/data/channel1/log-696, /local/flume-ng/data/channel1/log-697, /local/flume-ng/data/channel1/log-702, /local/flume-ng/data/channel1/log-710, /local/flume-ng/data/channel1/log-704, /local/flume-ng/data/channel1/log-711, /local/flume-ng/data/channel1/log-698, /local/flume-ng/data/channel1/log-686, /local/flume-ng/data/channel1/log-688, /local/flume-ng/data/channel1/log-706, /local/flume-ng/data/channel1/log-712, /local/flume-ng/data/channel1/log-705, /local/flume-ng/data/channel1/log-714, /local/flume-ng/data/channel1/log-713, /local/flume-ng/data/channel1/log-700, /local/flume-ng/data/channel1/log-689, /local/flume-ng/data/channel1/log-687, /local/flume-ng/data/channel1/log-690, /local/flume-ng/data/channel1/log-708, /local/flume-ng/data/channel1/log-701, /local/flume-ng/data/channel1/log-703, /local/flume-ng/data/channel1/log-692, /local/flume-ng/data/channel1/log-694, /local/flume-ng/data/channel1/log-709, /local/flume-ng/data/channel1/log-695, /local/flume-ng/data/channel1/log-693, /local/flume-ng/data/channel1/log-699]
          12 Aug 2014 21:36:38,288 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.EventQueueBackingStoreFileV3.<init>:53)  - Starting up with /local/flume-ng/checkpoints/channel1/checkpoint and /local/flume-ng/checkpoints/channel1/checkpoint.meta
          12 Aug 2014 21:36:38,289 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.EventQueueBackingStoreFileV3.<init>:57)  - Reading checkpoint metadata from /local/flume-ng/checkpoints/channel1/checkpoint.meta
          12 Aug 2014 21:36:38,723 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:249)  - Starting replay of [/local/flume-ng/data/channel1/log-686, /local/flume-ng/data/channel1/log-687, /local/flume-ng/data/channel1/log-688, /local/flume-ng/data/channel1/log-689, /local/flume-ng/data/channel1/log-690, /local/flume-ng/data/channel1/log-691, /local/flume-ng/data/channel1/log-692, /local/flume-ng/data/channel1/log-693, /local/flume-ng/data/channel1/log-694, /local/flume-ng/data/channel1/log-695, /local/flume-ng/data/channel1/log-696, /local/flume-ng/data/channel1/log-697, /local/flume-ng/data/channel1/log-698, /local/flume-ng/data/channel1/log-699, /local/flume-ng/data/channel1/log-700, /local/flume-ng/data/channel1/log-701, /local/flume-ng/data/channel1/log-702, /local/flume-ng/data/channel1/log-703, /local/flume-ng/data/channel1/log-704, /local/flume-ng/data/channel1/log-705, /local/flume-ng/data/channel1/log-706, /local/flume-ng/data/channel1/log-707, /local/flume-ng/data/channel1/log-708, /local/flume-ng/data/channel1/log-709, /local/flume-ng/data/channel1/log-710, /local/flume-ng/data/channel1/log-711, /local/flume-ng/data/channel1/log-712, /local/flume-ng/data/channel1/log-713, /local/flume-ng/data/channel1/log-714]
          12 Aug 2014 21:36:38,729 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-686
          12 Aug 2014 21:36:38,845 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-687
          12 Aug 2014 21:36:38,867 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-688
          12 Aug 2014 21:36:38,895 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-689
          12 Aug 2014 21:36:38,908 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-690
          12 Aug 2014 21:36:38,922 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-691
          12 Aug 2014 21:36:38,933 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-692
          12 Aug 2014 21:36:38,945 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-693
          12 Aug 2014 21:36:38,957 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-694
          12 Aug 2014 21:36:38,965 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-695
          12 Aug 2014 21:36:38,978 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-696
          12 Aug 2014 21:36:38,988 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-697
          12 Aug 2014 21:36:38,994 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-698
          12 Aug 2014 21:36:39,005 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-699
          12 Aug 2014 21:36:39,017 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-700
          12 Aug 2014 21:36:39,022 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262)  - Replaying /local/flume-ng/data/channel1/log-701
          12 Aug 2014 21:36:58,554 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623194835 in /local/flume-ng/data/channel1/log-702
          12 Aug 2014 21:36:59,311 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623195343 in /local/flume-ng/data/channel1/log-703
          12 Aug 2014 21:37:00,637 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623195377 in /local/flume-ng/data/channel1/log-704
          12 Aug 2014 21:37:02,575 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623195415 in /local/flume-ng/data/channel1/log-705
          12 Aug 2014 21:37:04,328 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623195243 in /local/flume-ng/data/channel1/log-706
          12 Aug 2014 21:37:04,460 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623195572 in /local/flume-ng/data/channel1/log-707
          12 Aug 2014 21:37:05,668 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623195404 in /local/flume-ng/data/channel1/log-708
          12 Aug 2014 21:37:05,710 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623194925 in /local/flume-ng/data/channel1/log-709
          12 Aug 2014 21:37:06,347 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 1623194204 in /local/flume-ng/data/channel1/log-710
          12 Aug 2014 21:37:07,900 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 712946491 in /local/flume-ng/data/channel1/log-711
          12 Aug 2014 21:37:07,908 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657)  - Encountered EOF at 38156 in /local/flume-ng/data/channel1/log-714
          12 Aug 2014 21:37:07,945 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.replay:470)  - Rolling /local/flume-ng/data/channel1
          12 Aug 2014 21:37:07,946 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.roll:932)  - Roll start /local/flume-ng/data/channel1
          12 Aug 2014 21:37:07,946 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$Writer.<init>:214)  - Opened /local/flume-ng/data/channel1/log-715
          12 Aug 2014 21:37:07,947 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:214)  - Start checkpoint for /local/flume-ng/checkpoints/channel1/checkpoint, elements to sync = 8465
          12 Aug 2014 21:37:07,973 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.writeCheckpoint:1005)  - Updated checkpoint for file: /local/flume-ng/data/channel1/log-715 position: 0 logWriteOrderID: 1408282684040
          12 Aug 2014 21:37:07,973 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-686
          12 Aug 2014 21:37:07,979 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.FileChannel.start:285)  - Queue Size after replay: 418 [channel=channel1]
          12 Aug 2014 21:37:07,980 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.instrumentation.MonitoredCounterGroup.register:119)  - Monitored counter group for type: CHANNEL, name: channel1: Successfully registered new MBean.
          12 Aug 2014 21:37:07,980 INFO  [lifecycleSupervisor-1-0] (org.apache.flume.instrumentation.MonitoredCounterGroup.start:95)  - Component type: CHANNEL, name: channel1 started
          12 Aug 2014 21:37:08,139 INFO  [SinkRunner-PollingRunner-DefaultSinkProcessor] (org.apache.flume.channel.file.LogFile$RandomReader.checkOut:544)  - Opening /local/flume-ng/data/channel1/log-686 for read, remaining number of file handles available for reads of this file is 50
          12 Aug 2014 21:37:08,139 INFO  [SinkRunner-PollingRunner-DefaultSinkProcessor] (org.apache.flume.channel.file.LogFile$RandomReader.checkOut:544)  - Opening /local/flume-ng/data/channel1/log-686 for read, remaining number of file handles available for reads of this file is 50
          12 Aug 2014 21:37:08,139 INFO  [SinkRunner-PollingRunner-DefaultSinkProcessor] (org.apache.flume.channel.file.LogFile$RandomReader.checkOut:544)  - Opening /local/flume-ng/data/channel1/log-686 for read, remaining number of file handles available for reads of this file is 50
          12 Aug 2014 21:38:18,255 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:214)  - Start checkpoint for /local/flume-ng/checkpoints/channel1/checkpoint, elements to sync = 418
          12 Aug 2014 21:38:18,257 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:239)  - Updating checkpoint metadata: logWriteOrderID: 1408282684463, queueSize: 0, queueHead: 2970240
          12 Aug 2014 21:38:18,260 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.Log.writeCheckpoint:1005)  - Updated checkpoint for file: /local/flume-ng/data/channel1/log-715 position: 17282 logWriteOrderID: 1408282684463
          12 Aug 2014 21:38:18,261 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-686
          12 Aug 2014 21:38:18,266 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-687
          12 Aug 2014 21:38:18,271 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-688
          12 Aug 2014 21:38:18,277 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-689
          12 Aug 2014 21:38:18,282 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-690
          12 Aug 2014 21:38:18,287 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-691
          12 Aug 2014 21:38:18,293 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-692
          12 Aug 2014 21:38:18,298 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-693
          12 Aug 2014 21:38:18,303 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-694
          12 Aug 2014 21:38:18,309 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-695
          12 Aug 2014 21:38:18,314 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-696
          12 Aug 2014 21:38:18,319 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-697
          12 Aug 2014 21:38:18,325 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-698
          12 Aug 2014 21:38:18,330 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-699
          12 Aug 2014 21:38:18,335 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-700
          12 Aug 2014 21:38:18,340 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-701
          12 Aug 2014 21:38:18,346 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-702
          12 Aug 2014 21:38:18,351 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-703
          12 Aug 2014 21:38:18,356 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-704
          12 Aug 2014 21:38:18,362 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-705
          12 Aug 2014 21:38:18,367 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-706
          12 Aug 2014 21:38:18,372 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-707
          12 Aug 2014 21:38:18,378 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-708
          12 Aug 2014 21:38:18,383 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-709
          12 Aug 2014 21:38:18,388 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-710
          12 Aug 2014 21:38:18,394 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-711
          12 Aug 2014 21:38:18,399 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-712
          12 Aug 2014 21:38:18,404 INFO  [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-713
          

          (source was disabled so nothing interresting futher)
          and after I realized that flume is not going to remove old files I stopped it:

          12 Aug 2014 21:42:20,084 INFO  [agent-shutdown-hook] (org.apache.flume.node.Application.stopAllComponents:102)  - Shutting down configuration: { sourceRunners:{} sinkRunners:{google-BQ-perf-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@6eb5845c counterGroup:{ name:null counters:{runner.backoffs.consecutive=40, runner.backoffs=40} } }, es-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4f04eccc counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, es-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4c566d9b counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, es-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@3e360244 counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, es-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4bcede44 counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@7a62693d counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@52eb6290 counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@5b940677 counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@53349d99 counterGroup:{ name:null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }} channels:{channel1=FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }, channel2=FileChannel channel2 { dataDirs: [/local/flume-ng/data/channel2] }, channel3=org.apache.flume.channel.MemoryChannel{name: channel3}} }
          12 Aug 2014 21:42:20,102 INFO  [agent-shutdown-hook] (org.apache.flume.node.Application.stopAllComponents:126)  - Stopping Channel channel1
          12 Aug 2014 21:42:20,102 INFO  [agent-shutdown-hook] (org.apache.flume.lifecycle.LifecycleSupervisor.unsupervise:171)  - Stopping component: FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }
          12 Aug 2014 21:42:20,102 INFO  [agent-shutdown-hook] (org.apache.flume.channel.file.FileChannel.stop:305)  - Stopping FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }...
          12 Aug 2014 21:42:20,103 INFO  [agent-shutdown-hook] (org.apache.flume.channel.file.LogFile$Writer.close:370)  - Closing /local/flume-ng/data/channel1/log-715
          12 Aug 2014 21:42:20,103 INFO  [agent-shutdown-hook] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-715
          12 Aug 2014 21:42:20,108 INFO  [agent-shutdown-hook] (org.apache.flume.channel.file.LogFile$RandomReader.close:504)  - Closing RandomReader /local/flume-ng/data/channel1/log-714
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:149)  - Component type: CHANNEL, name: channel1 stopped
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:155)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.start.time == 1407893827980
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:161)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.stop.time == 1407894140114
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.capacity == 16000000
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.current.size == 0
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.put.attempt == 0
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.put.success == 0
          12 Aug 2014 21:42:20,114 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.take.attempt == 678
          12 Aug 2014 21:42:20,115 INFO  [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177)  - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.take.success == 418
          

          No events in channel according to shutdown message: channel1. channel.current.size == 0

          Show
          nino4a Nina Safonova added a comment - - edited Unfortunately I already restarted flume. But channel was operating normally, sinks were reading from it, sources were writing to it. After restart no old logs was deleted so I did it manually. Here is the log of restart with channel1 related info: 12 Aug 2014 21:36:38,022 INFO [conf-file-poller-0] (org.apache.flume.channel.DefaultChannelFactory.create:40) - Creating instance of channel channel1 type FILE 12 Aug 2014 21:36:38,035 INFO [conf-file-poller-0] (org.apache.flume.node.AbstractConfigurationProvider.loadChannels:205) - Created channel channel1 12 Aug 2014 21:36:38,205 INFO [conf-file-poller-0] (org.apache.flume.node.AbstractConfigurationProvider.getConfiguration:119) - Channel channel1 connected to [es-sink1, es-sink4, es-sink3, es-sink2] 12 Aug 2014 21:36:38,221 INFO [conf-file-poller-0] (org.apache.flume.node.Application.startAllComponents:139) - Starting new configuration:{ sourceRunners:{} sinkRunners:{google-BQ-perf-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@6eb5845c counterGroup:{ name: null counters:{} } }, es-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4f04eccc counterGroup:{ name: null counters:{} } }, es-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4c566d9b counterGroup:{ name: null counters:{} } }, es-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@3e360244 counterGroup:{ name: null counters:{} } }, es-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4bcede44 counterGroup:{ name: null counters:{} } }, google-BQ-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@7a62693d counterGroup:{ name: null counters:{} } }, google-BQ-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@52eb6290 counterGroup:{ name: null counters:{} } }, google-BQ-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@5b940677 counterGroup:{ name: null counters:{} } }, google-BQ-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@53349d99 counterGroup:{ name: null counters:{} } }} channels:{channel1=FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }, channel2=FileChannel channel2 { dataDirs: [/local/flume-ng/data/channel2] }, channel3=org.apache.flume.channel.MemoryChannel{name: channel3}} } 12 Aug 2014 21:36:38,222 INFO [conf-file-poller-0] (org.apache.flume.node.Application.startAllComponents:146) - Starting Channel channel1 12 Aug 2014 21:36:38,222 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.FileChannel.start:259) - Starting FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }... 12 Aug 2014 21:36:38,270 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.replay:385) - Found NextFileID 714, from [/local/flume-ng/data/channel1/log-691, /local/flume-ng/data/channel1/log-707, /local/flume-ng/data/channel1/log-696, /local/flume-ng/data/channel1/log-697, /local/flume-ng/data/channel1/log-702, /local/flume-ng/data/channel1/log-710, /local/flume-ng/data/channel1/log-704, /local/flume-ng/data/channel1/log-711, /local/flume-ng/data/channel1/log-698, /local/flume-ng/data/channel1/log-686, /local/flume-ng/data/channel1/log-688, /local/flume-ng/data/channel1/log-706, /local/flume-ng/data/channel1/log-712, /local/flume-ng/data/channel1/log-705, /local/flume-ng/data/channel1/log-714, /local/flume-ng/data/channel1/log-713, /local/flume-ng/data/channel1/log-700, /local/flume-ng/data/channel1/log-689, /local/flume-ng/data/channel1/log-687, /local/flume-ng/data/channel1/log-690, /local/flume-ng/data/channel1/log-708, /local/flume-ng/data/channel1/log-701, /local/flume-ng/data/channel1/log-703, /local/flume-ng/data/channel1/log-692, /local/flume-ng/data/channel1/log-694, /local/flume-ng/data/channel1/log-709, /local/flume-ng/data/channel1/log-695, /local/flume-ng/data/channel1/log-693, /local/flume-ng/data/channel1/log-699] 12 Aug 2014 21:36:38,288 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.EventQueueBackingStoreFileV3.<init>:53) - Starting up with /local/flume-ng/checkpoints/channel1/checkpoint and /local/flume-ng/checkpoints/channel1/checkpoint.meta 12 Aug 2014 21:36:38,289 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.EventQueueBackingStoreFileV3.<init>:57) - Reading checkpoint metadata from /local/flume-ng/checkpoints/channel1/checkpoint.meta 12 Aug 2014 21:36:38,723 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:249) - Starting replay of [/local/flume-ng/data/channel1/log-686, /local/flume-ng/data/channel1/log-687, /local/flume-ng/data/channel1/log-688, /local/flume-ng/data/channel1/log-689, /local/flume-ng/data/channel1/log-690, /local/flume-ng/data/channel1/log-691, /local/flume-ng/data/channel1/log-692, /local/flume-ng/data/channel1/log-693, /local/flume-ng/data/channel1/log-694, /local/flume-ng/data/channel1/log-695, /local/flume-ng/data/channel1/log-696, /local/flume-ng/data/channel1/log-697, /local/flume-ng/data/channel1/log-698, /local/flume-ng/data/channel1/log-699, /local/flume-ng/data/channel1/log-700, /local/flume-ng/data/channel1/log-701, /local/flume-ng/data/channel1/log-702, /local/flume-ng/data/channel1/log-703, /local/flume-ng/data/channel1/log-704, /local/flume-ng/data/channel1/log-705, /local/flume-ng/data/channel1/log-706, /local/flume-ng/data/channel1/log-707, /local/flume-ng/data/channel1/log-708, /local/flume-ng/data/channel1/log-709, /local/flume-ng/data/channel1/log-710, /local/flume-ng/data/channel1/log-711, /local/flume-ng/data/channel1/log-712, /local/flume-ng/data/channel1/log-713, /local/flume-ng/data/channel1/log-714] 12 Aug 2014 21:36:38,729 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-686 12 Aug 2014 21:36:38,845 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-687 12 Aug 2014 21:36:38,867 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-688 12 Aug 2014 21:36:38,895 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-689 12 Aug 2014 21:36:38,908 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-690 12 Aug 2014 21:36:38,922 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-691 12 Aug 2014 21:36:38,933 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-692 12 Aug 2014 21:36:38,945 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-693 12 Aug 2014 21:36:38,957 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-694 12 Aug 2014 21:36:38,965 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-695 12 Aug 2014 21:36:38,978 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-696 12 Aug 2014 21:36:38,988 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-697 12 Aug 2014 21:36:38,994 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-698 12 Aug 2014 21:36:39,005 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-699 12 Aug 2014 21:36:39,017 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-700 12 Aug 2014 21:36:39,022 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.ReplayHandler.replayLog:262) - Replaying /local/flume-ng/data/channel1/log-701 12 Aug 2014 21:36:58,554 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623194835 in /local/flume-ng/data/channel1/log-702 12 Aug 2014 21:36:59,311 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623195343 in /local/flume-ng/data/channel1/log-703 12 Aug 2014 21:37:00,637 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623195377 in /local/flume-ng/data/channel1/log-704 12 Aug 2014 21:37:02,575 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623195415 in /local/flume-ng/data/channel1/log-705 12 Aug 2014 21:37:04,328 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623195243 in /local/flume-ng/data/channel1/log-706 12 Aug 2014 21:37:04,460 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623195572 in /local/flume-ng/data/channel1/log-707 12 Aug 2014 21:37:05,668 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623195404 in /local/flume-ng/data/channel1/log-708 12 Aug 2014 21:37:05,710 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623194925 in /local/flume-ng/data/channel1/log-709 12 Aug 2014 21:37:06,347 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 1623194204 in /local/flume-ng/data/channel1/log-710 12 Aug 2014 21:37:07,900 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 712946491 in /local/flume-ng/data/channel1/log-711 12 Aug 2014 21:37:07,908 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$SequentialReader.next:657) - Encountered EOF at 38156 in /local/flume-ng/data/channel1/log-714 12 Aug 2014 21:37:07,945 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.replay:470) - Rolling /local/flume-ng/data/channel1 12 Aug 2014 21:37:07,946 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.roll:932) - Roll start /local/flume-ng/data/channel1 12 Aug 2014 21:37:07,946 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$Writer.<init>:214) - Opened /local/flume-ng/data/channel1/log-715 12 Aug 2014 21:37:07,947 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:214) - Start checkpoint for /local/flume-ng/checkpoints/channel1/checkpoint, elements to sync = 8465 12 Aug 2014 21:37:07,973 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.Log.writeCheckpoint:1005) - Updated checkpoint for file: /local/flume-ng/data/channel1/log-715 position: 0 logWriteOrderID: 1408282684040 12 Aug 2014 21:37:07,973 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-686 12 Aug 2014 21:37:07,979 INFO [lifecycleSupervisor-1-0] (org.apache.flume.channel.file.FileChannel.start:285) - Queue Size after replay: 418 [channel=channel1] 12 Aug 2014 21:37:07,980 INFO [lifecycleSupervisor-1-0] (org.apache.flume.instrumentation.MonitoredCounterGroup.register:119) - Monitored counter group for type: CHANNEL, name: channel1: Successfully registered new MBean. 12 Aug 2014 21:37:07,980 INFO [lifecycleSupervisor-1-0] (org.apache.flume.instrumentation.MonitoredCounterGroup.start:95) - Component type: CHANNEL, name: channel1 started 12 Aug 2014 21:37:08,139 INFO [SinkRunner-PollingRunner-DefaultSinkProcessor] (org.apache.flume.channel.file.LogFile$RandomReader.checkOut:544) - Opening /local/flume-ng/data/channel1/log-686 for read, remaining number of file handles available for reads of this file is 50 12 Aug 2014 21:37:08,139 INFO [SinkRunner-PollingRunner-DefaultSinkProcessor] (org.apache.flume.channel.file.LogFile$RandomReader.checkOut:544) - Opening /local/flume-ng/data/channel1/log-686 for read, remaining number of file handles available for reads of this file is 50 12 Aug 2014 21:37:08,139 INFO [SinkRunner-PollingRunner-DefaultSinkProcessor] (org.apache.flume.channel.file.LogFile$RandomReader.checkOut:544) - Opening /local/flume-ng/data/channel1/log-686 for read, remaining number of file handles available for reads of this file is 50 12 Aug 2014 21:38:18,255 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.EventQueueBackingStoreFile.beginCheckpoint:214) - Start checkpoint for /local/flume-ng/checkpoints/channel1/checkpoint, elements to sync = 418 12 Aug 2014 21:38:18,257 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.EventQueueBackingStoreFile.checkpoint:239) - Updating checkpoint metadata: logWriteOrderID: 1408282684463, queueSize: 0, queueHead: 2970240 12 Aug 2014 21:38:18,260 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.Log.writeCheckpoint:1005) - Updated checkpoint for file: /local/flume-ng/data/channel1/log-715 position: 17282 logWriteOrderID: 1408282684463 12 Aug 2014 21:38:18,261 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-686 12 Aug 2014 21:38:18,266 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-687 12 Aug 2014 21:38:18,271 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-688 12 Aug 2014 21:38:18,277 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-689 12 Aug 2014 21:38:18,282 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-690 12 Aug 2014 21:38:18,287 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-691 12 Aug 2014 21:38:18,293 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-692 12 Aug 2014 21:38:18,298 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-693 12 Aug 2014 21:38:18,303 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-694 12 Aug 2014 21:38:18,309 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-695 12 Aug 2014 21:38:18,314 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-696 12 Aug 2014 21:38:18,319 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-697 12 Aug 2014 21:38:18,325 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-698 12 Aug 2014 21:38:18,330 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-699 12 Aug 2014 21:38:18,335 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-700 12 Aug 2014 21:38:18,340 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-701 12 Aug 2014 21:38:18,346 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-702 12 Aug 2014 21:38:18,351 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-703 12 Aug 2014 21:38:18,356 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-704 12 Aug 2014 21:38:18,362 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-705 12 Aug 2014 21:38:18,367 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-706 12 Aug 2014 21:38:18,372 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-707 12 Aug 2014 21:38:18,378 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-708 12 Aug 2014 21:38:18,383 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-709 12 Aug 2014 21:38:18,388 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-710 12 Aug 2014 21:38:18,394 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-711 12 Aug 2014 21:38:18,399 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-712 12 Aug 2014 21:38:18,404 INFO [Log-BackgroundWorker-channel1] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-713 (source was disabled so nothing interresting futher) and after I realized that flume is not going to remove old files I stopped it: 12 Aug 2014 21:42:20,084 INFO [agent-shutdown-hook] (org.apache.flume.node.Application.stopAllComponents:102) - Shutting down configuration: { sourceRunners:{} sinkRunners:{google-BQ-perf-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@6eb5845c counterGroup:{ name: null counters:{runner.backoffs.consecutive=40, runner.backoffs=40} } }, es-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4f04eccc counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, es-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4c566d9b counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, es-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@3e360244 counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, es-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@4bcede44 counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink4=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@7a62693d counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink3=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@52eb6290 counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink2=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@5b940677 counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }, google-BQ-sink1=SinkRunner: { policy:org.apache.flume.sink.DefaultSinkProcessor@53349d99 counterGroup:{ name: null counters:{runner.backoffs.consecutive=65, runner.backoffs=65} } }} channels:{channel1=FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }, channel2=FileChannel channel2 { dataDirs: [/local/flume-ng/data/channel2] }, channel3=org.apache.flume.channel.MemoryChannel{name: channel3}} } 12 Aug 2014 21:42:20,102 INFO [agent-shutdown-hook] (org.apache.flume.node.Application.stopAllComponents:126) - Stopping Channel channel1 12 Aug 2014 21:42:20,102 INFO [agent-shutdown-hook] (org.apache.flume.lifecycle.LifecycleSupervisor.unsupervise:171) - Stopping component: FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] } 12 Aug 2014 21:42:20,102 INFO [agent-shutdown-hook] (org.apache.flume.channel.file.FileChannel.stop:305) - Stopping FileChannel channel1 { dataDirs: [/local/flume-ng/data/channel1] }... 12 Aug 2014 21:42:20,103 INFO [agent-shutdown-hook] (org.apache.flume.channel.file.LogFile$Writer.close:370) - Closing /local/flume-ng/data/channel1/log-715 12 Aug 2014 21:42:20,103 INFO [agent-shutdown-hook] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-715 12 Aug 2014 21:42:20,108 INFO [agent-shutdown-hook] (org.apache.flume.channel.file.LogFile$RandomReader.close:504) - Closing RandomReader /local/flume-ng/data/channel1/log-714 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:149) - Component type: CHANNEL, name: channel1 stopped 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:155) - Shutdown Metric for type: CHANNEL, name: channel1. channel.start.time == 1407893827980 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:161) - Shutdown Metric for type: CHANNEL, name: channel1. channel.stop.time == 1407894140114 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177) - Shutdown Metric for type: CHANNEL, name: channel1. channel.capacity == 16000000 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177) - Shutdown Metric for type: CHANNEL, name: channel1. channel.current.size == 0 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177) - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.put.attempt == 0 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177) - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.put.success == 0 12 Aug 2014 21:42:20,114 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177) - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.take.attempt == 678 12 Aug 2014 21:42:20,115 INFO [agent-shutdown-hook] (org.apache.flume.instrumentation.MonitoredCounterGroup.stop:177) - Shutdown Metric for type: CHANNEL, name: channel1. channel.event.take.success == 418 No events in channel according to shutdown message: channel1. channel.current.size == 0
          Hide
          jlord Jeff Lord added a comment -

          Should we re-open this issue?
          Not sure if it is still occurring or what.

          Show
          jlord Jeff Lord added a comment - Should we re-open this issue? Not sure if it is still occurring or what.
          Hide
          nino4a Nina Safonova added a comment -

          It is still occurring.

          Show
          nino4a Nina Safonova added a comment - It is still occurring.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Which version of Flume? OS details and also stacktrace/logs?

          Show
          hshreedharan Hari Shreedharan added a comment - Which version of Flume? OS details and also stacktrace/logs?
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Your checkpoint interval is set to 27+ hours (that value is in seconds). Flume deletes files only on every checkpoint. In fact, once the events in a file are removed, it will remove the file in the 2nd checkpoint after that one (not the one immediately after). So in your case, it needs to wait for 54 hours before deleting the file. Even after a replay it waits for another checkpoint to delete the file. Did you wait for that long and verify?

          Show
          hshreedharan Hari Shreedharan added a comment - Your checkpoint interval is set to 27+ hours (that value is in seconds). Flume deletes files only on every checkpoint. In fact, once the events in a file are removed, it will remove the file in the 2nd checkpoint after that one (not the one immediately after). So in your case, it needs to wait for 54 hours before deleting the file. Even after a replay it waits for another checkpoint to delete the file. Did you wait for that long and verify?
          Hide
          hshreedharan Hari Shreedharan added a comment -

          Sorry - my mistake. That is in millis. But you should still account for the 2 checkpoints.

          Show
          hshreedharan Hari Shreedharan added a comment - Sorry - my mistake. That is in millis. But you should still account for the 2 checkpoints.
          Hide
          nino4a Nina Safonova added a comment -

          OS is CentOS 6.5, flume version is 1.5 as I mentioned above, all the log I also posted above.
          I waited log enough and no old files were deleted. This doesn't work that way all the time, from the start to some random moment it's working as expected and cleans old files, but at some point it just stops to do this and at some later point disk ran out of space.

          Show
          nino4a Nina Safonova added a comment - OS is CentOS 6.5, flume version is 1.5 as I mentioned above, all the log I also posted above. I waited log enough and no old files were deleted. This doesn't work that way all the time, from the start to some random moment it's working as expected and cleans old files, but at some point it just stops to do this and at some later point disk ran out of space.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          This does not seem like it is due to log timeout. Can you delete all of the checkpoint files (all files) and force a full replay?

          Did you delete the files before or after stopping the agent?

          Show
          hshreedharan Hari Shreedharan added a comment - This does not seem like it is due to log timeout. Can you delete all of the checkpoint files (all files) and force a full replay? Did you delete the files before or after stopping the agent?
          Hide
          nino4a Nina Safonova added a comment -

          I didn't try to delete checkpoint files.
          To keep processing data I deleted all the files (data and checkpoints) after I stopped the agent.

          Show
          nino4a Nina Safonova added a comment - I didn't try to delete checkpoint files. To keep processing data I deleted all the files (data and checkpoints) after I stopped the agent.
          Hide
          hshreedharan Hari Shreedharan added a comment -

          From your logs, I don't see the actual cause of this issue. Did you also delete the inflight* files too?

          Deleting the data files and keeping any of the checkpoint files can cause unpredictable issues.

          Show
          hshreedharan Hari Shreedharan added a comment - From your logs, I don't see the actual cause of this issue. Did you also delete the inflight* files too? Deleting the data files and keeping any of the checkpoint files can cause unpredictable issues.
          Hide
          nino4a Nina Safonova added a comment -

          I didn't delete any files before I start to experience this issue. After I ran out of disk space I tried to restart the agent. When it didn't help to clean old files I stopped the agent and deleted all the files manually.

          Show
          nino4a Nina Safonova added a comment - I didn't delete any files before I start to experience this issue. After I ran out of disk space I tried to restart the agent. When it didn't help to clean old files I stopped the agent and deleted all the files manually.
          Hide
          hshreedharan Hari Shreedharan added a comment - - edited

          From the logs you have here, I can't really see what the issue is. What is the exact version of Flume you are running?

          flume-ng version can give you that info.

          Show
          hshreedharan Hari Shreedharan added a comment - - edited From the logs you have here, I can't really see what the issue is. What is the exact version of Flume you are running? flume-ng version can give you that info.
          Hide
          nino4a Nina Safonova added a comment -

          Flume 1.5.0-cdh5.1.0
          Source code repository: https://git-wip-us.apache.org/repos/asf/flume.git
          Revision: 14be91ec816bac5a91c321b9e8620ffb04acf04c
          Compiled by jenkins on Sat Jul 12 09:17:48 PDT 2014
          From source with checksum bf4451b17198a612fea60ad6f5420bbc

          Show
          nino4a Nina Safonova added a comment - Flume 1.5.0-cdh5.1.0 Source code repository: https://git-wip-us.apache.org/repos/asf/flume.git Revision: 14be91ec816bac5a91c321b9e8620ffb04acf04c Compiled by jenkins on Sat Jul 12 09:17:48 PDT 2014 From source with checksum bf4451b17198a612fea60ad6f5420bbc
          Hide
          tezra tzachi added a comment -

          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).

          Show
          tezra tzachi added a comment - 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).

            People

            • Assignee:
              hshreedharan Hari Shreedharan
              Reporter:
              szesch Steve Zesch
            • Votes:
              0 Vote for this issue
              Watchers:
              11 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development