Avro
  1. Avro
  2. AVRO-1111

Malformed data can cause OutOfMemoryError in Avro IPC

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 1.6.3
    • Fix Version/s: 1.7.2
    • Component/s: java
    • Labels:
      None
    • Hadoop Flags:
      Reviewed

      Description

      If the data that comes in through the Netty channel buffer is not framed correctly/is not valid Avro data, then the incoming data can cause arbitrarily large array lists to be created, causing OutOfMemoryError.

      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;
      }

      If the buffer does not have valid Avro data, the listSize variable can have arbitrary values, causing massive ArrayLists to be created, leading to OutOfMemoryErrors.

      1. AVRO-1111-2.patch
        3 kB
        Mike Percy
      2. AVRO-1111-1.patch
        3 kB
        Mike Percy

        Issue Links

          Activity

          Brock Noland made changes -
          Link This issue is related to AVRO-1226 [ AVRO-1226 ]
          Doug Cutting made changes -
          Status Resolved [ 5 ] Closed [ 6 ]
          Tom White made changes -
          Status Patch Available [ 10002 ] Resolved [ 5 ]
          Hadoop Flags Reviewed [ 10343 ]
          Assignee Mike Percy [ mpercy ]
          Fix Version/s 1.7.2 [ 12322476 ]
          Resolution Fixed [ 1 ]
          Mike Percy made changes -
          Attachment AVRO-1111-2.patch [ 12544608 ]
          Mike Percy made changes -
          Status Open [ 1 ] Patch Available [ 10002 ]
          Mike Percy made changes -
          Attachment AVRO-1111-1.patch [ 12544408 ]
          Hari Shreedharan made changes -
          Description If an the data the comes in through the Netty channel buffer is not framed correctly, then the incoming data can cause arbitrarily large array lists to be created, causing OutOfMemoryError.

          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;
          }

          If the buffer does not have valid Avro data, the listSize variable can have arbitrary values, causing massive ArrayLists to be created, leading to OutOfMemoryErrors.
          If the data that comes in through the Netty channel buffer is not framed correctly/is not valid Avro data, then the incoming data can cause arbitrarily large array lists to be created, causing OutOfMemoryError.

          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;
          }

          If the buffer does not have valid Avro data, the listSize variable can have arbitrary values, causing massive ArrayLists to be created, leading to OutOfMemoryErrors.
          Hari Shreedharan made changes -
          Description If an the data the comes in through the Netty channel buffer is not framed correctly, then the incoming data can cause arbitrarily large array lists to be created, causing OutOfMemoryError.

          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;
          }

          If the variable - buffer does not have valid Avro data, the listSize variable can have arbitrary values, causing massive ArrayLists to be created, leading to OutOfMemoryErrors.
          If an the data the comes in through the Netty channel buffer is not framed correctly, then the incoming data can cause arbitrarily large array lists to be created, causing OutOfMemoryError.

          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;
          }

          If the buffer does not have valid Avro data, the listSize variable can have arbitrary values, causing massive ArrayLists to be created, leading to OutOfMemoryErrors.
          Hari Shreedharan made changes -
          Field Original Value New Value
          Link This issue blocks FLUME-1259 [ FLUME-1259 ]
          Hari Shreedharan created issue -

            People

            • Assignee:
              Mike Percy
              Reporter:
              Hari Shreedharan
            • Votes:
              0 Vote for this issue
              Watchers:
              9 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development