-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
GH-43994: [C++][Parquet] Fix schema conversion from two-level encoding nested list #43995
Conversation
@emkornfield @pitrou @mapleFU Would you mind taking a look? Thanks! |
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists Without legacy:
With backward capability: Some existing data does not include the inner element layer. For backward-compatibility, the type of elements in LIST-annotated structures should always be determined by the following rules:
So, seems this hit the (1)? |
Parquet schema is too tricky for me, I'd try to take a look at https://github.com/apache/parquet-java/blob/aec7bc64dffa373db678ab2fc8b46565b4c011a5/parquet-thrift/src/main/java/org/apache/parquet/thrift/ThriftSchemaConvertVisitor.java#L220 tomorrow... |
I've check Java related code: I'll dive into it this after noon |
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.
Our ListToSchemaField
is like this part of the code https://github.com/apache/parquet-java/blob/aec7bc64dffa373db678ab2fc8b46565b4c011a5/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java#L397-L421
Should we port the impl and testings in that?
IMO, the root cause is that the current code recognizes the schema above as a three-level encoding. However, the inner-most field can only be
It is obvious that
|
Yes. It's so tricky, I think we can just copying the Java code directly, lol |
I think we are just missing check of this line: https://github.com/apache/parquet-java/blob/aec7bc64dffa373db678ab2fc8b46565b4c011a5/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java#L588 |
This fixing itself LGTM, but I just think should we test and align more... |
What is "array"? Do you mean "list"? Can you fix the PR description?
Where is this in the Parquet spec? I cannot find a similar example.
|
The wording of the spec is very ambigious:
I think this just following the rule(4): repeated field's type is the element type with the repeated field's repetition.
I think maybe a testfile would be better |
I‘m using Hive schema, so that's why it is
The parquet-cli prints the following metadata:
|
@wgtmac Would you mind check testing file and add one if not exists in |
I will try to use parquet-java to create a minimal file and add it to parquet-testing. The file created by Hudi is too large due to a file-level bloom filter embedded in the file footer. |
Gentle ping :) @emkornfield @pitrou @mapleFU |
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
9bd3fa4
to
c99cf69
Compare
@emkornfield Thanks for your review! I've rebased it and the test failure in |
ASSERT_OK_NO_THROW( | ||
FileReader::Make(default_memory_pool(), std::move(file_reader), &reader)); | ||
std::shared_ptr<Table> table; | ||
ASSERT_OK(reader->ReadTable(&table)); |
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.
Would have been nice to add a validation call here.
ASSERT_OK(table->ValidateFull());
After merging your PR, Conbench analyzed the 3 benchmarking runs that have been run so far on merge-commit a6fe595. There were 132 benchmark results with an error:
There were no benchmark performance regressions. 🎉 The full Conbench report has more details. It also includes information about 4 possible false positives for unstable benchmarks that are known to sometimes produce them. |
… test (#44847) ### Rationale for this change Add a validate call as suggested by #43995 (comment) ### What changes are included in this PR? Add a table->ValidateFull() to LegacyTwoLevelList test case. ### Are these changes tested? Yes ### Are there any user-facing changes? No Authored-by: Gang Wu <[email protected]> Signed-off-by: Gang Wu <[email protected]>
Rationale for this change
The current C++ parquet implementation interprets following parquet schema as `array<struct<array:array>>, which is wrong:
What changes are included in this PR?
According to the parquet spec, the above schema should be inferred as
array<array<int>>
.Are these changes tested?
Yes, a test case has been added to verify the fix.
Are there any user-facing changes?
No.