AVRO-4247: Enforce decompression size limits#3745
AVRO-4247: Enforce decompression size limits#3745steveloughran wants to merge 7 commits intoapache:mainfrom
Conversation
…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
….java Thanks! Co-authored-by: Martin Grigorov <martin-g@users.noreply.github.com>
….java Co-authored-by: Martin Grigorov <martin-g@users.noreply.github.com>
- Move MAX_DECOMPRESS_LENGTH initialization to static block (read once at class load) - Add WARNING log for invalid property values (NumberFormatException) - Validate negative and zero values, reject with warning - Add "(bytes)" to error message for clarity - Add quotes around property name in error message Test command: java -Xmx64m -Dorg.apache.avro.limits.decompress.maxLength=1048576 \ -jar avro-tools-1.13.0-SNAPSHOT.jar tojson poc.avro Expected behavior: Exception in thread "main" org.apache.avro.AvroRuntimeException: Decompressed size 1056768 (bytes) exceeds maximum allowed size 1048576. This can be configured by setting the system property 'org.apache.avro.limits.decompress.maxLength'
Change-Id: Ib24c52cdf3234a3805628041946b229b221383ad
* Automatically available to all codecs * Does need an explicit constructor with no limit, used in DataFileWriter * No tests, though that new constructor makes it trivial Note: merged in main as DataFileWriter changes would otherwise stop merging Change-Id: Ifc5b8921a00425df331a4889472b3e78c6677bde
| private static final long MAX_DECOMPRESS_LENGTH; | ||
|
|
||
| static { | ||
| String prop = System.getProperty(MAX_DECOMPRESS_LENGTH_PROPERTY); |
There was a problem hiding this comment.
could move to SystemLimitException, as that's where the int equivalent lives.
There was a problem hiding this comment.
Yes, MAX_DECOMPRESS_LENGTH_PROPERTY makes more sense in the SystemLimitException class
| * @throws IllegalArgumentException if size is negative | ||
| */ | ||
| public NonCopyingByteArrayOutputStream(int size) { | ||
| this(size, MAX_DECOMPRESS_LENGTH); |
There was a problem hiding this comment.
this does change the default operation. Apart from DataFileWriter, it is only ever used in decompressors.
Options
- change the default (here)
- change the code uses to take a limit
- private two arg ctor and a public static creator method
There was a problem hiding this comment.
Thanks @steveloughran!
The changes look good but we need to add tests that validate that the fix works, including:
- Compressing data whose compressed output exceeds a small configured decompression limit.
- Decompressing data exactly equal to the limit.
There is also another issue. the trevni module has the same no enforced decompression size limits check issue so we need to get a copy of NonCopyingByteArrayOutputStream working on that module. I suggest a copy because trevni does not depend on avro core by design it is independent.
I was also wondering if we should be extra cautions and add some test in TestAllCodecs that ensure that all compression types are covered but maybe that's too much.
| private static final long MAX_DECOMPRESS_LENGTH; | ||
|
|
||
| static { | ||
| String prop = System.getProperty(MAX_DECOMPRESS_LENGTH_PROPERTY); |
There was a problem hiding this comment.
Yes, MAX_DECOMPRESS_LENGTH_PROPERTY makes more sense in the SystemLimitException class
| * @param size buffer capacity | ||
| * @param limit size limit or -1 for no limit. | ||
| */ | ||
| public NonCopyingByteArrayOutputStream(final int size, final long limit) { |
There was a problem hiding this comment.
make it private, since nobody is using it, it is probably better not to encourage the use.
There was a problem hiding this comment.
i'll go for package private and a test in this module with a tiny limit and verify that all write ops get rejected.
* Tests * Option read moved to SystemLimitException Change-Id: I01d4203ad65e0e09dde88b33f603879323b06425
|
@iemejia ok, pushed out test and the move of the property read. If you are happy with this, I can move a copy of the stream + test + exception into the trevni module. |
| * parsable as an int | ||
| * @return The value from the system property | ||
| */ | ||
| private static long getLongLimitFromProperty(String property, long defaultValue) { |
| * parsable as an int | ||
| * @return The value from the system property | ||
| */ | ||
| private static long getLongLimitFromProperty(String property, long defaultValue) { |
|
Thanks for taking this up! The bot made a good comment about parameters, simple mistake to fix then this LGTM. This needs to be cherry-picked to 1.12.2 after merge of course. Also thanks for keeping the contributor history ❤️ Good collaboration! |
Thanks a lot @steveloughran there is an issue I did not notice during the previous review :S the Another doubt I have is: |
|
the DataFileWriter was why my initial PR had the -1 for no limits option visible. If we make that visible and call it from there: no problem. I will look at SnappyCodec |
|
Ok my dear Claude says this (which I don't know if it should gives us confidence but it make somehow sense):
|
What is the purpose of the change
#3625 with size limit checks moved into the NonCopyingByteArrayOutputStream
There's a new constructor to
NonCopyingByteArrayOutputStreamto set a size limit, or no limit, and the default constructor now automatically picks up the size set by system property/fallback default.Those choices could be discussed, with options being
org.apache.avro.SystemLimitException, where the int parser lives.AI: No AI was used for this PR.
Verifying this change
Needs tests, if people are happy with the design I can put one in whichever module people would prefer...it's pretty straightforward
Documentation
Does this pull request introduce a new feature? (yes / no)
yes
If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
javadocs