[ 
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17519218#comment-17519218
 ] 

Timothy Miller commented on PARQUET-2069:
-----------------------------------------

Here's a log message that shows why it's failing:

{{[main] DEBUG org.apache.avro.SchemaCompatibility - Checking compatibility of 
}}
{{reader 
\{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]}
 with }}
{{writer 
\{"type":"record","name":"array","fields":[{"name":"element","type":["null","string"],"default":null}]}}}

When {{AvroRecordConverter.newConverter(Schema schema, Type type, GenericData 
model, Class<?> knownClass, ParentValueContainer setter)}} encounters an ARRAY 
type, it calls {{{}AvroRecordConverter.AvroCollectionConverter{}}}, which calls 
{{{}AvroRecordConverter.isElementType{}}}, which calls 
{{{}SchemaCompatibility.checkReaderWriterCompatibility{}}}. The type that 
returns is INCOMPATIBLE, and this is because the record name for the reader is 
"list" while the record name for the writer is "array", and these are 
considered incompatible.

Either some change has to be made to the compatibility check to allow array and 
list types to be considered compatible, or the writer schema has to be computed 
differently. The latter appears to be done in 
{{{}AvroSchemaConverter.convertFields{}}}, which is called from 
{{{}AvroRecordConverter.isElementType{}}}, like this:

{{                Schema schemaFromRepeated = 
CONVERTER.convert(repeatedType.asGroupType());}}

 

> 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.1#820001)

Reply via email to