Skip to content

Commit

Permalink
Add tests for Parquet variant writers.
Browse files Browse the repository at this point in the history
  • Loading branch information
rdblue committed Feb 20, 2025
1 parent d4c4556 commit 360f531
Show file tree
Hide file tree
Showing 13 changed files with 578 additions and 28 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -47,11 +47,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
86 changes: 86 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,86 @@
/*
*
* * 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.commons.compress.utils.Lists;

public class VariantVisitor<R> {
public R object(VariantObject object, 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<R> fieldResults = Lists.newArrayList();
for (String fieldName : object.fieldNames()) {
visitor.beforeObjectField(fieldName);
try {
fieldResults.add(visit(object.get(fieldName), visitor));
} finally {
visitor.afterObjectField(fieldName);
}
}

return visitor.object(object, 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 @@ -85,6 +85,16 @@ public static ShreddedObject object(VariantMetadata metadata) {
return new ShreddedObject(metadata);
}

public static ShreddedObject object(VariantObject object) {
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 @@ -24,6 +24,8 @@
import java.util.Map;
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 @@ -103,6 +105,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 @@ -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
15 changes: 12 additions & 3 deletions parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,7 @@
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.Type;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -160,9 +161,10 @@ public static class WriteBuilder implements InternalData.WriteBuilder {
private final Map<String, String> metadata = Maps.newLinkedHashMap();
private final Map<String, String> config = Maps.newLinkedHashMap();
private Schema schema = null;
private BiFunction<Integer, String, Type> variantShreddingFunc = null;
private String name = "table";
private WriteSupport<?> writeSupport = null;
private Function<MessageType, ParquetValueWriter<?>> createWriterFunc = null;
private BiFunction<Schema, MessageType, ParquetValueWriter<?>> createWriterFunc = null;
private MetricsConfig metricsConfig = MetricsConfig.getDefault();
private ParquetFileWriter.Mode writeMode = ParquetFileWriter.Mode.CREATE;
private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
Expand Down Expand Up @@ -192,6 +194,11 @@ public WriteBuilder schema(Schema newSchema) {
return this;
}

public WriteBuilder variantShreddingFunc(BiFunction<Integer, String, Type> func) {
this.variantShreddingFunc = func;
return this;
}

@Override
public WriteBuilder named(String newName) {
this.name = newName;
Expand Down Expand Up @@ -222,7 +229,7 @@ public WriteBuilder meta(String property, String value) {

public WriteBuilder createWriterFunc(
Function<MessageType, ParquetValueWriter<?>> newCreateWriterFunc) {
this.createWriterFunc = newCreateWriterFunc;
this.createWriterFunc = (schema, type) -> newCreateWriterFunc.apply(type);
return this;
}

Expand Down Expand Up @@ -292,6 +299,7 @@ private void setBloomFilterConfig(

Map<Integer, String> fieldIdToParquetPath =
parquetSchema.getColumns().stream()
.filter(col -> col.getPrimitiveType().getId() != null)
.collect(
Collectors.toMap(
col -> col.getPrimitiveType().getId().intValue(),
Expand Down Expand Up @@ -362,7 +370,7 @@ public <D> FileAppender<D> build() throws IOException {
}

set("parquet.avro.write-old-list-structure", "false");
MessageType type = ParquetSchemaUtil.convert(schema, name);
MessageType type = ParquetSchemaUtil.convert(schema, name, variantShreddingFunc);

FileEncryptionProperties fileEncryptionProperties = null;
if (fileEncryptionKey != null) {
Expand Down Expand Up @@ -406,6 +414,7 @@ public <D> FileAppender<D> build() throws IOException {
conf,
file,
schema,
type,
rowGroupSize,
metadata,
createWriterFunc,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiFunction;
import java.util.function.Function;
import org.apache.iceberg.Schema;
import org.apache.iceberg.mapping.NameMapping;
Expand All @@ -42,6 +43,11 @@ public static MessageType convert(Schema schema, String name) {
return new TypeToMessageType().convert(schema, name);
}

public static MessageType convert(
Schema schema, String name, BiFunction<Integer, String, Type> variantShreddingFunc) {
return new TypeToMessageType(variantShreddingFunc).convert(schema, name);
}

/**
* Converts a Parquet schema to an Iceberg schema. Fields without IDs are kept and assigned
* fallback IDs.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,10 @@ public static UnboxedWriter<Short> shorts(ColumnDescriptor desc) {
return new ShortWriter(desc);
}

public static <T> ParquetValueWriter<T> unboxed(ColumnDescriptor desc) {
return new UnboxedWriter<>(desc);
}

public static UnboxedWriter<Integer> ints(ColumnDescriptor desc) {
return new UnboxedWriter<>(desc);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ static ParquetValueWriter<VariantValue> objects(
int valueDefinitionLevel,
ParquetValueWriter<?> valueWriter,
int typedDefinitionLevel,
int fieldDefinitionLevel,
List<String> fieldNames,
List<ParquetValueWriter<?>> fieldWriters) {
ImmutableMap.Builder<String, ParquetValueWriter<VariantValue>> builder = ImmutableMap.builder();
Expand All @@ -92,6 +93,7 @@ static ParquetValueWriter<VariantValue> objects(
valueDefinitionLevel,
(ParquetValueWriter<VariantValue>) valueWriter,
typedDefinitionLevel,
fieldDefinitionLevel,
builder.build());
}

Expand Down Expand Up @@ -128,7 +130,7 @@ public void setColumnStore(ColumnWriteStore columnStore) {

private abstract static class VariantBinaryWriter<T> implements ParquetValueWriter<T> {
private final ParquetValueWriter<ByteBuffer> bytesWriter;
private ByteBuffer reusedBuffer = ByteBuffer.allocate(2048);
private ByteBuffer reusedBuffer = ByteBuffer.allocate(2048).order(ByteOrder.LITTLE_ENDIAN);

private VariantBinaryWriter(ParquetValueWriter<ByteBuffer> bytesWriter) {
this.bytesWriter = bytesWriter;
Expand Down Expand Up @@ -157,6 +159,8 @@ private void ensureCapacity(int requiredSize) {
if (reusedBuffer.capacity() < requiredSize) {
int newCapacity = capacityFor(requiredSize);
this.reusedBuffer = ByteBuffer.allocate(newCapacity).order(ByteOrder.LITTLE_ENDIAN);
} else {
reusedBuffer.limit(requiredSize);
}
}

Expand Down Expand Up @@ -282,17 +286,20 @@ private static class ShreddedObjectWriter implements ParquetValueWriter<VariantV
private final int valueDefinitionLevel;
private final ParquetValueWriter<VariantValue> valueWriter;
private final int typedDefinitionLevel;
private final int fieldDefinitionLevel;
private final Map<String, ParquetValueWriter<VariantValue>> typedWriters;
private final List<TripleWriter<?>> children;

private ShreddedObjectWriter(
int valueDefinitionLevel,
ParquetValueWriter<VariantValue> valueWriter,
int typedDefinitionLevel,
int fieldDefinitionLevel,
Map<String, ParquetValueWriter<VariantValue>> typedWriters) {
this.valueDefinitionLevel = valueDefinitionLevel;
this.valueWriter = valueWriter;
this.typedDefinitionLevel = typedDefinitionLevel;
this.fieldDefinitionLevel = fieldDefinitionLevel;
this.typedWriters = typedWriters;
this.children =
children(
Expand All @@ -305,14 +312,14 @@ public void write(int repetitionLevel, VariantValue value) {
if (value.type() != PhysicalType.OBJECT) {
valueWriter.write(repetitionLevel, value);

// write null for all fields
// write null for the typed_value group
for (ParquetValueWriter<?> writer : typedWriters.values()) {
writeNull(writer, repetitionLevel, typedDefinitionLevel);
}

} else {
VariantObject object = value.asObject();
ShreddedObject shredded = Variants.object(null, object);
ShreddedObject shredded = Variants.object(object);
for (Map.Entry<String, ParquetValueWriter<VariantValue>> entry : typedWriters.entrySet()) {
String fieldName = entry.getKey();
ParquetValueWriter<VariantValue> writer = entry.getValue();
Expand All @@ -324,7 +331,7 @@ public void write(int repetitionLevel, VariantValue value) {
writer.write(repetitionLevel, fieldValue);
} else {
// missing: write null to both value and typed_value
writeNull(writer, repetitionLevel, typedDefinitionLevel);
writeNull(writer, repetitionLevel, fieldDefinitionLevel);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import java.io.UncheckedIOException;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.function.BiFunction;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.MetricsConfig;
Expand Down Expand Up @@ -75,9 +75,10 @@ class ParquetWriter<T> implements FileAppender<T>, Closeable {
Configuration conf,
OutputFile output,
Schema schema,
MessageType parquetSchema,
long rowGroupSize,
Map<String, String> metadata,
Function<MessageType, ParquetValueWriter<?>> createWriterFunc,
BiFunction<Schema, MessageType, ParquetValueWriter<?>> createWriterFunc,
CompressionCodecName codec,
ParquetProperties properties,
MetricsConfig metricsConfig,
Expand All @@ -88,8 +89,8 @@ class ParquetWriter<T> implements FileAppender<T>, Closeable {
this.metadata = ImmutableMap.copyOf(metadata);
this.compressor =
new ParquetCodecFactory(conf, props.getPageSizeThreshold()).getCompressor(codec);
this.parquetSchema = ParquetSchemaUtil.convert(schema, "table");
this.model = (ParquetValueWriter<T>) createWriterFunc.apply(parquetSchema);
this.parquetSchema = parquetSchema;
this.model = (ParquetValueWriter<T>) createWriterFunc.apply(schema, parquetSchema);
this.metricsConfig = metricsConfig;
this.columnIndexTruncateLength =
conf.getInt(COLUMN_INDEX_TRUNCATE_LENGTH, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH);
Expand Down
Loading

0 comments on commit 360f531

Please sign in to comment.