-
Notifications
You must be signed in to change notification settings - Fork 3k
Parquet: Implement Variant writers #12323
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
e93c910
d273904
7feb172
03ce5d8
e69ec6a
4bcf388
36a224a
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,43 @@ | ||
| /* | ||
| * 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.variants; | ||
|
|
||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
|
||
| class VariantData implements Variant { | ||
| private final VariantMetadata metadata; | ||
| private final VariantValue value; | ||
|
|
||
| VariantData(VariantMetadata metadata, VariantValue value) { | ||
| Preconditions.checkArgument(metadata != null, "Invalid variant metadata: null"); | ||
| Preconditions.checkArgument(value != null, "Invalid variant value: null"); | ||
| this.metadata = metadata; | ||
| this.value = value; | ||
| } | ||
|
|
||
| @Override | ||
| public VariantMetadata metadata() { | ||
| return metadata; | ||
| } | ||
|
|
||
| @Override | ||
| public VariantValue value() { | ||
| return value; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,85 @@ | ||
| /* | ||
| * 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.variants; | ||
|
|
||
| import java.util.List; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
|
|
||
| public class VariantVisitor<R> { | ||
| public R object(VariantObject object, List<String> fieldNames, List<R> fieldResults) { | ||
| return null; | ||
| } | ||
|
|
||
| public R array(VariantArray array, List<R> elementResults) { | ||
| return null; | ||
| } | ||
|
|
||
| public R primitive(VariantPrimitive<?> primitive) { | ||
| return null; | ||
| } | ||
|
|
||
| public void beforeArrayElement(int index) {} | ||
|
|
||
| public void afterArrayElement(int index) {} | ||
|
|
||
| public void beforeObjectField(String fieldName) {} | ||
|
|
||
| public void afterObjectField(String fieldName) {} | ||
|
|
||
| public static <R> R visit(Variant variant, VariantVisitor<R> visitor) { | ||
| return visit(variant.value(), visitor); | ||
| } | ||
|
|
||
| public static <R> R visit(VariantValue value, VariantVisitor<R> visitor) { | ||
| switch (value.type()) { | ||
| case ARRAY: | ||
| VariantArray array = value.asArray(); | ||
| List<R> elementResults = Lists.newArrayList(); | ||
| for (int index = 0; index < array.numElements(); index += 1) { | ||
| visitor.beforeArrayElement(index); | ||
| try { | ||
| elementResults.add(visit(array.get(index), visitor)); | ||
| } finally { | ||
| visitor.afterArrayElement(index); | ||
| } | ||
| } | ||
|
|
||
| return visitor.array(array, elementResults); | ||
|
|
||
| case OBJECT: | ||
| VariantObject object = value.asObject(); | ||
| List<String> fieldNames = Lists.newArrayList(); | ||
| List<R> fieldResults = Lists.newArrayList(); | ||
| for (String fieldName : object.fieldNames()) { | ||
| fieldNames.add(fieldName); | ||
| visitor.beforeObjectField(fieldName); | ||
| try { | ||
| fieldResults.add(visit(object.get(fieldName), visitor)); | ||
| } finally { | ||
| visitor.afterObjectField(fieldName); | ||
| } | ||
| } | ||
|
|
||
| return visitor.object(object, fieldNames, fieldResults); | ||
|
|
||
| default: | ||
| return visitor.primitive(value.asPrimitive()); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -45,6 +45,16 @@ public static ShreddedObject object(VariantMetadata metadata) { | |
| return new ShreddedObject(metadata); | ||
| } | ||
|
|
||
| public static ShreddedObject object(VariantObject object) { | ||
|
Contributor
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. Used to create a shredded object from an existing object when writing. It uses the object's metadata. This avoids exposing |
||
| if (object instanceof ShreddedObject) { | ||
| return new ShreddedObject(((ShreddedObject) object).metadata(), object); | ||
| } else if (object instanceof SerializedObject) { | ||
| return new ShreddedObject(((SerializedObject) object).metadata(), object); | ||
| } | ||
|
|
||
| throw new UnsupportedOperationException("Metadata is required for object: " + object); | ||
| } | ||
|
|
||
| public static <T> VariantPrimitive<T> of(PhysicalType type, T value) { | ||
| return new PrimitiveWrapper<>(type, value); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,13 +18,17 @@ | |
| */ | ||
| package org.apache.iceberg.data.parquet; | ||
|
|
||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import java.util.Optional; | ||
| import org.apache.iceberg.parquet.ParquetTypeVisitor; | ||
| import org.apache.iceberg.parquet.ParquetValueWriter; | ||
| import org.apache.iceberg.parquet.ParquetValueWriters; | ||
| import org.apache.iceberg.parquet.ParquetVariantVisitor; | ||
| import org.apache.iceberg.parquet.TypeWithSchemaVisitor; | ||
| import org.apache.iceberg.parquet.VariantWriterBuilder; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.parquet.column.ColumnDescriptor; | ||
| import org.apache.parquet.schema.GroupType; | ||
| import org.apache.parquet.schema.LogicalTypeAnnotation; | ||
|
|
@@ -34,9 +38,14 @@ | |
|
|
||
| abstract class BaseParquetWriter<T> { | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| protected ParquetValueWriter<T> createWriter(MessageType type) { | ||
| return (ParquetValueWriter<T>) ParquetTypeVisitor.visit(type, new WriteBuilder(type)); | ||
| return createWriter(null, type); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| protected ParquetValueWriter<T> createWriter(Types.StructType struct, MessageType type) { | ||
| return (ParquetValueWriter<T>) | ||
| TypeWithSchemaVisitor.visit(struct, type, new WriteBuilder(type)); | ||
| } | ||
|
|
||
| protected abstract ParquetValueWriters.StructWriter<T> createStructWriter( | ||
|
|
@@ -62,7 +71,7 @@ protected ParquetValueWriter<?> timestampWriter(ColumnDescriptor desc, boolean i | |
| } | ||
| } | ||
|
|
||
| private class WriteBuilder extends ParquetTypeVisitor<ParquetValueWriter<?>> { | ||
| private class WriteBuilder extends TypeWithSchemaVisitor<ParquetValueWriter<?>> { | ||
|
Contributor
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. In order to detect a variant type, this needs to use the original Iceberg schema. When Parquet exposes the |
||
| private final MessageType type; | ||
|
|
||
| private WriteBuilder(MessageType type) { | ||
|
|
@@ -71,14 +80,14 @@ private WriteBuilder(MessageType type) { | |
|
|
||
| @Override | ||
| public ParquetValueWriter<?> message( | ||
| MessageType message, List<ParquetValueWriter<?>> fieldWriters) { | ||
| Types.StructType struct, MessageType message, List<ParquetValueWriter<?>> fieldWriters) { | ||
|
|
||
| return struct(message.asGroupType(), fieldWriters); | ||
| return struct(struct, message.asGroupType(), fieldWriters); | ||
| } | ||
|
|
||
| @Override | ||
| public ParquetValueWriter<?> struct( | ||
| GroupType struct, List<ParquetValueWriter<?>> fieldWriters) { | ||
| Types.StructType iceberg, GroupType struct, List<ParquetValueWriter<?>> fieldWriters) { | ||
| List<Type> fields = struct.getFields(); | ||
| List<ParquetValueWriter<?>> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); | ||
| for (int i = 0; i < fields.size(); i += 1) { | ||
|
|
@@ -91,7 +100,8 @@ public ParquetValueWriter<?> struct( | |
| } | ||
|
|
||
| @Override | ||
| public ParquetValueWriter<?> list(GroupType array, ParquetValueWriter<?> elementWriter) { | ||
| public ParquetValueWriter<?> list( | ||
| Types.ListType iceberg, GroupType array, ParquetValueWriter<?> elementWriter) { | ||
| GroupType repeated = array.getFields().get(0).asGroupType(); | ||
| String[] repeatedPath = currentPath(); | ||
|
|
||
|
|
@@ -107,7 +117,10 @@ public ParquetValueWriter<?> list(GroupType array, ParquetValueWriter<?> element | |
|
|
||
| @Override | ||
| public ParquetValueWriter<?> map( | ||
| GroupType map, ParquetValueWriter<?> keyWriter, ParquetValueWriter<?> valueWriter) { | ||
| Types.MapType iceberg, | ||
| GroupType map, | ||
| ParquetValueWriter<?> keyWriter, | ||
| ParquetValueWriter<?> valueWriter) { | ||
| GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); | ||
| String[] repeatedPath = currentPath(); | ||
|
|
||
|
|
@@ -127,7 +140,8 @@ public ParquetValueWriter<?> map( | |
| } | ||
|
|
||
| @Override | ||
| public ParquetValueWriter<?> primitive(PrimitiveType primitive) { | ||
| public ParquetValueWriter<?> primitive( | ||
| org.apache.iceberg.types.Type.PrimitiveType iceberg, PrimitiveType primitive) { | ||
| ColumnDescriptor desc = type.getColumnDescription(currentPath()); | ||
| LogicalTypeAnnotation logicalType = primitive.getLogicalTypeAnnotation(); | ||
| if (logicalType != null) { | ||
|
|
@@ -157,6 +171,16 @@ public ParquetValueWriter<?> primitive(PrimitiveType primitive) { | |
| throw new UnsupportedOperationException("Unsupported type: " + primitive); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public ParquetValueWriter<?> variant(Types.VariantType iVariant, ParquetValueWriter<?> result) { | ||
| return result; | ||
| } | ||
|
|
||
| @Override | ||
| public ParquetVariantVisitor<ParquetValueWriter<?>> variantVisitor() { | ||
|
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. We can reduce the scope to protected for now. It doesn't seem to be accessed outside.
Contributor
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. This is part of the Parquet visitor contract now. It needs to be public because that class is public. |
||
| return new VariantWriterBuilder(type, Arrays.asList(currentPath())); | ||
| } | ||
| } | ||
|
|
||
| private class LogicalTypeWriterVisitor | ||
|
|
||
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 same as variant reader PR, we are not including array, right?
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.
That's right.