-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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
base: main
Are you sure you want to change the base?
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
cpp/src/parquet/arrow/schema.cc
Outdated
@@ -681,6 +681,10 @@ Status ListToSchemaField(const GroupNode& group, LevelInfo current_levels, | |||
// List of primitive type | |||
RETURN_NOT_OK( | |||
NodeToSchemaField(*list_group.field(0), current_levels, ctx, out, child_field)); | |||
} else if (list_group.field_count() == 1 && list_group.field(0)->is_repeated()) { |
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.
BTW, it looks like the HasStructListName is not correct for _tuple, as it only checks the name ends in _tuple
and not that it is the top level list appended with _tuple
.
cpp/src/parquet/arrow/schema.cc
Outdated
@@ -681,6 +681,10 @@ Status ListToSchemaField(const GroupNode& group, LevelInfo current_levels, | |||
// List of primitive type | |||
RETURN_NOT_OK( | |||
NodeToSchemaField(*list_group.field(0), current_levels, ctx, out, child_field)); | |||
} else if (list_group.field_count() == 1 && list_group.field(0)->is_repeated()) { |
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'm not sure this is correct, or at least the comments above need to be updated to explain the logic further.
Specifially from the exapmles on logical type.md:
// List<OneTuple<String>> (nullable list, non-null elements)
optional group my_list (LIST) {
repeated group array {
required binary str (STRING);
};
}
This seems to imply that despite how the file was written with Avro bindings there should in fact be an intermediate struct and not a llist<list<>>, its not clear to me if this is a bug in the spec, or a bug in Avro java writer implementation.
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.
The rule (3) of backward-compatibility rules is that If the repeated field is a group with one field and is named either array or uses the LIST-annotated group's name with _tuple appended then the repeated type is the element type and elements are required.
. It says that the repeated type is the element type.
optional group my_list (LIST) {
repeated group array {
required binary str (STRING);
};
}
So for the schema you've just mentioned above, its element type is group array { required binary str (STRING); }
which perfectly resolves to OneTuple<String>
.
optional group a (LIST) {
repeated group array (LIST) {
repeated int32 array;
}
}
However, for the schema I've mentioned in this issue, its element type is group array (LIST) { repeated int32 array; }
and it perfectly resolves to List<int32>
according to rule (1) which is If the repeated field is not a group, then its type is the element type and elements are required.
.
The parquet-java implementation has interpreted this case in the same way: https://github.com/apache/parquet-java/blob/42cf31c0fbe4f000d4ddb1e1092c6634989ea3ca/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java#L588
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 have opened apache/parquet-format#466 to clarify things.
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.
Commented, its not clear that this is correct, and the bug might be with Avro writer if I am reading the spec correctly.
::std::string_view name{node.name()}; | ||
return name == "array" || EndsWith(name, "_tuple"); | ||
return name == "array" || name == (parent.name() + "_tuple"); |
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.
@emkornfield Fix the matching of _tuple
to follow the spec.
RETURN_NOT_OK(GroupToStruct(list_group, current_levels, ctx, out, child_field)); | ||
} else if (list_group.field_count() == 1) { | ||
const auto& repeated_field = list_group.field(0); | ||
if (repeated_field->is_repeated()) { |
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.
Thanks for the careful explanation on my last questions, after rereading I think I mostly agree that this is a bug that needs to be fixed. I think I missed the second (LIST) annotation. Which even though this check corresponds to the java, code, it seems an important factor here is that the list_group is in fact a (LIST and maybe even possibly a map), not that the inner element is repeated?
So I think the logic might make more sense as (pseudocode):
if (list_group.field_count() > 1) {
...
} else if (HasListElementName(list_group, group)) {
if (IsMap(list_group)) {
RETURN_NOT_OK(
ListToMapField(*list_group, current_levels, ctx, out, child_field));
} else if IsList(list_group)) {
RETURN_NOT_OK(
ListToSchemaField(*list_group, current_levels, ctx, out, child_field));
} else {
RETURN_NOT_OK(GroupToStruct(list_group, current_levels, ctx, out, child_field));
}
} else {
...
}
Does this formulation work?
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.
No, this does not work. It has two issues:
- MAP annotation will never happen. I have explained at GH-465: Clarify backward-compatibility rules on LIST type parquet-format#466 (comment). Same for the non-legacy three-level list.
ListToSchemaField()
will throw atarrow/cpp/src/parquet/arrow/schema.cc
Line 635 in 3917b60
return Status::Invalid("LIST-annotated groups must not be repeated.");
@@ -727,6 +780,60 @@ TEST_F(TestConvertParquetSchema, ParquetRepeatedNestedSchema) { | |||
ASSERT_NO_FATAL_FAILURE(CheckFlatSchema(arrow_schema)); | |||
} | |||
|
|||
TEST_F(TestConvertParquetSchema, IllegalParquetNestedSchema) { |
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.
This case verifies that three-level list and map cannot be nested in a legacy two-level list.
cpp/src/parquet/arrow/schema.cc
Outdated
// The Parquet spec requires that LIST-annotated group cannot be repeated when | ||
// it applies normal three-level encoding. We need to figure out legacy list | ||
// structures and do not enforce this rule for them. | ||
bool is_legacy_list_structure = true; |
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.
Now I have changed the repetition check for only non-legacy list.
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.