[
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17630212#comment-17630212
]
ASF GitHub Bot commented on PARQUET-2069:
-----------------------------------------
wgtmac commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r1016186398
##########
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##########
@@ -136,10 +139,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) {
+ log.error("Warning, Avro schema doesn't match Parquet schema, falling
back to conversion: ", e);
+ // If the Avro schema is bad, fall back to reconstructing it from the
Parquet schema
+ avroSchema = new
AvroSchemaConverter(configuration).convert(parquetSchema);
Review Comment:
Is it possible to fix the converter itself as we know it is relevant to list
(and map type mentioned in the JIRA)?
##########
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:
With this fix, the reader still throws when reading from the attached file
in the JIRA
(https://issues.apache.org/jira/secure/attachment/13030884/original.parquet). I
can verify that modified.parquet is fixed with the fallback. Why is that? Does
it mean there is any corrupted schema like original.parquet in production (not
formally released)?
##########
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:
Vote +1 for adding a new config parameter. The fallback mechanism
transparently omits extra schema information which users may depend on.
Therefore it is good to let them get the error and try the workaround setting
popped up from the exception message.
> 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.10#820010)