Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 1 addition & 11 deletions api/src/main/java/org/apache/iceberg/variants/Variant.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,17 +29,7 @@ public interface Variant {
VariantValue value();

static Variant of(VariantMetadata metadata, VariantValue value) {
return new Variant() {
@Override
public VariantMetadata metadata() {
return metadata;
}

@Override
public VariantValue value() {
return value;
}
};
return new VariantData(metadata, value);
}

static Variant from(ByteBuffer buffer) {
Expand Down
43 changes: 43 additions & 0 deletions api/src/main/java/org/apache/iceberg/variants/VariantData.java
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
Expand Up @@ -46,11 +46,11 @@ public class ShreddedObject implements VariantObject {
private SerializationState serializationState = null;

ShreddedObject(VariantMetadata metadata) {
this.metadata = metadata;
this.unshredded = null;
this(metadata, null);
}

ShreddedObject(VariantMetadata metadata, VariantObject unshredded) {
Preconditions.checkArgument(metadata != null, "Invalid metadata: null");
this.metadata = metadata;
this.unshredded = unshredded;
}
Expand Down
85 changes: 85 additions & 0 deletions core/src/main/java/org/apache/iceberg/variants/VariantVisitor.java
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;
Copy link
Contributor

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's right.


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());
}
}
}
10 changes: 10 additions & 0 deletions core/src/main/java/org/apache/iceberg/variants/Variants.java
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,16 @@ public static ShreddedObject object(VariantMetadata metadata) {
return new ShreddedObject(metadata);
}

public static ShreddedObject object(VariantObject object) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 VariantObject.metadata because metadata is carried by Variant instead of values.

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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import org.apache.iceberg.data.Record;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.variants.Variant;
import org.apache.iceberg.variants.VariantTestUtil;

public class InternalTestHelpers {

Expand Down Expand Up @@ -106,6 +108,12 @@ private static void assertEquals(Type type, Object expected, Object actual) {
assertThat(actual).as("Actual should be a Map").isInstanceOf(Map.class);
assertEquals(type.asMapType(), (Map<?, ?>) expected, (Map<?, ?>) actual);
break;
case VARIANT:
assertThat(expected).as("Expected should be a Variant").isInstanceOf(Variant.class);
assertThat(actual).as("Actual should be a Variant").isInstanceOf(Variant.class);
VariantTestUtil.assertEqual(((Variant) expected).metadata(), ((Variant) actual).metadata());
VariantTestUtil.assertEqual(((Variant) expected).value(), ((Variant) actual).value());
break;
default:
throw new IllegalArgumentException("Not a supported type: " + type);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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(
Expand All @@ -62,7 +71,7 @@ protected ParquetValueWriter<?> timestampWriter(ColumnDescriptor desc, boolean i
}
}

private class WriteBuilder extends ParquetTypeVisitor<ParquetValueWriter<?>> {
private class WriteBuilder extends TypeWithSchemaVisitor<ParquetValueWriter<?>> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 VARIANT logical type annotation, we can update this to no longer require the schema.

private final MessageType type;

private WriteBuilder(MessageType type) {
Expand All @@ -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) {
Expand All @@ -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();

Expand All @@ -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();

Expand All @@ -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) {
Expand Down Expand Up @@ -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() {
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.iceberg.parquet.ParquetValueWriters;
import org.apache.iceberg.parquet.ParquetValueWriters.StructWriter;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.schema.MessageType;

Expand All @@ -38,9 +39,14 @@ public class InternalWriter<T extends StructLike> extends BaseParquetWriter<T> {

private InternalWriter() {}

@SuppressWarnings("unchecked")
public static <T extends StructLike> ParquetValueWriter<T> create(MessageType type) {
return (ParquetValueWriter<T>) INSTANCE.createWriter(type);
return create(null, type);
}

@SuppressWarnings("unchecked")
public static <T extends StructLike> ParquetValueWriter<T> create(
Types.StructType struct, MessageType type) {
return (ParquetValueWriter<T>) INSTANCE.createWriter(struct, type);
}

@Override
Expand Down
Loading