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

Hari Shreedharan commented on FLUME-1259:
-----------------------------------------

Hi Will,

This seems to be because the Avro API tries to read from the buffer with the 
incoming data, and using some value(it expects an int at a certain location in 
the buffer) in that the create another buffer. Since the incoming buffer itself 
is malformed, the int read from the buffer is arbitrarily large - and it tries 
to create an array list of that size, and failing. This happens well before our 
code is even called, so I doubt there is anything we can do immediately from 
our side. If someone volunteers to submit a patch for Avro to fix this, then it 
would be awesome. The relevant code(org.apache.avro.ipc.NettyTransportCodec):


    private boolean decodePackHeader(ChannelHandlerContext ctx, Channel channel,
        ChannelBuffer buffer) throws Exception {
      if (buffer.readableBytes()<8) {
        return false;
      }

      int serial = buffer.readInt();
      listSize = buffer.readInt();
      dataPack = new NettyDataPack(serial, new ArrayList<ByteBuffer>(listSize));
      return true;
    }

I ran the exact case you described through a debugger and end up seeing 
arbitrarily large values for listSize(like 1862953079,1919224434,1684564842 
etc). This is what causes the gc issues.
                
> Flume throws OutOfMemory error when sending data from netcat to avro source 
> (negative test case)
> ------------------------------------------------------------------------------------------------
>
>                 Key: FLUME-1259
>                 URL: https://issues.apache.org/jira/browse/FLUME-1259
>             Project: Flume
>          Issue Type: Bug
>          Components: Sinks+Sources
>    Affects Versions: v1.2.0
>         Environment: RHEL 6.2 64-bit
>            Reporter: Will McQueen
>            Assignee: Arvind Prabhakar
>            Priority: Minor
>             Fix For: v1.2.0
>
>
> This is a negative test case.
> I mistakenly sent data from netcat to an avro source, and Flume through an 
> OutOfMemory error. I sent just 5 one-character events using interactive 
> netcat (nc localhost 41414) and the following config file:
> agent.channels = c1
> agent.sources = r1
> agent.sinks = k1
> #
> agent.channels.c1.type = MEMORY
> #
> agent.sources.r1.channels = c1
> agent.sources.r1.type = AVRO
> agent.sources.r1.bind = 0.0.0.0
> agent.sources.r1.port = 41414
> #
> agent.sinks.k1.channel = c1
> agent.sinks.k1.type = LOGGER
> Here's the exception:
> 2012-06-05 13:51:36,622 INFO source.AvroSource: Avro source 
> starting:AvroSource: { bindAddress:0.0.0.0 port:41414 }
> 2012-06-05 13:51:36,852 DEBUG source.AvroSource: Avro source started
> 2012-06-05 13:51:37,395 INFO ipc.NettyServer: [id: 0x0b07f45d, 
> /0:0:0:0:0:0:0:1:49091 => /0:0:0:0:0:0:0:1:41414] OPEN
> 2012-06-05 13:51:37,399 INFO ipc.NettyServer: [id: 0x0b07f45d, 
> /0:0:0:0:0:0:0:1:49091 => /0:0:0:0:0:0:0:1:41414] BOUND: 
> /0:0:0:0:0:0:0:1:41414
> 2012-06-05 13:51:37,399 INFO ipc.NettyServer: [id: 0x0b07f45d, 
> /0:0:0:0:0:0:0:1:49091 => /0:0:0:0:0:0:0:1:41414] CONNECTED: 
> /0:0:0:0:0:0:0:1:49091
> 2012-06-05 13:52:06,622 DEBUG properties.PropertiesFileConfigurationProvider: 
> Checking file:/etc/flume-ng/conf/flume.conf for changes
> 2012-06-05 13:52:36,623 DEBUG properties.PropertiesFileConfigurationProvider: 
> Checking file:/etc/flume-ng/conf/flume.conf for changes
> 2012-06-05 13:52:56,468 WARN ipc.NettyServer: Unexpected exception from 
> downstream.
> java.lang.OutOfMemoryError: Java heap space
>       at java.util.ArrayList.<init>(ArrayList.java:112)
>       at 
> org.apache.avro.ipc.NettyTransportCodec$NettyFrameDecoder.decodePackHeader(NettyTransportCodec.java:154)
>       at 
> org.apache.avro.ipc.NettyTransportCodec$NettyFrameDecoder.decode(NettyTransportCodec.java:131)
>       at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:282)
>       at 
> org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:214)
>       at 
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:274)
>       at 
> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:261)
>       at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:351)
>       at 
> org.jboss.netty.channel.socket.nio.NioWorker.processSelectedKeys(NioWorker.java:282)
>       at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:202)
>       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)
> 2012-06-05 13:52:56,470 INFO ipc.NettyServer: [id: 0x0b07f45d, 
> /0:0:0:0:0:0:0:1:49091 :> /0:0:0:0:0:0:0:1:41414] DISCONNECTED
> 2012-06-05 13:52:56,470 INFO ipc.NettyServer: [id: 0x0b07f45d, 
> /0:0:0:0:0:0:0:1:49091 :> /0:0:0:0:0:0:0:1:41414] UNBOUND
> 2012-06-05 13:52:56,471 INFO ipc.NettyServer: [id: 0x0b07f45d, 
> /0:0:0:0:0:0:0:1:49091 :> /0:0:0:0:0:0:0:1:41414] CLOSED
> I'm wondering if there's some kind of validation of incoming avro format that 
> we can do that would prevent this error.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to