Skip to content

AVRO-4241: [Java] BinaryDecoder should verify available bytes before reading#3725

Open
iemejia wants to merge 3 commits intoapache:mainfrom
iemejia:AVRO-4241-binarydecoder-verify-available-bytes-before-read
Open

AVRO-4241: [Java] BinaryDecoder should verify available bytes before reading#3725
iemejia wants to merge 3 commits intoapache:mainfrom
iemejia:AVRO-4241-binarydecoder-verify-available-bytes-before-read

Conversation

@iemejia
Copy link
Copy Markdown
Member

@iemejia iemejia commented Apr 6, 2026

Add ensureAvailableBytes() pre-check in readString, readBytes, readArrayStart, arrayNext, readMapStart, and mapNext to verify the source has sufficient data before proceeding.

Byte-array-backed sources return an exact remaining count. Stream-backed sources return buffered bytes plus InputStream.available(), which is reliable for the finite streams used by DataFileReader and DataFileStream.

Includes regression tests and updated array/map limit tests.

R: @RyanSkraba @martin-g

What is the purpose of the change

(For example: This pull request improves file read performance by buffering data, fixing AVRO-XXXX.)

Verifying this change

(Please pick one of the following options)

This change is a trivial rework / code cleanup without any test coverage.

(or)

This change is already covered by existing tests, such as (please describe tests).

(or)

This change added tests and can be verified as follows:

(example:)

  • Extended interop tests to verify consistent valid schema names between SDKs
  • Added test that validates that Java throws an AvroRuntimeException on invalid binary data
  • Manually verified the change by building the website and checking the new redirect

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

@github-actions github-actions Bot added the Java Pull Requests for Java binding label Apr 6, 2026
@iemejia iemejia force-pushed the AVRO-4241-binarydecoder-verify-available-bytes-before-read branch 3 times, most recently from ef9c52a to ce192d2 Compare April 7, 2026 13:44
…reading

Add ensureAvailableBytes() pre-check in readString, readBytes,
readArrayStart, arrayNext, readMapStart, and mapNext to verify the
source has sufficient data before proceeding.

Byte-array-backed sources return an exact remaining count.
Stream-backed sources return buffered bytes plus InputStream.available(),
which is reliable for the finite streams used by DataFileReader and
DataFileStream.

Includes regression tests and updated array/map limit tests.
@iemejia iemejia force-pushed the AVRO-4241-binarydecoder-verify-available-bytes-before-read branch from ce192d2 to 8471fc2 Compare April 7, 2026 15:35
@iemejia iemejia requested review from RyanSkraba and martin-g April 8, 2026 07:25
@iemejia iemejia changed the title AVRO-4241: [Java] BinaryDecoder should verify available bytes before … AVRO-4241: [Java] BinaryDecoder should verify available bytes before reading Apr 8, 2026
@RyanSkraba
Copy link
Copy Markdown
Contributor

To cherry pick to 1.12!

Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

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

Pull request overview

This PR strengthens Java binary decoding against truncated/malicious inputs by adding “bytes remaining” awareness and using it to fail fast (EOF) before allocating large buffers or collection backing structures.

Changes:

  • Add a Decoder#remainingBytes() API (default -1) and plumb it through BinaryDecoder and ValidatingDecoder.
  • Add early “ensure available bytes” checks for BinaryDecoder.readString/readBytes and schema-aware prechecks for array/map block counts in GenericDatumReader.
  • Add/adjust regression tests covering string/bytes length validation and array/map byte-limit behavior.

Reviewed changes

Copilot reviewed 7 out of 7 changed files in this pull request and generated 2 comments.

Show a summary per file
File Description
lang/java/avro/src/main/java/org/apache/avro/io/Decoder.java Introduces remainingBytes() default API for decoders.
lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java Implements remainingBytes() and adds early byte-availability checks for string/bytes reads.
lang/java/avro/src/main/java/org/apache/avro/io/ValidatingDecoder.java Delegates remainingBytes() to the underlying decoder.
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java Adds schema-aware byte validation for arrays/maps before allocating/reading blocks.
lang/java/avro/src/main/java/org/apache/avro/util/ByteBufferInputStream.java Implements available() to support accurate remaining-byte reporting.
lang/java/avro/src/test/java/org/apache/avro/io/TestBinaryDecoder.java Adds tests ensuring EOF is thrown before large allocations for string/bytes.
lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumReader.java Adds unit tests for minBytesPerElement and end-to-end collection byte validation.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

int buffered = ba.getLim() - ba.getPos();
try {
if (in.getClass() == ByteArrayInputStream.class || in.getClass() == ByteBufferInputStream.class) {
return buffered + in.available();
Comment on lines +408 to +411
int minBytesPerEntry = 1 + minBytesPerElement(valueSchema);
if (count > 0) {
int remaining = decoder.remainingBytes();
if (remaining >= 0 && count * (long) minBytesPerEntry > remaining) {
*/
private static void ensureAvailableMapBytes(Decoder decoder, long count, Schema valueSchema) throws EOFException {
// Map keys are always strings: at least 1 byte for the length varint
int minBytesPerEntry = 1 + minBytesPerElement(valueSchema);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

go with copilot and use longs here to avoid problems on very large files

private long arrayNext(ResolvingDecoder in, Schema elementType) throws IOException {
long l = in.arrayNext();
if (l > 0) {
ensureAvailableCollectionBytes(in, l, elementType);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

if this a no-op on l <= 0 then you wouldn't need to guard all the uses

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Got it fixing it

}

@Override
public int available() throws IOException {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

there's this real oddness with available(), in that some interpret it as "all that is left in the stream" but it can also be interpreted as "bytes you can read() before blocking for new data". that's probably the correct one.

it does hold here, just important not to use available() as measures of how much is left in a stream, which may be larger. looks like you are doing the right thing and testing it later on.

return 0; // break recursion for self-referencing schemas
}
long sum = 0;
for (Schema.Field f : schema.getFields()) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

I worry about the cost of this operation on a complex wide and recursive structure, as it'll be invoked once per record.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I added a JMH benchmark test to measure the impact of this for wide and deeply nested structures the results are promising apparently the extra cost is negligible.
https://gist.github.com/iemejia/bae3302ec0f3d2abf92e99911ccba606

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

that's cool. I'm hitting serious field enum problems on variants in parquet, as shredded variants explode the schema.

iemejia added 2 commits April 30, 2026 21:10
…n on GenericDatumReader

Benchmark to measure the overhead of {@code minBytesPerElement} computation during array/map decoding via {@link GenericDatumReader}. Tests complex, wide, and recursive schema structures to verify that the per-block schema traversal cost is acceptable.
@iemejia
Copy link
Copy Markdown
Member Author

iemejia commented Apr 30, 2026

All PR review comments addressed
PTAL again @steveloughran and @RyanSkraba

Copy link
Copy Markdown

@steveloughran steveloughran left a comment

Choose a reason for hiding this comment

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

LGTM

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.

5 participants