1717 */
1818package org .apache .avro .generic ;
1919
20+ import java .io .EOFException ;
2021import java .io .IOException ;
2122import java .lang .reflect .Constructor ;
2223import java .nio .ByteBuffer ;
2324import java .util .Collection ;
2425import java .util .HashMap ;
26+ import java .util .HashSet ;
2527import java .util .Map ;
28+ import java .util .Set ;
2629import java .util .concurrent .ConcurrentHashMap ;
2730import java .util .function .Function ;
2831
@@ -291,6 +294,7 @@ protected Object readArray(Object old, Schema expected, ResolvingDecoder in) thr
291294 long l = in .readArrayStart ();
292295 long base = 0 ;
293296 if (l > 0 ) {
297+ ensureAvailableCollectionBytes (in , l , expectedType );
294298 LogicalType logicalType = expectedType .getLogicalType ();
295299 Conversion <?> conversion = getData ().getConversionFor (logicalType );
296300 Object array = newArray (old , (int ) l , expected );
@@ -306,13 +310,25 @@ protected Object readArray(Object old, Schema expected, ResolvingDecoder in) thr
306310 }
307311 }
308312 base += l ;
309- } while ((l = in . arrayNext ()) > 0 );
313+ } while ((l = arrayNext (in , expectedType )) > 0 );
310314 return pruneArray (array );
311315 } else {
312316 return pruneArray (newArray (old , 0 , expected ));
313317 }
314318 }
315319
320+ /**
321+ * Reads the next array block count and validates remaining bytes before the
322+ * caller allocates storage.
323+ */
324+ private long arrayNext (ResolvingDecoder in , Schema elementType ) throws IOException {
325+ long l = in .arrayNext ();
326+ if (l > 0 ) {
327+ ensureAvailableCollectionBytes (in , l , elementType );
328+ }
329+ return l ;
330+ }
331+
316332 private Object pruneArray (Object object ) {
317333 if (object instanceof GenericArray <?>) {
318334 ((GenericArray <?>) object ).prune ();
@@ -348,6 +364,9 @@ protected Object readMap(Object old, Schema expected, ResolvingDecoder in) throw
348364 long l = in .readMapStart ();
349365 LogicalType logicalType = eValue .getLogicalType ();
350366 Conversion <?> conversion = getData ().getConversionFor (logicalType );
367+ if (l > 0 ) {
368+ ensureAvailableMapBytes (in , l , eValue );
369+ }
351370 Object map = newMap (old , (int ) l );
352371 if (l > 0 ) {
353372 do {
@@ -361,11 +380,40 @@ protected Object readMap(Object old, Schema expected, ResolvingDecoder in) throw
361380 addToMap (map , readMapKey (null , expected , in ), readWithoutConversion (null , eValue , in ));
362381 }
363382 }
364- } while ((l = in . mapNext ()) > 0 );
383+ } while ((l = mapNext (in , eValue )) > 0 );
365384 }
366385 return map ;
367386 }
368387
388+ /**
389+ * Reads the next map block count and validates remaining bytes before the
390+ * caller allocates storage.
391+ */
392+ private long mapNext (ResolvingDecoder in , Schema valueType ) throws IOException {
393+ long l = in .mapNext ();
394+ if (l > 0 ) {
395+ ensureAvailableMapBytes (in , l , valueType );
396+ }
397+ return l ;
398+ }
399+
400+ /**
401+ * Validates remaining bytes for a map block. Each map entry has a string key
402+ * (at least 1 byte for the length varint) plus a value, so the minimum bytes
403+ * per entry is {@code 1 + minBytesPerElement(valueSchema)}.
404+ */
405+ private static void ensureAvailableMapBytes (Decoder decoder , long count , Schema valueSchema ) throws EOFException {
406+ // Map keys are always strings: at least 1 byte for the length varint
407+ int minBytesPerEntry = 1 + minBytesPerElement (valueSchema );
408+ if (count > 0 ) {
409+ int remaining = decoder .remainingBytes ();
410+ if (remaining >= 0 && count * (long ) minBytesPerEntry > remaining ) {
411+ throw new EOFException ("Map claims " + count + " entries with at least " + minBytesPerEntry
412+ + " bytes each, but only " + remaining + " bytes are available" );
413+ }
414+ }
415+ }
416+
369417 /**
370418 * Called by the default implementation of {@link #readMap} to read a key value.
371419 * The default implementation returns delegates to
@@ -384,6 +432,73 @@ protected void addToMap(Object map, Object key, Object value) {
384432 ((Map ) map ).put (key , value );
385433 }
386434
435+ /**
436+ * Returns the minimum number of bytes required to encode a single value of the
437+ * given schema in Avro binary format. Used to validate that the decoder has
438+ * enough data remaining before allocating collection backing arrays.
439+ * <p>
440+ * Returns 0 for types whose binary encoding is empty ({@code null}, zero-length
441+ * {@code fixed}, records with only zero-byte fields). Returns a positive value
442+ * for all other types.
443+ */
444+ static int minBytesPerElement (Schema schema ) {
445+ return minBytesPerElement (schema , new HashSet <>());
446+ }
447+
448+ private static int minBytesPerElement (Schema schema , Set <String > visited ) {
449+ switch (schema .getType ()) {
450+ case NULL :
451+ return 0 ;
452+ case FIXED :
453+ return schema .getFixedSize ();
454+ case FLOAT :
455+ return 4 ;
456+ case DOUBLE :
457+ return 8 ;
458+ case RECORD :
459+ String fullName = schema .getFullName ();
460+ if (!visited .add (fullName )) {
461+ return 0 ; // break recursion for self-referencing schemas
462+ }
463+ int sum = 0 ;
464+ for (Schema .Field f : schema .getFields ()) {
465+ sum += minBytesPerElement (f .schema (), visited );
466+ if (sum > 0 ) {
467+ break ; // at least 1 byte is enough to confirm non-zero
468+ }
469+ }
470+ visited .remove (fullName );
471+ return sum ;
472+ case UNION :
473+ // The branch index varint is always at least 1 byte
474+ return 1 ;
475+ default :
476+ // BOOLEAN, INT, LONG, ENUM, STRING, BYTES, ARRAY, MAP are all >= 1 byte
477+ return 1 ;
478+ }
479+ }
480+
481+ /**
482+ * Validates that the decoder has enough remaining bytes to hold {@code count}
483+ * elements of the given schema, assuming each element requires at least
484+ * {@link #minBytesPerElement} bytes. Throws {@link EOFException} if the decoder
485+ * reports fewer remaining bytes than required.
486+ * <p>
487+ * This check prevents out-of-memory errors from pre-allocating huge backing
488+ * arrays when the source data is truncated or malicious.
489+ */
490+ private static void ensureAvailableCollectionBytes (Decoder decoder , long count , Schema elementSchema )
491+ throws EOFException {
492+ int minBytes = minBytesPerElement (elementSchema );
493+ if (minBytes > 0 && count > 0 ) {
494+ int remaining = decoder .remainingBytes ();
495+ if (remaining >= 0 && count * (long ) minBytes > remaining ) {
496+ throw new EOFException ("Collection claims " + count + " elements with at least " + minBytes
497+ + " bytes each, but only " + remaining + " bytes are available" );
498+ }
499+ }
500+ }
501+
387502 /**
388503 * Called to read a fixed value. May be overridden for alternate fixed
389504 * representations. By default, returns {@link GenericFixed}.
0 commit comments