Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

GH-3115-Fix int96 read issue in complex type #3118

Merged
merged 5 commits into from
Jan 30, 2025
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ public AvroSchemaConverter() {
this.assumeRepeatedIsListElement = assumeRepeatedIsListElement;
this.writeOldListStructure = WRITE_OLD_LIST_STRUCTURE_DEFAULT;
this.writeParquetUUID = WRITE_PARQUET_UUID_DEFAULT;
this.readInt96AsFixed = READ_INT96_AS_FIXED_DEFAULT;
this.readInt96AsFixed = true;
this.pathsToInt96 = Collections.emptySet();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.parquet.avro;

import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED;
import static org.apache.parquet.avro.AvroTestUtil.array;
import static org.apache.parquet.avro.AvroTestUtil.field;
import static org.apache.parquet.avro.AvroTestUtil.instance;
Expand Down Expand Up @@ -1136,6 +1137,25 @@ public void testIsElementTypeRequiredRepeatedRecord() {
avroSchema.getFields().get(0).schema()));
}

@Test
public void testIsElementTypeInt96Element() {
Configuration configuration = new Configuration();
configuration.setBoolean(READ_INT96_AS_FIXED, true);

MessageType parquetSchema = MessageTypeParser.parseMessageType("message SchemaWithInt96 {\n"
+ " optional group list (LIST) {\n"
+ " repeated group list {\n"
+ " optional int96 a_timestamp;\n"
+ " }\n"
+ " }\n"
+ "}");
Schema avroSchema = new AvroSchemaConverter(configuration).convert(parquetSchema);
Assert.assertTrue(AvroRecordConverter.isElementType(
parquetSchema.getType("list").asGroupType().getType("list"),
AvroSchemaConverter.getNonNull(avroSchema.getFields().get(0).schema())
.getElementType()));
}

@Test
public void testIsElementTypeOptionalRepeatedRecord() {
// Test `_tuple` style naming
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ private void testParquetToAvroConversion(Schema avroSchema, String schemaString)

private void testParquetToAvroConversion(Configuration conf, Schema avroSchema, String schemaString)
throws Exception {
AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(conf);
AvroSchemaConverter avroSchemaConverter = conf == null ? new AvroSchemaConverter() : new AvroSchemaConverter(conf);
Schema schema = avroSchemaConverter.convert(MessageTypeParser.parseMessageType(schemaString));
assertEquals("converting " + schemaString + " to " + avroSchema, avroSchema.toString(), schema.toString());
}
Expand Down Expand Up @@ -587,16 +587,12 @@ public void testParquetInt96AsFixed12AvroType() throws Exception {
}

@Test
public void testParquetInt96DefaultFail() throws Exception {
public void testParquetInt96DefaultPass() throws Exception {
Schema schema = Schema.createRecord("myrecord", null, null, false);
Schema int96schema = Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12);
schema.setFields(Collections.singletonList(new Schema.Field("int96_field", int96schema, null, null)));

MessageType parquetSchemaWithInt96 =
MessageTypeParser.parseMessageType("message myrecord {\n required int96 int96_field;\n}\n");

assertThrows(
"INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array.",
IllegalArgumentException.class,
() -> new AvroSchemaConverter().convert(parquetSchemaWithInt96));
testParquetToAvroConversion(null, schema, "message myrecord {\n required int96 int96_field;\n}\n");
}

@Test
Expand Down
Loading