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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to