Skip to content

Commit 3adb3a0

Browse files
committed
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.
1 parent 7473a6e commit 3adb3a0

3 files changed

Lines changed: 67 additions & 5 deletions

File tree

lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader12.java

Lines changed: 16 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,21 +17,22 @@
1717
*/
1818
package org.apache.avro.file;
1919

20-
import java.io.IOException;
2120
import java.io.Closeable;
21+
import java.io.IOException;
2222
import java.nio.ByteBuffer;
2323
import java.nio.charset.StandardCharsets;
2424
import java.util.Arrays;
2525
import java.util.HashMap;
2626
import java.util.Iterator;
2727
import java.util.Map;
2828

29+
import org.apache.avro.AvroRuntimeException;
2930
import org.apache.avro.InvalidAvroMagicException;
3031
import org.apache.avro.Schema;
3132
import org.apache.avro.UnknownAvroCodecException;
33+
import org.apache.avro.io.BinaryDecoder;
3234
import org.apache.avro.io.DatumReader;
3335
import org.apache.avro.io.DecoderFactory;
34-
import org.apache.avro.io.BinaryDecoder;
3536

3637
/** Read files written by Avro version 1.2. */
3738
public class DataFileReader12<D> implements FileReader<D>, Closeable {
@@ -88,7 +89,7 @@ public DataFileReader12(SeekableInput sin, DatumReader<D> reader) throws IOExcep
8889
if (codec != null && !codec.equals(NULL_CODEC)) {
8990
throw new UnknownAvroCodecException("Unknown codec: " + codec);
9091
}
91-
this.schema = new Schema.Parser().parse(getMetaString(SCHEMA));
92+
this.schema = parseSchema();
9293
this.reader = reader;
9394

9495
reader.setSchema(schema);
@@ -115,6 +116,18 @@ public synchronized long getMetaLong(String key) {
115116
return Long.parseLong(getMetaString(key));
116117
}
117118

119+
private Schema parseSchema() throws IOException {
120+
String schemaJson = getMetaString(SCHEMA);
121+
if (schemaJson == null) {
122+
throw new IOException("Missing required metadata: " + SCHEMA);
123+
}
124+
try {
125+
return new Schema.Parser().parse(schemaJson);
126+
} catch (AvroRuntimeException e) {
127+
throw new IOException("Invalid schema in metadata: " + SCHEMA, e);
128+
}
129+
}
130+
118131
/** Return the schema used in this file. */
119132
@Override
120133
public Schema getSchema() {

lang/java/avro/src/main/java/org/apache/avro/file/DataFileStream.java

Lines changed: 13 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -140,8 +140,7 @@ void initialize(InputStream in, byte[] magic) throws IOException {
140140

141141
// finalize the header
142142
header.metaKeyList = Collections.unmodifiableList(header.metaKeyList);
143-
header.schema = new Schema.Parser(NameValidator.NO_VALIDATION).setValidateDefaults(false)
144-
.parse(getMetaString(DataFileConstants.SCHEMA));
143+
header.schema = parseHeaderSchema();
145144
this.codec = resolveCodec();
146145
reader.setSchema(header.schema);
147146
}
@@ -198,6 +197,18 @@ public long getMetaLong(String key) {
198197
return Long.parseLong(getMetaString(key));
199198
}
200199

200+
private Schema parseHeaderSchema() throws IOException {
201+
String schemaJson = getMetaString(DataFileConstants.SCHEMA);
202+
if (schemaJson == null) {
203+
throw new IOException("Missing required metadata: " + DataFileConstants.SCHEMA);
204+
}
205+
try {
206+
return new Schema.Parser(NameValidator.NO_VALIDATION).setValidateDefaults(false).parse(schemaJson);
207+
} catch (AvroRuntimeException e) {
208+
throw new IOException("Invalid schema in metadata: " + DataFileConstants.SCHEMA, e);
209+
}
210+
}
211+
201212
/**
202213
* Returns an iterator over entries in this file. Note that this iterator is
203214
* shared with other users of the file: it does not contain a separate pointer

lang/java/avro/src/test/java/org/apache/avro/TestDataFileReader.java

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,19 +24,25 @@
2424
import java.io.FileInputStream;
2525
import java.io.FileWriter;
2626
import java.io.IOException;
27+
import java.io.ByteArrayInputStream;
28+
import java.io.ByteArrayOutputStream;
2729
import java.lang.management.ManagementFactory;
2830
import java.lang.management.OperatingSystemMXBean;
2931
import java.nio.file.Files;
3032
import java.nio.file.Path;
3133
import com.sun.management.UnixOperatingSystemMXBean;
3234
import org.apache.avro.file.DataFileReader;
35+
import org.apache.avro.file.DataFileConstants;
3336
import org.apache.avro.file.DataFileStream;
3437
import org.apache.avro.file.DataFileWriter;
3538
import org.apache.avro.file.FileReader;
39+
import org.apache.avro.file.SeekableByteArrayInput;
3640
import org.apache.avro.file.SeekableFileInput;
3741
import org.apache.avro.file.SeekableInput;
3842
import org.apache.avro.generic.GenericDatumReader;
3943
import org.apache.avro.generic.GenericDatumWriter;
44+
import org.apache.avro.io.BinaryEncoder;
45+
import org.apache.avro.io.EncoderFactory;
4046
import org.junit.jupiter.api.Test;
4147
import org.junit.jupiter.api.io.TempDir;
4248

@@ -238,4 +244,36 @@ void invalidMagicBytes() throws IOException {
238244
() -> DataFileReader.openReader(fileInput, new GenericDatumReader<>()));
239245
}
240246
}
247+
248+
@Test
249+
void missingSchemaMetadataDoesNotThrowNullPointerException() throws IOException {
250+
byte[] malformedFile = buildContainerHeaderWithoutSchema();
251+
252+
IOException streamException = assertThrows(IOException.class,
253+
() -> new DataFileStream<>(new ByteArrayInputStream(malformedFile), new GenericDatumReader<>()));
254+
assertNotNull(streamException.getMessage());
255+
assertTrue(streamException.getMessage().contains(DataFileConstants.SCHEMA));
256+
257+
IOException readerException = assertThrows(IOException.class,
258+
() -> new DataFileReader<>(new SeekableByteArrayInput(malformedFile), new GenericDatumReader<>()));
259+
assertNotNull(readerException.getMessage());
260+
assertTrue(readerException.getMessage().contains(DataFileConstants.SCHEMA));
261+
}
262+
263+
private static byte[] buildContainerHeaderWithoutSchema() throws IOException {
264+
ByteArrayOutputStream output = new ByteArrayOutputStream();
265+
output.write(DataFileConstants.MAGIC);
266+
267+
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(output, null);
268+
encoder.writeMapStart();
269+
encoder.setItemCount(1);
270+
encoder.startItem();
271+
encoder.writeString(DataFileConstants.CODEC);
272+
encoder.writeBytes("null".getBytes());
273+
encoder.writeMapEnd();
274+
encoder.writeFixed(new byte[DataFileConstants.SYNC_SIZE]);
275+
encoder.flush();
276+
277+
return output.toByteArray();
278+
}
241279
}

0 commit comments

Comments
 (0)