From ace42fdbc31666619ecfa7f8672458bd38b7425f Mon Sep 17 00:00:00 2001 From: arnavb Date: Tue, 9 Sep 2025 10:01:38 +0000 Subject: [PATCH 1/3] update --- .../parquet/avro/AvroSchemaConverter.java | 4 ++- .../parquet/avro/TestAvroSchemaConverter.java | 26 +++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java index 58e9c2e198..782c009db4 100644 --- a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java +++ b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java @@ -404,7 +404,9 @@ public Schema convertINT64(PrimitiveTypeName primitiveTypeName) { @Override public Schema convertINT96(PrimitiveTypeName primitiveTypeName) { if (readInt96AsFixed) { - return Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + String name = parquetType.getName(); + String ns = namespace(name, names); + return Schema.createFixed(name, "INT96 represented as byte[12]", ns, 12); } throw new IllegalArgumentException( "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array."); diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java index d54cd4310a..db3246a15b 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -599,6 +599,32 @@ public void testParquetInt96DefaultFail() throws Exception { () -> new AvroSchemaConverter().convert(parquetSchemaWithInt96)); } + @Test + public void testMultipleInt96FieldsToStringConversion() throws Exception { + Configuration enableInt96ReadingConfig = new Configuration(); + enableInt96ReadingConfig.setBoolean(AvroReadSupport.READ_INT96_AS_FIXED, true); + + Types.MessageTypeBuilder builder = Types.buildMessage(); + builder.optional(PrimitiveType.PrimitiveTypeName.INT96).named("timestamp_1"); + builder.optional(PrimitiveType.PrimitiveTypeName.INT96).named("timestamp_2"); + MessageType int96Schema = builder.named("int96Schema"); + + AvroSchemaConverter converter = new AvroSchemaConverter(enableInt96ReadingConfig); + Schema avroSchema = converter.convert(int96Schema); + + String schemaString = avroSchema.toString(true); + System.out.println("Generated Avro schema:"); + System.out.println(schemaString); + + Assert.assertTrue("First field should have full timestamp_1 definition", + schemaString.contains("\"name\" : \"timestamp_1\"")); + Assert.assertTrue("Second field should have full timestamp_2 definition", + schemaString.contains("\"name\" : \"timestamp_2\"")); + + Assert.assertFalse("Should not reference bare 'INT96' type anymore", + schemaString.contains("\"type\" : [ \"null\", \"INT96\" ]")); + } + @Test public void testDateType() throws Exception { Schema date = LogicalTypes.date().addToSchema(Schema.create(INT)); From 0223303ed54758136e222db511e657d3c902c338 Mon Sep 17 00:00:00 2001 From: arnavb Date: Tue, 9 Sep 2025 10:03:03 +0000 Subject: [PATCH 2/3] update --- .../apache/parquet/avro/TestAvroSchemaConverter.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java index db3246a15b..5bd9960fe0 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -613,15 +613,16 @@ public void testMultipleInt96FieldsToStringConversion() throws Exception { Schema avroSchema = converter.convert(int96Schema); String schemaString = avroSchema.toString(true); - System.out.println("Generated Avro schema:"); - System.out.println(schemaString); - Assert.assertTrue("First field should have full timestamp_1 definition", + Assert.assertTrue( + "First field should have full timestamp_1 definition", schemaString.contains("\"name\" : \"timestamp_1\"")); - Assert.assertTrue("Second field should have full timestamp_2 definition", + Assert.assertTrue( + "Second field should have full timestamp_2 definition", schemaString.contains("\"name\" : \"timestamp_2\"")); - Assert.assertFalse("Should not reference bare 'INT96' type anymore", + Assert.assertFalse( + "Should not reference bare 'INT96' type anymore", schemaString.contains("\"type\" : [ \"null\", \"INT96\" ]")); } From 57eef21a704c87a6a5715b7919b477978633312b Mon Sep 17 00:00:00 2001 From: arnavb Date: Tue, 9 Sep 2025 13:25:14 +0000 Subject: [PATCH 3/3] update --- .../java/org/apache/parquet/avro/TestAvroSchemaConverter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java index 5bd9960fe0..130192e155 100644 --- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java +++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -579,7 +579,7 @@ public void testParquetInt96AsFixed12AvroType() throws Exception { enableInt96ReadingConfig.setBoolean(AvroReadSupport.READ_INT96_AS_FIXED, true); Schema schema = Schema.createRecord("myrecord", null, null, false); - Schema int96schema = Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + Schema int96schema = Schema.createFixed("int96_field", "INT96 represented as byte[12]", null, 12); schema.setFields(Collections.singletonList(new Schema.Field("int96_field", int96schema, null, null))); testParquetToAvroConversion(