-
Notifications
You must be signed in to change notification settings - Fork 3k
[Parquet] Add support to read Parquet files written with old 2-level list structures. #3774
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
Conversation
|
@rdblue @RussellSpitzer can you help review this? Thanks |
parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java
Outdated
Show resolved
Hide resolved
|
3773 is merged, could you rebase? |
b51243f to
222f45e
Compare
@jackye1995 done |
|
LGTM |
|
@jackye1995 @RussellSpitzer @kbendick @rdblue can I get a review on this, thanks |
parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java
Outdated
Show resolved
Hide resolved
…et-thrift. Also rename ParquetSchemaUtil.isListElementType to ParquetSchemaUtil.isOldListElementType
RussellSpitzer
left a comment
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 looks good to me, but I'm not a parquet expert so @rdblue could you also take a quick look before we merge?
parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
Outdated
Show resolved
Hide resolved
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java
Show resolved
Hide resolved
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java
Outdated
Show resolved
Hide resolved
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java
Outdated
Show resolved
Hide resolved
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java
Outdated
Show resolved
Hide resolved
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
Outdated
Show resolved
Hide resolved
| T elementResult = null; | ||
| if (repeatedElement.getFieldCount() > 0) { | ||
| Type elementField = repeatedElement.getType(0); | ||
| if (repeatedElement.isPrimitive() || repeatedElement.asGroupType().getFieldCount() > 0) { |
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 changes to this class are inconsistent with the changes to the TypeWithSchemaVisitor. Here, the repeated element is always visited (beforeRepeatedElement call above) and may be processed again as the element. The other avoids pushing the name on the stack. If beforeRepeatedElement were used to track names, I think it would get a duplicate name for the repeated group.
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, addressed this. It would be nice to have some tests to check this behavior. But, I don't think we need to block on that, unless you disagree.
parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java
Outdated
Show resolved
Hide resolved
parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java
Outdated
Show resolved
Hide resolved
| private static <T> T visitTwoLevelList(Types.ListType iListType, Types.NestedField iListElement, GroupType pListType, | ||
| Type pListElement, TypeWithSchemaVisitor<T> visitor) { | ||
| T elementResult = visitField(iListElement, pListElement, visitor); | ||
|
|
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.
Style: extra newline. Also, we either wrap argument lists at the same level or start all arguments on the next line at 2 indents.
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 style used in various parts of code are different. For example, IIUC ParquetReadSupport.prepareForRead is different than what you are saying. Earlier you also had mentioned Iceberg does not use new param at new line pattern. Updating this part to keep the same level (align params start with previous line) and wrap.
Let me know which style we should try to follow and I can try to update the intellij-style that we provide with Iceberg repo accordingly. I don't know if it is possible, but I can try.
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.
Don't worry about where still is incorrect in other places. We'll eventually track those down and fix them, but we do want to keep style from diverging when it is caught by a review. So please do fix this.
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.
It should already be fixed. Does the update still have style issue?
|
@SinghAsDev, I did another round of review. This also still needs to update Iceberg generics, I think. |
Thanks for thorough reviews @rdblue ! I thought I already covered the Iceberg generic with this change. Am I missing something? |
|
@SinghAsDev, looks like it is missing a test for Iceberg generics. I see the update for it now. Thanks! |
…ment needs to be popped out after visiting list.
|
Thanks, @SinghAsDev! |
Add support to read Parquet files written with old 2-level list structures. This should resolve #3759.