[
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17555813#comment-17555813
]
ASF GitHub Bot commented on PARQUET-2069:
-----------------------------------------
shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r900688194
##########
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##########
@@ -136,10 +137,22 @@ public RecordMaterializer<T> prepareForRead(
GenericData model = getDataModel(configuration);
String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
- if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
- return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+ try {
+ if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+ return newCompatMaterializer(parquetSchema, avroSchema, model);
+ }
+ return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model);
+ } catch (InvalidRecordException | ClassCastException e) {
Review Comment:
I understand the targetted issue can be solved by this retry with a
converted schema. But I am not sure if it is safe to just ignore Avro schema in
case of exception. @rdblue @wesm Do you have some time to have a look at this?
##########
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##########
@@ -136,10 +137,22 @@ public RecordMaterializer<T> prepareForRead(
GenericData model = getDataModel(configuration);
String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
- if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
- return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+ try {
+ if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+ return newCompatMaterializer(parquetSchema, avroSchema, model);
+ }
+ return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model);
+ } catch (InvalidRecordException | ClassCastException e) {
Review Comment:
I understand the target issue can be solved by this retry with a converted
schema. But I am not sure if it is safe to just ignore Avro schema in case of
exception. @rdblue @wesm Do you have some time to have a look at this?
> 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)