-
Notifications
You must be signed in to change notification settings - Fork 1.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
PARQUET-2069: Allow list and array record types to be compatible. #957
base: master
Are you sure you want to change the base?
Conversation
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java
Outdated
Show resolved
Hide resolved
Can you add tests? |
Changed formatting of IF statements.
I'm not sure how to add a whole new test. I'll see if I can figure it out. Also, the best way to test this would be to include the parquet file from the bug report, and I'm not sure where I'd put that in the source tree. Any suggestions would be much appreciated. |
I won't be able to add a test any time soon. Here's why. First take note of the two parquet files attached to https://issues.apache.org/jira/browse/PARQUET-2069. When I implement my own Parquet reader, the fix in this PR is able to make the "modified.parquet" file readable by ParquetMR. So what I did was copy org.apache.parquet.avro.TestBackwardCompatibility and modify it to read a new parquet file that I added to the resources folder, imitating the way I did it in my own reader. If I make my new test (TestArrayListCompatibility) point to original.parquet, it reads just fine, and the test passes. But if I make it point to modified.parquet, then I get an exception no matter whether this PR's fix is in or not. And the exception thrown is not the same as the exception described in the bug report. Instead, I get this: org.apache.parquet.io.InvalidRecordException: Parquet/Avro schema mismatch: Avro field 'elements' not found This has exposed some other bug(s) in Parquet/Avro. The thing is, since this isn't reproducible when I use my own reader, then the only way to reproduce it is to use tests built into ParquetMR. But due to ParquetMR's unfortunate reliance on runtime-generated code, it's impossible to run tests from the IDE, which makes them incredibly difficult to debug. If someone has a solution to that problem, I'd really appreciate some help. |
Sometimes the avro schema is bad. This fix will attempt to use the avro schema, but if that fails, it'll fall back to converting the parquet schema.
OK, check out the code changes. I've redone this completely. Now what it does is try out the avro schema, and if that fails, it catches the exception and tries again with an avro schema that it converts from the parquet schema. This fixes not only 2069 but at least one other bug (whose number I can't remember). |
@theosib-amazon I've hit a similar issue. Tried your fix and I see this error unfortunately. Can you test on Map<string, struct<bigint, string>>? Warning, Avro schema doesn't match Parquet schema, falling back to conversion: java.lang.ClassCastException: required binary element (STRING) is not a group |
@islamismailov Can you provide me with a parquet file and changes to the test bench that reproduce this error? |
I debugged this some more and it looks like some of the problem is coming from conversion between parquet and avro. Especially if you read parquetSchema, convert it to avro and set projection in avro schema format, it would get converted back to parquet and it will look different from the original.
|
Yes, there are some major problems with the conversions between schemas that we should turn our attention to. |
This is a "list" AND a "map" issue, not just list. If you're using Iceberg, good news: just apply this PR to your iceberg branch apache/iceberg#3309 Link to the original issue: apache/iceberg#2962 This worked for us. If you still want to fix it in parquet you might be interested in this change, or something along those lines (not recommended as I didn't fully test this change):
|
This patch of yours is cool. I can't tell you without further analysis if it's a universal fix, but how about you make your own PR but borrow the test I've included in my PR? |
try { | ||
firstRecord = (GenericData.Record) parquetReader.read(); | ||
} catch (Exception x) { | ||
x.printStackTrace(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think if you don't catch, it would still print out the stack.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, I got rid of the extra catch. I'm not sure what kind of exceptions parquetReader.read() can throw, though, so we'll see if we get a compile error from not specifying it in the function signature. :)
} | ||
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); | ||
} catch (InvalidRecordException | ClassCastException e) { | ||
System.err.println("Warning, Avro schema doesn't match Parquet schema, falling back to conversion: " + e.toString()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Any reason we don't use Log4j?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oversight on my part.
return newCompatMaterializer(parquetSchema, avroSchema, model); | ||
} | ||
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); | ||
} catch (InvalidRecordException | ClassCastException e) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the underlying problem is that some versions of ParquetMR produce bad schemas, so when we try to load those same files, parsing fails, since the Parquet schema implicit in the file metadata doesn't match up with the stored Avro schema. I'm not sure what to do about bad schemas other than to throw them away and try a fallback.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't have a good solution either. I am just afraid that if we introduce this, there could be some unknown side effects. Given this is a problematic area already(I see you commented on https://issues.apache.org/jira/browse/PARQUET-1681), I am not confident to merge it now.
Or at least, we can have a feature flag to turn on/off this fix.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's up to you. I see this change as just a fall-back in case it bombs. Either it'll work, or it'll bomb again, in which case we're no worse off.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
Allow any exception from parquetReader.read() propagate up without intermediate catch.
Use log4j
return newCompatMaterializer(parquetSchema, avroSchema, model); | ||
} | ||
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); | ||
} catch (InvalidRecordException | ClassCastException e) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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)?
return newCompatMaterializer(parquetSchema, avroSchema, model); | ||
} | ||
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); | ||
} catch (InvalidRecordException | ClassCastException e) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
} 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); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it possible to fix the converter itself as we know it is relevant to list (and map type mentioned in the JIRA)?
This PR addresses the following JIRA entry:
https://issues.apache.org/jira/browse/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. The proposed change allows Avro to detect that the "array" and "list" types are compatible.