-
Notifications
You must be signed in to change notification settings - Fork 3k
Support name mapping resolution for parquet #830
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
Changes from all commits
fe05997
b2f6b96
502b1b9
1fd7d60
d851ad5
1980936
7db31f5
c314524
fd4c7d0
3323ae0
d718765
a12f430
7d2a1dd
0c8b3da
4baa900
d628545
9529f0d
526e79b
a128fa7
18a803f
2f47631
3ca14c7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,110 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.parquet; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Objects; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.iceberg.mapping.MappedField; | ||
| import org.apache.iceberg.mapping.NameMapping; | ||
| import org.apache.parquet.Preconditions; | ||
| import org.apache.parquet.schema.GroupType; | ||
| import org.apache.parquet.schema.MessageType; | ||
| import org.apache.parquet.schema.PrimitiveType; | ||
| import org.apache.parquet.schema.Type; | ||
| import org.apache.parquet.schema.Types; | ||
|
|
||
| class ApplyNameMapping extends ParquetTypeVisitor<Type> { | ||
| private final NameMapping nameMapping; | ||
|
|
||
| ApplyNameMapping(NameMapping nameMapping) { | ||
| this.nameMapping = nameMapping; | ||
| } | ||
|
|
||
| @Override | ||
| public Type message(MessageType message, List<Type> fields) { | ||
| Types.MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); | ||
| fields.stream().filter(Objects::nonNull).forEach(builder::addField); | ||
|
|
||
| return builder.named(message.getName()); | ||
| } | ||
|
|
||
| @Override | ||
| public Type struct(GroupType struct, List<Type> types) { | ||
| MappedField field = nameMapping.find(currentPath()); | ||
| List<Type> actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); | ||
| Type structType = struct.withNewFields(actualTypes); | ||
|
|
||
| return field == null ? structType : structType.withId(field.id()); | ||
| } | ||
|
|
||
| @Override | ||
| public Type list(GroupType list, Type elementType) { | ||
| Preconditions.checkArgument(elementType != null, | ||
| "List type must have element field"); | ||
|
|
||
| MappedField field = nameMapping.find(currentPath()); | ||
| Type listType = org.apache.parquet.schema.Types.list(list.getRepetition()) | ||
| .element(elementType) | ||
| .named(list.getName()); | ||
|
|
||
| return field == null ? listType : listType.withId(field.id()); | ||
| } | ||
|
|
||
| @Override | ||
| public Type map(GroupType map, Type keyType, Type valueType) { | ||
| Preconditions.checkArgument(keyType != null && valueType != null, | ||
| "Map type must have both key field and value field"); | ||
|
|
||
| MappedField field = nameMapping.find(currentPath()); | ||
| Type mapType = org.apache.parquet.schema.Types.map(map.getRepetition()) | ||
| .key(keyType) | ||
| .value(valueType) | ||
| .named(map.getName()); | ||
|
|
||
| return field == null ? mapType : mapType.withId(field.id()); | ||
| } | ||
|
|
||
| @Override | ||
| public Type primitive(PrimitiveType primitive) { | ||
| MappedField field = nameMapping.find(currentPath()); | ||
| return field == null ? primitive : primitive.withId(field.id()); | ||
| } | ||
|
|
||
| @Override | ||
| public void beforeRepeatedElement(Type element) { | ||
| // do not add the repeated element's name | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you add comments to these methods to explain why they are here?
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK, let me add the comments back. |
||
|
|
||
| @Override | ||
| public void afterRepeatedElement(Type element) { | ||
| // do not remove the repeated element's name | ||
| } | ||
|
|
||
| @Override | ||
| public void beforeRepeatedKeyValue(Type keyValue) { | ||
| // do not add the repeated element's name | ||
| } | ||
|
|
||
| @Override | ||
| public void afterRepeatedKeyValue(Type keyValue) { | ||
| // do not remove the repeated element's name | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,12 +19,16 @@ | |
|
|
||
| package org.apache.iceberg.parquet; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Set; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.mapping.NameMapping; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Sets; | ||
| import org.apache.iceberg.types.TypeUtil; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.parquet.schema.GroupType; | ||
| import org.apache.parquet.schema.MessageType; | ||
| import org.apache.parquet.schema.PrimitiveType; | ||
| import org.apache.parquet.schema.Type; | ||
| import org.apache.parquet.schema.Types.MessageTypeBuilder; | ||
|
|
||
|
|
@@ -83,22 +87,7 @@ public static MessageType pruneColumnsFallback(MessageType fileSchema, Schema ex | |
| } | ||
|
|
||
| public static boolean hasIds(MessageType fileSchema) { | ||
| try { | ||
| // Try to convert the type to Iceberg. If an ID assignment is needed, return false. | ||
| ParquetTypeVisitor.visit(fileSchema, new MessageTypeToType(fileSchema) { | ||
| @Override | ||
| protected int nextId() { | ||
| throw new IllegalStateException("Needed to assign ID"); | ||
| } | ||
| }); | ||
|
|
||
| // no assignment was needed | ||
| return true; | ||
|
|
||
| } catch (IllegalStateException e) { | ||
| // at least one field was missing an id. | ||
| return false; | ||
| } | ||
| return ParquetTypeVisitor.visit(fileSchema, new HasIds()); | ||
| } | ||
|
|
||
| public static MessageType addFallbackIds(MessageType fileSchema) { | ||
|
|
@@ -112,4 +101,41 @@ public static MessageType addFallbackIds(MessageType fileSchema) { | |
|
|
||
| return builder.named(fileSchema.getName()); | ||
| } | ||
|
|
||
| public static MessageType applyNameMapping(MessageType fileSchema, NameMapping nameMapping) { | ||
| return (MessageType) ParquetTypeVisitor.visit(fileSchema, new ApplyNameMapping(nameMapping)); | ||
| } | ||
|
|
||
| public static class HasIds extends ParquetTypeVisitor<Boolean> { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks good. |
||
| @Override | ||
| public Boolean message(MessageType message, List<Boolean> fields) { | ||
| return struct(message, fields); | ||
| } | ||
|
|
||
| @Override | ||
| public Boolean struct(GroupType struct, List<Boolean> hasIds) { | ||
| for (Boolean hasId : hasIds) { | ||
| if (hasId) { | ||
| return true; | ||
| } | ||
| } | ||
| return struct.getId() != null; | ||
| } | ||
|
|
||
| @Override | ||
| public Boolean list(GroupType array, Boolean hasId) { | ||
| return hasId || array.getId() != null; | ||
| } | ||
|
|
||
| @Override | ||
| public Boolean map(GroupType map, Boolean keyHasId, Boolean valueHasId) { | ||
| return keyHasId || valueHasId || map.getId() != null; | ||
| } | ||
|
|
||
| @Override | ||
| public Boolean primitive(PrimitiveType primitive) { | ||
| return primitive.getId() != null; | ||
| } | ||
| } | ||
|
|
||
| } | ||
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.
@rdsr, just want to confirm that you're okay with this change? If it is going to cause you problems because you've already deployed it, we can go with the original.