-
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-1441: SchemaParseException: Can't redefine: list in AvroIndexedRecordConverter #560
Conversation
…edRecordConverter Parquet Avro reader can't convert Parquet such a schema where a group field name is reused in an inner structure. The converter creates Avro record schma in this case, but in Avro record types should have a unique name, therefore the result is an invalid Avro schema. This patch fixes this case by adding a namespace for the record if the name was defined before, this way making the record names unique.
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.
LGTM
Really looking forward for this patch! So I've tested the patch locally, and it fails on following schema for 2nd parquet nested column:
This is the
The exception I'm getting is:
The exception is triggered just after successfully processing the following parquet column (first nested column it encountered):
|
@liorchaga is this something that used to work without this patch? It would be awkward to cause regression. Would you mind demonstrate the failure with a unit test too, or help me how to reproduce? The schema you provided doesn't look familiar for me (one with name pv_events), what kind of schema is it? |
@nandorKollar No, this is not something that used to work. I tried making it work last week with parquet-mr 1.8.2 but got The schema I provided is just the name of a column we have in production data. I will provide a test or reproduction steps tomorrow... Thanks! |
@nandorKollar Used the following code snippet to get avro schema:
part-00000-a13d1582-160e-4e18-9be3-9b178641db76-c000.snappy.parquet.zip |
@liorchaga this particular case I think you should explicitly tell, that the file schema is written using 3-level structure by setting
|
Thanks @nandorKollar . I was under the impression that patch obviates the need for this flag.
parquet schema example:
part-00000-83d942f8-dec0-4420-aaaa-4e3e33ba340a-c000.snappy.parquet.zip |
Strange, I tried, but I didn't get the exception you mentioned. |
Well, just to make sure - this is the code I executed. Now I see it actually failed on the
|
Ok, my bad. If failed on another file with wider schema, containing also this field. I'll check myself again (must be out of focus today) |
Ok, so once I have two upper level arrays, and one of them contains nested arrays, it actually generates a valid schema, but fails on schema.toString() with schema:
file attached. Full stack trace:
|
Also - when I'm running on full schema, same exception occurs on |
Well, turns out that if I just switch the order of columns in the schema, it fails on read. Attached file here: Stack trace:
|
@liorchaga thanks for catching this, I think I found the problem: I used Map#merge wrongly. Pushed a new commit, it should fix this case. |
Superb. I'll test it on full schema again soon |
@nandorKollar fix works likes magic. Thanks! |
Any updates to the state of this pull request? This effects us downstream of Apache Spark, which is at the moment still going through the RC process for a version 2.4.1 release. (This issue broke us as of Spark version 2.4.0) https://issues.apache.org/jira/browse/SPARK-25588 |
@rdblue what do you think, is using Avro namespace to solve this problem the right approach? It looks like name collision in Avro schema (while converting Parquet lists for example, where the group name is the same for embedded list) causes problems like this, and I think Avro namespaces would be a good option to solve issues like this. |
I personally think this fix is superb. Already use it in production with shaded jar including tgis patch, reading and writing very complex parquet schema. |
I think it is okay to use the namespace to solve this problem. The idea is to use the path within the schema as the namespace and the last component as the record name, right? So it would produce list_one.element and list_two.element instead of two records named element? We fixed this in Iceberg by moving to IDs. You could consider doing something similar when there is a field ID defined on the Parquet type. We use |
@rdblue something similar, but the namespace is not the entire path of the record, but simply the record name + a counter. So in your example the namespaces will be element1 and element2, and the two record names are element1.element and element2.element. Namespaces are used only when it is indeed required: when the record name was encountered before, so if there's only one list with element, then no namespace is used. Would you prefer entire path to the record as namespaces? IDs you mean Parquet field IDs? In that case, IDs would give the namespaces for the Avro records right? That sounds also promising for me, however in that case the client should modify the Parquet schema by adding unique ID fields, since AFAIK IDs are optional. |
Sounds reasonable to me. I don't think it matters whether it is a path or a counter. We used field IDs to name the record itself without using namespaces. |
Might it be possible to get this merged and into a bugfix release 1.10.2 soon? If the Spark 2.4.1 RC process drags out a bit longer, it would be nice to get this in. |
I don't think it is necessary to get this into Spark 2.4.1. Spark doesn't use parquet-avro, except in tests. |
Sorry if I sound frustrated, I feel like I keep having to re-explain myself. The versions of Parquet and Avro in Spark are incompatible with each other, and downstream we don't have any way to workaround the conflict. Per https://issues.apache.org/jira/browse/SPARK-25588 We have been struggling with Spark's conflicting Parquet and Avro dependencies for many versions. Our most recent workaround is to pin parquet-avro to version 1.8.1 and exclude all its transitive dependencies. This workaround worked for 2.3.2, thus I gave the last Spark RC a non-binding +1. https://github.com/bigdatagenomics/adam/blob/master/pom.xml#L520 That workaround does not work for 2.4.0, as this pinned version 1.8.1 conflicts at runtime with version 1.10.0 brought in by Spark.
Removing the pinned version and dependency exclusions, bringing the build dependency version to 1.10.0, results in the error reported here in our unit tests under Spark version 2.4.0. I believe this pull request will fix the |
@heuermh, you should be able to shade parquet-avro to avoid that conflict. Have you tried that? |
I've not yet found any combination of dependency exclusion, version overrides, and dependency shading that works, not for lack of trying. Spark 2.4.1 rc8 has failed and there might be a chance to get a PR in rc9 or a later release candidate that updates the Parquet dependency to include this fix. How heavyweight is the release process here for a 1.10.2 release? |
@heuermh, have you tried setting the schemas manually instead of letting Parquet convert them? Unfortunately, I think this needs to wait to be in the 1.11.0 release. 1.10.2 would be a patch release and we want to keep those to just bug fixes. While this is a bug, it can break existing code because it changes the names of structures in the Avro schema. This could cause a problem when reading with a projection schema. Inside of unions, types are matched by name. Because this changes the full name of types, it could cause existing code to not be able to read unions when projecting with an Avro schema. |
Ah right, good point.
It is Spark SQL generating the schemas when saving a Dataset or DataFrame as Parquet; I'm not sure there is a hook to specify an Avro-generated schema there. Will look into it. In any case, I generated and deployed SNAPSHOT builds all the way up our stack from this branch (Parquet → Spark → bdg-utils → ADAM) and it seems to work for us. |
@zivanfi Thank you for the merge! Which Parquet version would this fix be released in? The workaround we had to put in place is very brittle -- it involves a copy of parquet-avro source with modifications in our build, shading of external parquet-avro classes, and a custom maven-shade-plugin Shader that excludes the external parquet-avro copies of the modified classes in our build. |
@heuermh Sorry, I can't tell, unfortunately I don't have too much impact on the release process. |
While trying to read with master branch build, no exceptions are thrown, but the values of repeated array types are set to null. I verified the values indeed exist in parquet file via
Corresponding avro schema:
I have this Hadoop configuration:
@heuermh , is there any property I am missing in config ? |
The issue was the parquet file was written with Parquet version 1.8.1 and avro 1.8.2, while trying to read with the built from master (1.12) it causes extra issues. Maybe backward compatibility is broken.
|
Parquet Avro reader can't convert such Parquet schemas, where a group field name is reused
in an inner structure. The converter creates Avro record schma in this case,
but in Avro record types should have a unique name, therefore the result is an invalid Avro
schema. This patch fixes this case by adding a namespace for the record if the name was
defined before, this way making the record names unique.