Skip to content

PARQUET-2450: Fix Avro projection for single-field repeated record types#1300

Merged
wgtmac merged 3 commits intoapache:masterfrom
clairemcginty:parquet-2450
Mar 19, 2024
Merged

PARQUET-2450: Fix Avro projection for single-field repeated record types#1300
wgtmac merged 3 commits intoapache:masterfrom
clairemcginty:parquet-2450

Conversation

@clairemcginty
Copy link
Copy Markdown
Contributor

Jira

Tests

  • My PR adds the following unit tests OR does not need testing for this extremely good reason:

Commits

  • My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines
    from "How to write a good git commit message":
    1. Subject is separated from body by a blank line
    2. Subject is limited to 50 characters (not including Jira issue reference)
    3. Subject does not end with a period
    4. Subject uses the imperative mood ("add", not "adding")
    5. Body wraps at 72 characters
    6. Body explains "what" and "why", not "how"

Style

  • My contribution adheres to the code style guidelines and Spotless passes.
    • To apply the necessary changes, run mvn spotless:apply -Pvector-plugins

Documentation

  • In case of new functionality, my PR adds documentation that describes how to use it.
    • All the public functions and the classes in the PR contain Javadoc that explain what it does

static boolean isElementType(Type repeatedType, Schema elementSchema) {
if (repeatedType.isPrimitive()
|| repeatedType.asGroupType().getFieldCount() > 1
|| repeatedType.getName().equals("array")
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This now matches the List logical-type spec:

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.

as well as the equivalent isElementType check in AvroSchemaConverter: https://github.com/apache/parquet-mr/blob/61bdf44/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java#L575-L584

Note: We don't have to add another clause to check for the _tuple style naming scheme, as that's already covered by line 934: repeatedType.asGroupType().getType(0).isRepetition(REPEATED)).

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

1. If the repeated field is not a group, then its type is the element type and elements are required.
2. If the repeated field is a group with multiple fields, then its type is the element type and elements are required.
3. 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.
4. Otherwise, the repeated field's type is the element type with the repeated field's repetition.

repeatedType.getName().equals("array") is partially for rule 3, and repeatedType.asGroupType().getType(0).isRepetition(REPEATED)) is for rule 4. But rule 3 also states the _tuple style naming scheme should come with the repeated field is a group with one field, isn't it? If the repeated field is a group with multiple fields with _tuple appended, is it still a valid element type?

Copy link
Copy Markdown
Contributor Author

@clairemcginty clairemcginty Mar 15, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh interesting, so a schema like this?

message MySchema {
  optional group list_field (LIST) {
    repeated group list_field_tuple_1 (LIST) {
      repeated int32 int_field_1;
    }
    repeated group list_field_tuple_2 (LIST) {
      repeated int32 int_field_2
    }
  }
}

I think luckily, it's impossible to create this in Avro. I tried running it through AvroSchemaConverter, and it threw an exception:

 > MessageType parquetSchema = MessageTypeParser.parseMessageType(
        "message MySchema {\n"
        + "  optional group list_field (LIST) {\n"
        + "    repeated group list_field_tuple_1 (LIST) {\n"
        + "      repeated int32 int_field_1;\n"
        + "    }\n"
        + "    repeated group list_field_tuple_2 (LIST) {\n"
        + "      repeated int32 int_field_2;\n"
        + "    }\n"
        + "  }\n"
        + "}");

> new AvroSchemaConverter().convert(parquetSchema));
java.lang.UnsupportedOperationException: Invalid list type optional group list_field (LIST) {
  repeated group list_field_tuple_1 (LIST) {
    required int32 int_field_1;
  }
  repeated group list_field_tuple_2 (LIST) {
    required int32 int_field_2;
  }
}

Path test = writeDirect(
"message AvroCompatOptionalGroupInListWithSchema {" + " optional group locations (LIST) {"
+ " repeated group array {"
+ " repeated group my_list {"
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test started failing after I made the change to AvroRecordConverter. Looking at it, this schema didn't match the logical type spec for List types -- any repeated group named array must contain element type fields directly. So I updated the schema naming.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with you. Is it better to add a test case with the incorrect schema and check it throws?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sounds good! added.

Copy link
Copy Markdown
Member

@wgtmac wgtmac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the detail test!

LGTM

@wgtmac wgtmac merged commit 30fa1de into apache:master Mar 19, 2024
@asuresh8
Copy link
Copy Markdown

What's the timeline on getting a version of parquet-mr with this change published to Maven? When will 1.14.0 be release? Can this be backported 1.13?

@wgtmac
Copy link
Copy Markdown
Member

wgtmac commented Apr 10, 2024

Yes, I expect this will be included in the 1.14.0 release which is targeted in the mid 2024.

@asuresh8
Copy link
Copy Markdown

Can this be added as a patch to 1.13? This is currently affecting our customers so happy to provide any support on our end to expedite the release of this fix.

@wgtmac
Copy link
Copy Markdown
Member

wgtmac commented Apr 10, 2024

I don't think 1.13.2 is planned. Could you try to reply to this ML thread to expedite the 1.14.0 release: https://lists.apache.org/thread/kttwbl5l7opz6nwb5bck2gghc2y3td0o ?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants