Details

    • Type: Bug Bug
    • Status: Patch Available
    • Priority: Critical Critical
    • Resolution: Unresolved
    • Affects Version/s: v1.2.0, v1.3.0, v1.4.0
    • Fix Version/s: None
    • Component/s: None
    • Release Note:
      HDFS Sink memory usage best practices.

      Description

      We run a 3 node/1 collector test cluster pushing about 350events/sec per node... Not really high stress, but just something to evaluate flume with.

      Consistently our collector has been dying because of an OOMError killing the SinkRunner after running for about 30-40 hours(seems pretty consistent as we've had it 3 times now).

      Suspected cause would be a memory leak somewhere in HdfsSink. The feeder nodes which run AvroSink instead of HdfsSink have been up and running for about a week without restarts.

      flume-load/act-wap02/2012-06-26-17.1340697637324.tmp, packetSize=65557, chunksPerPacket=127, bytesCurBlock=29731328
      java.lang.OutOfMemoryError: GC overhead limit exceeded
      2012-06-26 17:12:56,080 (SinkRunner-PollingRunner-DefaultSinkProcessor) [ERROR - org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:411)] process failed
      java.lang.OutOfMemoryError: GC overhead limit exceeded
      at java.util.Arrays.copyOfRange(Arrays.java:3209)
      at java.lang.String.<init>(String.java:215)
      at java.lang.StringBuilder.toString(StringBuilder.java:430)
      at org.apache.flume.formatter.output.BucketPath.escapeString(BucketPath.java:306)
      at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:367)
      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)
      Exception in thread "SinkRunner-PollingRunner-DefaultSinkProcessor" java.lang.OutOfMemoryError: GC overhead limit exceeded
      at java.util.Arrays.copyOfRange(Arrays.java:3209)
      at java.lang.String.<init>(String.java:215)
      at java.lang.StringBuilder.toString(StringBuilder.java:430)
      at org.apache.flume.formatter.output.BucketPath.escapeString(BucketPath.java:306)
      at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:367)
      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)

      1. FLUME-1326.patch
        1 kB
        dave sinclair

        Activity

        Hide
        li xiang added a comment -

        Hi Dave, does it also apply to the file roll? The file roll(controlled by hdfs.rollSize, hdfs.rollInterval and hdfs.rollCount) can also make a lot of HDF file generated. Can it also lead to OOM?

        Show
        li xiang added a comment - Hi Dave, does it also apply to the file roll? The file roll(controlled by hdfs.rollSize, hdfs.rollInterval and hdfs.rollCount) can also make a lot of HDF file generated. Can it also lead to OOM?
        Hide
        li xiang added a comment -

        Hi Dave, does it also apply to the file roll? The file roll(controlled by hdfs.rollSize, hdfs.rollInterval and hdfs.rollCount) can also make a lot of HDF file generated. Can it also lead to OOM?

        Show
        li xiang added a comment - Hi Dave, does it also apply to the file roll? The file roll(controlled by hdfs.rollSize, hdfs.rollInterval and hdfs.rollCount) can also make a lot of HDF file generated. Can it also lead to OOM?
        Hide
        dave sinclair added a comment -

        Sorry, been meaning to post this patch. Let me know what you think Hari.

        thanks

        Show
        dave sinclair added a comment - Sorry, been meaning to post this patch. Let me know what you think Hari. thanks
        Hide
        Hari Shreedharan added a comment -

        Makes sense. Do you want to submit a patch?

        Show
        Hari Shreedharan added a comment - Makes sense. Do you want to submit a patch?
        Hide
        dave sinclair added a comment -

        Hari,

        I think the documentation may want to include a info/warning section highlighting the potential memory issue if you have a high number of HDFS Sinks on a node, or the filename/pathname changes often that causes a high number of bucket writers to be created.

        Show
        dave sinclair added a comment - Hari, I think the documentation may want to include a info/warning section highlighting the potential memory issue if you have a high number of HDFS Sinks on a node, or the filename/pathname changes often that causes a high number of bucket writers to be created.
        Hide
        Hari Shreedharan added a comment -

        As I mentioned on the list:

        This is because like you said you have too many files open at the same time. HDFS stream classes keep a pretty large buffer (this is HDFS client code not Flume) which will be cleaned up when the file is closed. Keeping maxOpenFiles to a smaller number is a good way to handle this.

        Show
        Hari Shreedharan added a comment - As I mentioned on the list: This is because like you said you have too many files open at the same time. HDFS stream classes keep a pretty large buffer (this is HDFS client code not Flume) which will be cleaned up when the file is closed. Keeping maxOpenFiles to a smaller number is a good way to handle this.
        Hide
        dave sinclair added a comment -

        I have determined the cause of the memory leak, at least for my usage, to be the WriterLinkedHashMap sfWriters. Depending on how you have your path and filename specified, this can continue to grow until it hits the max number of open files or run out of memory. The way I have gotten around it is by setting an idleTimeout so idle writers are retired (could also lower the number of open files).

        My path was creating a new set of writers every hour, and like clock-work the heap would increase on the hour and never release that memory. A heap dump and subsequent inspection led me to the map of the writers.

        Let me know if additional details are needed.

        dave

        Show
        dave sinclair added a comment - I have determined the cause of the memory leak, at least for my usage, to be the WriterLinkedHashMap sfWriters. Depending on how you have your path and filename specified, this can continue to grow until it hits the max number of open files or run out of memory. The way I have gotten around it is by setting an idleTimeout so idle writers are retired (could also lower the number of open files). My path was creating a new set of writers every hour, and like clock-work the heap would increase on the hour and never release that memory. A heap dump and subsequent inspection led me to the map of the writers. Let me know if additional details are needed. dave
        Hide
        Juhani Connolly added a comment -

        Go for it! If there's anything I can help with let me know, it would be nice to see this fixed. I've tried looking through our heap dumps but with the huge amount of objects(1800 gc roots!) active at any time it has been difficult to track down what is going in

        Show
        Juhani Connolly added a comment - Go for it! If there's anything I can help with let me know, it would be nice to see this fixed. I've tried looking through our heap dumps but with the huge amount of objects(1800 gc roots!) active at any time it has been difficult to track down what is going in
        Hide
        NO NAME added a comment -

        I'm going to try and dig into this in a test environment - if someone is already working on this please let me know so I don't duplicate effort.

        Show
        NO NAME added a comment - I'm going to try and dig into this in a test environment - if someone is already working on this please let me know so I don't duplicate effort.
        Hide
        Juhani Connolly added a comment - - edited

        Just as an update, with 40mb memory allocated, it took significantly longer to oom, but it still happened. Had meant to attached gc logs, but succeeded in overwriting them when restarting flume in a moment of inattention.

        Show
        Juhani Connolly added a comment - - edited Just as an update, with 40mb memory allocated, it took significantly longer to oom, but it still happened. Had meant to attached gc logs, but succeeded in overwriting them when restarting flume in a moment of inattention.
        Hide
        Juhani Connolly added a comment -

        Set up gc logging and got dumps of a fresh run running instance and one at oom, I just realized I can't really post them because they contain internal server logs, sorry _

        As far as the content is concerned though, it looks like about 5mb is retained by MemoryChannel, virtually all of that in the msg queue. The rest seems to be spread out, but most buffers seem to be attached to various Buffers and their owning streams

        It's possible that we're just legitimately running out of memory as various buffers happen to get full up, and with a default sized memory channel this wouldn't happen, I'm going to try running for a while on 40 mb without changing other stuff, and keep an eye on the gc logs, see what happens

        Show
        Juhani Connolly added a comment - Set up gc logging and got dumps of a fresh run running instance and one at oom, I just realized I can't really post them because they contain internal server logs, sorry _ As far as the content is concerned though, it looks like about 5mb is retained by MemoryChannel, virtually all of that in the msg queue. The rest seems to be spread out, but most buffers seem to be attached to various Buffers and their owning streams It's possible that we're just legitimately running out of memory as various buffers happen to get full up, and with a default sized memory channel this wouldn't happen, I'm going to try running for a while on 40 mb without changing other stuff, and keep an eye on the gc logs, see what happens
        Hide
        Juhani Connolly added a comment -

        I'll set that up when time permits. And yeah, with a 20gb heap it'll take a long time to get an error seeing as it took over 24 hours with 20mb

        Show
        Juhani Connolly added a comment - I'll set that up when time permits. And yeah, with a 20gb heap it'll take a long time to get an error seeing as it took over 24 hours with 20mb
        Hide
        Mubarak Seyed added a comment -

        @Juhani
        Do you have GC logging enabled? If no, can you please enable it? We have been running load test for a month using HDFSEventSink, we haven't encountered OOM from HDFSEventSink (but our min/max heap is 20G). One other easy way is if you enable JMX then connect to JVM using VisualVM, you can check the objects counts over GC.

        Show
        Mubarak Seyed added a comment - @Juhani Do you have GC logging enabled? If no, can you please enable it? We have been running load test for a month using HDFSEventSink, we haven't encountered OOM from HDFSEventSink (but our min/max heap is 20G). One other easy way is if you enable JMX then connect to JVM using VisualVM, you can check the objects counts over GC.
        Hide
        Juhani Connolly added a comment -

        Unfortunately I had the heap dump writing to a a directory the running user didn't have permissions to, and there's some weirdness with the jdk packages we use(jmap doesn't seem to play nice)

        I'll put up the heap dump spit out with HeapDumpOnOutOfMemory as soon as it dies again

        Show
        Juhani Connolly added a comment - Unfortunately I had the heap dump writing to a a directory the running user didn't have permissions to, and there's some weirdness with the jdk packages we use(jmap doesn't seem to play nice) I'll put up the heap dump spit out with HeapDumpOnOutOfMemory as soon as it dies again
        Hide
        Mubarak Seyed added a comment -

        @Juhani,
        Do you mind to post the heap dump?

        Show
        Mubarak Seyed added a comment - @Juhani, Do you mind to post the heap dump?
        Hide
        Juhani Connolly added a comment -

        We still run the vm with the standard 20M limit.

        conf:

        receiver1.channels.ch1.type = memory
        receiver1.channels.ch1.capacity = 5000
        receiver1.channels.ch1.transactionCapacity = 500
        receiver1.channels.ch1.checkpointDir = /tmp/flume-check
        receiver1.channels.ch1.dataDirs = /tmp/flume-data

        receiver1.sources.avro.type = avro
        receiver1.sources.avro.bind = <address>
        receiver1.sources.avro.port = 41414
        receiver1.sources.avro.channels = ch1

        receiver1.sinks.hdfs-sink.type = hdfs
        receiver1.sinks.hdfs-sink.namenode = hdfs://pat-nn01/
        receiver1.sinks.hdfs-sink.hdfs.path = hdfs://pat-nn01/tmp/flume-load/%

        {host}

        receiver1.sinks.hdfs-sink.hdfs.writeFormat = text
        receiver1.sinks.hdfs-sink.hdfs.fileType = SequenceFile
        receiver1.sinks.hdfs-sink.channel = ch1
        receiver1.sinks.hdfs-sink.runner.type = polling
        receiver1.sinks.hdfs-sink.runner.polling.interval = 60
        receiver1.sinks.hdfs-sink.hdfs.rollSize = 0
        receiver1.sinks.hdfs-sink.hdfs.rollInterval = 4000
        receiver1.sinks.hdfs-sink.hdfs.filePrefix = %Y-%m-%d-%H
        receiver1.sinks.hdfs-sink.hdfs.rollCount = 0
        receiver1.sinks.hdfs-sink.hdfs.batchSize = 200

        receiver1.sources = avro
        receiver1.sinks = hdfs-sink
        receiver1.channels = ch1

        Show
        Juhani Connolly added a comment - We still run the vm with the standard 20M limit. conf: receiver1.channels.ch1.type = memory receiver1.channels.ch1.capacity = 5000 receiver1.channels.ch1.transactionCapacity = 500 receiver1.channels.ch1.checkpointDir = /tmp/flume-check receiver1.channels.ch1.dataDirs = /tmp/flume-data receiver1.sources.avro.type = avro receiver1.sources.avro.bind = <address> receiver1.sources.avro.port = 41414 receiver1.sources.avro.channels = ch1 receiver1.sinks.hdfs-sink.type = hdfs receiver1.sinks.hdfs-sink.namenode = hdfs://pat-nn01/ receiver1.sinks.hdfs-sink.hdfs.path = hdfs://pat-nn01/tmp/flume-load/% {host} receiver1.sinks.hdfs-sink.hdfs.writeFormat = text receiver1.sinks.hdfs-sink.hdfs.fileType = SequenceFile receiver1.sinks.hdfs-sink.channel = ch1 receiver1.sinks.hdfs-sink.runner.type = polling receiver1.sinks.hdfs-sink.runner.polling.interval = 60 receiver1.sinks.hdfs-sink.hdfs.rollSize = 0 receiver1.sinks.hdfs-sink.hdfs.rollInterval = 4000 receiver1.sinks.hdfs-sink.hdfs.filePrefix = %Y-%m-%d-%H receiver1.sinks.hdfs-sink.hdfs.rollCount = 0 receiver1.sinks.hdfs-sink.hdfs.batchSize = 200 receiver1.sources = avro receiver1.sinks = hdfs-sink receiver1.channels = ch1

          People

          • Assignee:
            Unassigned
            Reporter:
            Juhani Connolly
          • Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

            • Created:
              Updated:

              Development