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 61bdf44ae PARQUET-2425: Support non-grouped repeated fields in
AvroSchemaConverter (#1262)
61bdf44ae is described below
commit 61bdf44aeded2e0483351e635fdbc908c0967fc0
Author: Claire McGinty <[email protected]>
AuthorDate: Mon Feb 19 03:40:21 2024 -0500
PARQUET-2425: Support non-grouped repeated fields in AvroSchemaConverter
(#1262)
---
.../java/org/apache/parquet/avro/AvroSchemaConverter.java | 5 ++---
.../org/apache/parquet/avro/TestAvroSchemaConverter.java | 15 +++++++++++++++
2 files changed, 17 insertions(+), 3 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 6e25bf12c..59009bc70 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
@@ -300,9 +300,8 @@ public class AvroSchemaConverter {
Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue +
1);
for (Type parquetType : parquetFields) {
Schema fieldSchema = convertField(parquetType, names);
- if (parquetType.isRepetition(REPEATED)) {
- throw new UnsupportedOperationException(
- "REPEATED not supported outside LIST or MAP. Type: " +
parquetType);
+ if (parquetType.isRepetition(REPEATED)) { // If a repeated field is
ungrouped, treat as REQUIRED per spec
+ fields.add(new Schema.Field(parquetType.getName(),
Schema.createArray(fieldSchema)));
} else if (parquetType.isRepetition(Type.Repetition.OPTIONAL)) {
fields.add(new Schema.Field(parquetType.getName(),
optional(fieldSchema), null, NULL_VALUE));
} else { // REQUIRED
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 702ac8c43..4e7658d94 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
@@ -422,6 +422,21 @@ public class TestAvroSchemaConverter {
+ "}");
}
+ @Test
+ public void testConvertUngroupedRepeatedField() throws Exception {
+ testParquetToAvroConversion(
+ NEW_BEHAVIOR,
+ new Schema.Parser()
+ .parse("{\"type\": \"record\","
+ + " \"name\": \"SchemaWithRepeatedField\","
+ + " \"fields\": [{"
+ + " \"name\": \"repeatedField\","
+ + " \"type\": {\"type\": \"array\",\"items\": \"int\"}"
+ + " }]"
+ + "}"),
+ "message SchemaWithRepeatedField { repeated int32 repeatedField; }");
+ }
+
@Test
public void testOldThriftListOfLists() throws Exception {
Schema listOfLists =
optional(Schema.createArray(Schema.createArray(Schema.create(INT))));