Uploaded image for project: 'Apache Avro'
  1. Apache Avro
  2. AVRO-1696

Handshake request is not handled causing an OutOfMemoryError

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 1.7.4, 1.8.0
    • None
    • java
    • None

    Description

      A handshake request is not read because connection.isConnected() == true here:

      https://github.com/apache/avro/blob/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java#L208

      This should be an invalid state, but we see this happening in production which causes the agent to have OOM errors. Seems to happen when there are rapid client disconnects and connects or connection reset by peer and broken pipe errors.

      Here is the base64 encoded form of buffers.get(0).array() when this occurs:

      hqra4sRUdMD+k//Q8jUKZQCGqtrixFR0wP6T/9DyNQplAgAADGFwcGVuZA==
      

      which appears to be the serialized handshake info.

      The handshake is not read because this code is skipped:

      https://github.com/apache/avro/blob/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java#L210

      When not read by the handshakeReader, the BinaryDecoder's position in the internal byte array is not advanced (to 19).

      Causing this code:

      https://github.com/apache/avro/blob/trunk/lang/java/ipc/src/main/java/org/apache/avro/ipc/Responder.java#L124

      to use index 0 of the handshake byte array as the size of the map it is expecting to deserialize.

      At this point:

      https://github.com/apache/avro/blob/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java#L308

      The size of the hash map is determined to be 1452339317379, or 640371331 when cast to an int here:

      https://github.com/apache/avro/blob/trunk/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java#L311

      and we get an OOM error:

      2015-06-29 15:30:24,590 (New I/O worker #4) [WARN - org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.exceptionCaught(NettyServer.java:201)] Unexpected exception from downstream.
      java.lang.OutOfMemoryError: Java heap space
              at java.util.HashMap.<init>(HashMap.java:187)
              at java.util.HashMap.<init>(HashMap.java:199)
              at org.apache.avro.generic.GenericDatumReader.newMap(GenericDatumReader.java:330)
              at org.apache.avro.generic.GenericDatumReader.readMap(GenericDatumReader.java:239)
              at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:151)
              at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:139)
              at org.apache.avro.ipc.Responder.respond(Responder.java:124)
              at org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.messageReceived(NettyServer.java:188)
              at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
              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:791)
              at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:462)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:443)
              at org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303)
              at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
              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:88)
              at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108)
              at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
              at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89)
              at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
              at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
              at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
              at java.lang.Thread.run(Thread.java:695)
      

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              wildsheep Masanobu Horiyama
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated: