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

li xiang commented on FLUME-1850:
---------------------------------

Hi Juhani, Mike and all, I encountered an out-of-memory issue, I am not sure it 
is a HDFS sink issue like this one, or it is a memory channel issue, as 
FLUME-2197, how can I determine ? Thanks
1. Exception
30 Jul 2014 23:10:06,984 ERROR [SinkRunner-PollingRunner-DefaultSinkProcessor] 
(org.apache.flume.sink.hdfs.HDFSEventSink.process:460) - process failed
java.lang.OutOfMemoryError: Java heap space
at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:239)
at java.util.concurrent.FutureTask.get(FutureTask.java:102)
at 
org.apache.flume.sink.hdfs.HDFSEventSink.callWithTimeout(HDFSEventSink.java:345)
at org.apache.flume.sink.hdfs.HDFSEventSink.append(HDFSEventSink.java:727)
at org.apache.flume.sink.hdfs.HDFSEventSink.process(HDFSEventSink.java:430)
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:738)
30 Jul 2014 23:10:06,981 WARN [pool-13-thread-4] 
(org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.exceptionCaught:201) - 
Unexpected exception from downstream.
java.lang.OutOfMemoryError: Java heap space
at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:50)
at java.nio.ByteBuffer.allocate(ByteBuffer.java:323)
at 
org.apache.avro.ipc.NettyTransportCodec$NettyFrameDecoder.decodePackBody(NettyTransportCodec.java:192)
at 
org.apache.avro.ipc.NettyTransportCodec$NettyFrameDecoder.decode(NettyTransportCodec.java:144)
at 
org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:286)
at 
org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:220)
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 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:897)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:919)
at java.lang.Thread.run(Thread.java:738)

2. IBM HeapAnalyzer shows the leak suspect as : 
A Linked List Data Structure Detected.
8,426,896,520 bytes (99.14 %) of Java heap is used by 931,705 instances of 
java/util/concurrent/LinkedBlockingDeque$Node
Contains the following object:
931,705 instances of org/apache/flume/event/SimpleEvent holding 8,382,174,632 
bytes

3. The reference tree is attached as HA_result.jpg, also by HeapAnalyzer

4. Javacore
3XMTHREADINFO3 Java callstack:
4XESTACKTRACE at java/lang/Throwable.fillInStackTrace(Native Method)
4XESTACKTRACE at java/lang/Throwable.<init>(Throwable.java:56(Compiled Code))
4XESTACKTRACE at java/lang/Throwable.<init>(Throwable.java:67)
4XESTACKTRACE at java/lang/OutOfMemoryError.<init>(OutOfMemoryError.java:46)
4XESTACKTRACE at 
java/util/concurrent/locks/AbstractQueuedSynchronizer$ConditionObject.addConditionWaiter(AbstractQueuedSynchronizer.java:1812(Compiled
 Code))
4XESTACKTRACE at 
java/util/concurrent/locks/AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2027(Compiled
 Code))
4XESTACKTRACE at 
java/util/concurrent/DelayQueue.take(DelayQueue.java:175(Compiled Code))
4XESTACKTRACE at 
java/util/concurrent/ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:620(Compiled
 Code))
4XESTACKTRACE at 
java/util/concurrent/ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:613(Compiled
 Code))
4XESTACKTRACE at 
java/util/concurrent/ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:958(Compiled
 Code))
4XESTACKTRACE at 
java/util/concurrent/ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
4XESTACKTRACE at java/lang/Thread.run(Thread.java:738)

> HDFSSink lower hdfs.maxOpenFiles to 100
> ---------------------------------------
>
>                 Key: FLUME-1850
>                 URL: https://issues.apache.org/jira/browse/FLUME-1850
>             Project: Flume
>          Issue Type: Bug
>          Components: Node
>    Affects Versions: v1.3.0
>         Environment: RHEL 6
>            Reporter: Mohit Anchlia
>         Attachments: HA_result.jpg, Screen Shot 2013-01-16 at 11.05.55 
> PM.png, flume-oo.docx
>
>
> We are using flume-1.3.0. After flume is up for a while (30 days+) we get 
> OutOfMemory error. Our heap is set to 2G and load on the system is very low. 
> Around 50 request/minute. We use AvroClient and long lived connection.
> Below is the stack trace. I don't have the heap dump but I plan to enable 
> that for next time.
> 13/01/16 09:09:38 ERROR hdfs.HDFSEventSink: process failed
> java.lang.OutOfMemoryError: Java heap space
>         at java.util.Arrays.copyOf(Arrays.java:2786)
>         at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:94)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         at org.apache.hadoop.io.Text.write(Text.java:282)
>         at 
> org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:90)
>         at 
> org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:77)
>         at 
> org.apache.hadoop.io.SequenceFile$BlockCompressWriter.append(SequenceFile.java:1320)
>         at 
> org.apache.flume.sink.hdfs.HDFSSequenceFile.append(HDFSSequenceFile.java:72)
>         at 
> org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:376)
>         at 
> org.apache.flume.sink.hdfs.HDFSEventSink$2.call(HDFSEventSink.java:729)
>         at 
> org.apache.flume.sink.hdfs.HDFSEventSink$2.call(HDFSEventSink.java:727)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>         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:662)
> Exception in thread "SinkRunner-PollingRunner-DefaultSinkProcessor" 
> java.lang.OutOfMemoryError: Java heap space
>         at java.util.Arrays.copyOf(Arrays.java:2786)
>         at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:94)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         at org.apache.hadoop.io.Text.write(Text.java:282)
>         at 
> org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:90)
>         at 
> org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:77)
>         at 
> org.apache.hadoop.io.SequenceFile$BlockCompressWriter.append(SequenceFile.java:1320)
>         at 
> org.apache.flume.sink.hdfs.HDFSSequenceFile.append(HDFSSequenceFile.java:72)
>         at 
> org.apache.flume.sink.hdfs.BucketWriter.append(BucketWriter.java:376)
>         at 
> org.apache.flume.sink.hdfs.HDFSEventSink$2.call(HDFSEventSink.java:729)
>         at 
> org.apache.flume.sink.hdfs.HDFSEventSink$2.call(HDFSEventSink.java:727)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>         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:662)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to