[
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17534976#comment-17534976
]
ASF GitHub Bot commented on PARQUET-2069:
-----------------------------------------
shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r870479919
##########
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##########
@@ -866,6 +866,20 @@ static boolean isElementType(Type repeatedType, Schema
elementSchema) {
} else if (elementSchema != null &&
elementSchema.getType() == Schema.Type.RECORD) {
Schema schemaFromRepeated =
CONVERTER.convert(repeatedType.asGroupType());
+
+ // Fix for PARQUET-2069
+ // ParquetMR breaks compatibility with itself by including a JSON
+ // representation of a schema that names a record "list", when
+ // it should be named "array" to match with the rest of the metadata.
+ // Inserting this code allows Avro to detect that the "array" and "list"
+ // types are compatible. Since this alias is being added to something
+ // that is the result of parsing JSON, we can't add the alias at the
+ // time of construction. Therefore we have to do it here where the the
data
+ // structures have been unwrapped to the point where we have the
+ // incompatible structure and can add the necessary alias.
+ if (elementSchema.getName().equals("list"))
elementSchema.addAlias("array", "");
Review Comment:
Follow the above standard like line 866.
> Parquet file containing arrays, written by Parquet-MR, cannot be read again
> by Parquet-MR
> -----------------------------------------------------------------------------------------
>
> Key: PARQUET-2069
> URL: https://issues.apache.org/jira/browse/PARQUET-2069
> Project: Parquet
> Issue Type: Bug
> Components: parquet-avro
> Affects Versions: 1.12.0
> Environment: Windows 10
> Reporter: Devon Kozenieski
> Priority: Blocker
> Attachments: modified.parquet, original.parquet, parquet-diff.png
>
>
> In the attached files, there is one original file, and one written modified
> file that results after reading the original file and writing it back with
> Parquet-MR, with a few values modified. The schema should not be modified,
> since the schema of the input file is used as the schema to write the output
> file. However, the output file has a slightly modified schema that then
> cannot be read back the same way again with Parquet-MR, resulting in the
> exception message: java.lang.ClassCastException: optional binary element
> (STRING) is not a group
> My guess is that the issue lies in the Avro schema conversion.
> The Parquet files attached have some arrays and some nested fields.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)