From 893d488accf2c9c8adf13c23d4f7f50b22ec9ceb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Mon, 6 Apr 2026 19:54:14 +0000 Subject: [PATCH] AVRO-4242: [Java] Fix NPE in DataFileStream and DataFileReader when schema metadata is missing Malformed Avro container files without the 'avro.schema' metadata entry caused a NullPointerException in both DataFileStream and DataFileReader12 when the null value was passed directly to Schema.Parser.parse(). Replace inline parsing with null-safe helper methods that throw a descriptive IOException instead. --- .../apache/avro/file/DataFileReader12.java | 10 +++-- .../org/apache/avro/file/DataFileStream.java | 20 +++++++++- .../org/apache/avro/TestDataFileReader.java | 38 +++++++++++++++++++ 3 files changed, 63 insertions(+), 5 deletions(-) diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java index c057a86db73..e114013e8a7 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java @@ -17,8 +17,8 @@ */ package org.apache.avro.file; -import java.io.IOException; import java.io.Closeable; +import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -29,9 +29,9 @@ import org.apache.avro.InvalidAvroMagicException; import org.apache.avro.Schema; import org.apache.avro.UnknownAvroCodecException; +import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.DatumReader; import org.apache.avro.io.DecoderFactory; -import org.apache.avro.io.BinaryDecoder; /** Read files written by Avro version 1.2. */ public class DataFileReader12 implements FileReader, Closeable { @@ -88,7 +88,7 @@ public DataFileReader12(SeekableInput sin, DatumReader reader) throws IOExcep if (codec != null && !codec.equals(NULL_CODEC)) { throw new UnknownAvroCodecException("Unknown codec: " + codec); } - this.schema = new Schema.Parser().parse(getMetaString(SCHEMA)); + this.schema = parseSchema(); this.reader = reader; reader.setSchema(schema); @@ -115,6 +115,10 @@ public synchronized long getMetaLong(String key) { return Long.parseLong(getMetaString(key)); } + private Schema parseSchema() throws IOException { + return DataFileStream.parseSchemaFromMetadata(getMetaString(SCHEMA), SCHEMA, new Schema.Parser()); + } + /** Return the schema used in this file. */ @Override public Schema getSchema() { diff --git a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java index e2e79d8eaed..ba02b4d3837 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java +++ b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java @@ -140,8 +140,7 @@ void initialize(InputStream in, byte[] magic) throws IOException { // finalize the header header.metaKeyList = Collections.unmodifiableList(header.metaKeyList); - header.schema = new Schema.Parser(NameValidator.NO_VALIDATION).setValidateDefaults(false) - .parse(getMetaString(DataFileConstants.SCHEMA)); + header.schema = parseHeaderSchema(); this.codec = resolveCodec(); reader.setSchema(header.schema); } @@ -198,6 +197,23 @@ public long getMetaLong(String key) { return Long.parseLong(getMetaString(key)); } + static Schema parseSchemaFromMetadata(String schemaJson, String schemaMetadataKey, Schema.Parser parser) + throws IOException { + if (schemaJson == null) { + throw new IOException("Missing required metadata: " + schemaMetadataKey); + } + try { + return parser.parse(schemaJson); + } catch (AvroRuntimeException e) { + throw new IOException("Invalid schema in metadata: " + schemaMetadataKey, e); + } + } + + private Schema parseHeaderSchema() throws IOException { + return parseSchemaFromMetadata(getMetaString(DataFileConstants.SCHEMA), DataFileConstants.SCHEMA, + new Schema.Parser(NameValidator.NO_VALIDATION).setValidateDefaults(false)); + } + /** * Returns an iterator over entries in this file. Note that this iterator is * shared with other users of the file: it does not contain a separate pointer diff --git a/lang/java/avro/src/test/java/org/apache/avro/TestDataFileReader.java b/lang/java/avro/src/test/java/org/apache/avro/TestDataFileReader.java index 6ed6b35cc50..16d4ccdcb87 100644 --- a/lang/java/avro/src/test/java/org/apache/avro/TestDataFileReader.java +++ b/lang/java/avro/src/test/java/org/apache/avro/TestDataFileReader.java @@ -24,19 +24,25 @@ import java.io.FileInputStream; import java.io.FileWriter; import java.io.IOException; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.lang.management.ManagementFactory; import java.lang.management.OperatingSystemMXBean; import java.nio.file.Files; import java.nio.file.Path; import com.sun.management.UnixOperatingSystemMXBean; import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.DataFileConstants; import org.apache.avro.file.DataFileStream; import org.apache.avro.file.DataFileWriter; import org.apache.avro.file.FileReader; +import org.apache.avro.file.SeekableByteArrayInput; import org.apache.avro.file.SeekableFileInput; import org.apache.avro.file.SeekableInput; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.EncoderFactory; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -238,4 +244,36 @@ void invalidMagicBytes() throws IOException { () -> DataFileReader.openReader(fileInput, new GenericDatumReader<>())); } } + + @Test + void missingSchemaMetadataDoesNotThrowNullPointerException() throws IOException { + byte[] malformedFile = buildContainerHeaderWithoutSchema(); + + IOException streamException = assertThrows(IOException.class, + () -> new DataFileStream<>(new ByteArrayInputStream(malformedFile), new GenericDatumReader<>())); + assertNotNull(streamException.getMessage()); + assertTrue(streamException.getMessage().contains(DataFileConstants.SCHEMA)); + + IOException readerException = assertThrows(IOException.class, + () -> new DataFileReader<>(new SeekableByteArrayInput(malformedFile), new GenericDatumReader<>())); + assertNotNull(readerException.getMessage()); + assertTrue(readerException.getMessage().contains(DataFileConstants.SCHEMA)); + } + + private static byte[] buildContainerHeaderWithoutSchema() throws IOException { + ByteArrayOutputStream output = new ByteArrayOutputStream(); + output.write(DataFileConstants.MAGIC); + + BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(output, null); + encoder.writeMapStart(); + encoder.setItemCount(1); + encoder.startItem(); + encoder.writeString(DataFileConstants.CODEC); + encoder.writeBytes("null".getBytes()); + encoder.writeMapEnd(); + encoder.writeFixed(new byte[DataFileConstants.SYNC_SIZE]); + encoder.flush(); + + return output.toByteArray(); + } }