This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git
The following commit(s) were added to refs/heads/master by this push:
new 67fc368c9 PARQUET-2456: Avoid type name conflicts on fixed type (#1304)
67fc368c9 is described below
commit 67fc368c91a3b81ec020327fd31eee5bb87bb6cd
Author: Michel Davit <[email protected]>
AuthorDate: Fri Apr 5 04:43:43 2024 +0200
PARQUET-2456: Avoid type name conflicts on fixed type (#1304)
---
.../apache/parquet/avro/AvroSchemaConverter.java | 13 +++--
.../parquet/avro/TestAvroSchemaConverter.java | 62 ++++++++++++++++------
2 files changed, 55 insertions(+), 20 deletions(-)
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 59009bc70..d5f85ce44 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
@@ -296,8 +296,8 @@ public class AvroSchemaConverter {
}
private Schema convertFields(String name, List<Type> parquetFields,
Map<String, Integer> names) {
+ String ns = namespace(name, names);
List<Schema.Field> fields = new ArrayList<Schema.Field>();
- Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue +
1);
for (Type parquetType : parquetFields) {
Schema fieldSchema = convertField(parquetType, names);
if (parquetType.isRepetition(REPEATED)) { // If a repeated field is
ungrouped, treat as REQUIRED per spec
@@ -308,7 +308,7 @@ public class AvroSchemaConverter {
fields.add(new Schema.Field(parquetType.getName(), fieldSchema, null,
(Object) null));
}
}
- Schema schema = Schema.createRecord(name, null, nameCount > 1 ? name +
nameCount : null, false);
+ Schema schema = Schema.createRecord(name, null, ns, false);
schema.setFields(fields);
return schema;
}
@@ -360,7 +360,9 @@ public class AvroSchemaConverter {
return Schema.create(Schema.Type.STRING);
} else {
int size = parquetType.asPrimitiveType().getTypeLength();
- return Schema.createFixed(parquetType.getName(), null, null,
size);
+ String name = parquetType.getName();
+ String ns = namespace(name, names);
+ return Schema.createFixed(name, null, ns, size);
}
}
@@ -595,4 +597,9 @@ public class AvroSchemaConverter {
}
return path + '.' + fieldName;
}
+
+ private static String namespace(String name, Map<String, Integer> names) {
+ Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue +
1);
+ return nameCount > 1 ? name + nameCount : null;
+ }
}
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 4e7658d94..3b8d5551e 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
@@ -23,6 +23,7 @@ import static org.apache.avro.Schema.Type.LONG;
import static org.apache.avro.Schema.Type.STRING;
import static
org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE;
import static
org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility;
+import static org.apache.parquet.avro.AvroTestUtil.array;
import static org.apache.parquet.avro.AvroTestUtil.field;
import static org.apache.parquet.avro.AvroTestUtil.optionalField;
import static org.apache.parquet.avro.AvroTestUtil.primitive;
@@ -761,12 +762,13 @@ public class TestAvroSchemaConverter {
Schema outerA1 = record("a1", field("a2", primitive(Schema.Type.FLOAT)),
optionalField("a1", innerA1));
Schema schema = record("Message", optionalField("a1", outerA1));
- String parquetSchema = "message Message {\n" + " optional group a1
{\n"
- + " required float a2;\n"
- + " optional group a1 {\n"
- + " required float a4;\n"
- + " }\n"
- + " }\n"
+ String parquetSchema = "message Message {\n"
+ + " optional group a1 {\n"
+ + " required float a2;\n"
+ + " optional group a1 {\n"
+ + " required float a4;\n"
+ + " }\n"
+ + " }\n"
+ "}\n";
testParquetToAvroConversion(schema, parquetSchema);
@@ -784,17 +786,43 @@ public class TestAvroSchemaConverter {
Schema schema = record("Message", optionalField("a1", a1),
optionalField("a3", a3));
- String parquetSchema = "message Message {\n" + " optional group a1
{\n"
- + " optional group a2 {\n"
- + " required float a4;\n"
- + " }\n"
- + " }\n"
- + " optional group a3 {\n"
- + " optional group a2 {\n"
- + " required float a4;\n"
- + " required float a5;\n"
- + " }\n"
- + " }\n"
+ String parquetSchema = "message Message {\n"
+ + " optional group a1 {\n"
+ + " optional group a2 {\n"
+ + " required float a4;\n"
+ + " }\n"
+ + " }\n"
+ + " optional group a3 {\n"
+ + " optional group a2 {\n"
+ + " required float a4;\n"
+ + " required float a5;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n";
+
+ testParquetToAvroConversion(schema, parquetSchema);
+ testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema);
+ }
+
+ @Test
+ public void testReuseNamesArrays() throws Exception {
+ Schema a1 = record("array", field("a4", primitive(Schema.Type.FLOAT)));
+ Schema a2 = Schema.createFixed("array", null, "array2", 1);
+ Schema a3 = Schema.createFixed("array", null, "array3", 1);
+ Schema schema = record("Message", field("a1", array(a1)), field("a2",
array(a2)), field("a3", array(a3)));
+
+ String parquetSchema = "message Message {\n"
+ + " required group a1 (LIST) {\n"
+ + " repeated group array {\n"
+ + " required float a4;\n"
+ + " }\n"
+ + " }\n"
+ + " required group a2 (LIST) {\n"
+ + " repeated fixed_len_byte_array(1) array;\n"
+ + " }\n"
+ + " required group a3 (LIST) {\n"
+ + " repeated fixed_len_byte_array(1) array;\n"
+ + " }\n"
+ "}\n";
testParquetToAvroConversion(schema, parquetSchema);