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
130 changes: 130 additions & 0 deletions core/src/main/java/org/apache/iceberg/variants/ValueArray.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* 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.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.List;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;

public class ValueArray implements VariantArray {
private SerializationState serializationState = null;
private List<VariantValue> elements = Lists.newArrayList();

ValueArray() {}

@Override
public VariantValue get(int index) {
return elements.get(index);
}

@Override
public int numElements() {
return elements.size();
}

public void add(VariantValue value) {
elements.add(value);
this.serializationState = null;
}

@Override
public int sizeInBytes() {
if (null == serializationState) {
this.serializationState = new SerializationState(elements);
}

return serializationState.size();
}

@Override
public int writeTo(ByteBuffer buffer, int offset) {
Preconditions.checkArgument(
buffer.order() == ByteOrder.LITTLE_ENDIAN, "Invalid byte order: big endian");

if (null == serializationState) {
this.serializationState = new SerializationState(elements);
}

return serializationState.writeTo(buffer, offset);
}

/** Common state for {@link #size()} and {@link #writeTo(ByteBuffer, int)} */
private static class SerializationState {
private final List<VariantValue> elements;
private final int numElements;
private final boolean isLarge;
private final int dataSize;
private final int offsetSize;

private SerializationState(List<VariantValue> elements) {
this.elements = elements;
this.numElements = elements.size();
this.isLarge = numElements > 0xFF;

int totalDataSize = 0;
for (VariantValue value : elements) {
totalDataSize += value.sizeInBytes();
}

this.dataSize = totalDataSize;
this.offsetSize = VariantUtil.sizeOf(totalDataSize);
}

private int size() {
return 1 /* header */
+ (isLarge ? 4 : 1) /* num elements size */
+ (1 + numElements) * offsetSize /* offset list size */
+ dataSize;
}

private int writeTo(ByteBuffer buffer, int offset) {
int offsetListOffset =
offset + 1 /* header size */ + (isLarge ? 4 : 1) /* num elements size */;
int dataOffset = offsetListOffset + ((1 + numElements) * offsetSize);
byte header = VariantUtil.arrayHeader(isLarge, offsetSize);

VariantUtil.writeByte(buffer, header, offset);
VariantUtil.writeLittleEndianUnsigned(buffer, numElements, offset + 1, isLarge ? 4 : 1);

// Insert element offsets
int nextValueOffset = 0;
int index = 0;
for (VariantValue element : elements) {
// write the data offset
VariantUtil.writeLittleEndianUnsigned(
buffer, nextValueOffset, offsetListOffset + (index * offsetSize), offsetSize);

// write the data
int valueSize = element.writeTo(buffer, dataOffset + nextValueOffset);

nextValueOffset += valueSize;
index += 1;
}

// write the final size of the data section
VariantUtil.writeLittleEndianUnsigned(
buffer, nextValueOffset, offsetListOffset + (index * offsetSize), offsetSize);

// return the total size
return (dataOffset - offset) + dataSize;
}
}
}
4 changes: 4 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 @@ -121,6 +121,10 @@ public static boolean isNull(ByteBuffer valueBuffer) {
return VariantUtil.readByte(valueBuffer, 0) == 0;
}

public static ValueArray array() {
return new ValueArray();
}

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 @@ -217,11 +217,12 @@ public void testPartiallyShreddedObjectSerializationLargeBuffer() {
.isEqualTo(DateTimeUtil.isoDateToDays("2024-10-12"));
}

@Test
public void testTwoByteOffsets() {
// a string larger than 255 bytes to push the value offset size above 1 byte
String randomString = RandomUtil.generateString(300, random);
SerializedPrimitive bigString = VariantTestUtil.createString(randomString);
@ParameterizedTest
@ValueSource(ints = {300, 70_000, 16_777_300})
public void testMultiByteOffsets(int len) {
// Use a string exceeding 255 bytes to test value offset sizes of 2, 3, and 4 bytes
String randomString = RandomUtil.generateString(len, random);
VariantPrimitive<String> bigString = Variants.of(randomString);

Map<String, VariantValue> data = Maps.newHashMap();
data.putAll(FIELDS);
Expand All @@ -244,60 +245,6 @@ public void testTwoByteOffsets() {
assertThat(object.get("big").asPrimitive().get()).isEqualTo(randomString);
}

@Test
public void testThreeByteOffsets() {
// a string larger than 65535 bytes to push the value offset size above 2 bytes
String randomString = RandomUtil.generateString(70_000, random);
SerializedPrimitive reallyBigString = VariantTestUtil.createString(randomString);

Map<String, VariantValue> data = Maps.newHashMap();
data.putAll(FIELDS);
data.put("really-big", reallyBigString);

ShreddedObject shredded = createShreddedObject(data);
VariantValue value = roundTripLargeBuffer(shredded, shredded.metadata());

assertThat(value.type()).isEqualTo(PhysicalType.OBJECT);
SerializedObject object = (SerializedObject) value;
assertThat(object.numFields()).isEqualTo(4);

assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT32);
assertThat(object.get("a").asPrimitive().get()).isEqualTo(34);
assertThat(object.get("b").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("b").asPrimitive().get()).isEqualTo("iceberg");
assertThat(object.get("c").type()).isEqualTo(PhysicalType.DECIMAL4);
assertThat(object.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21"));
assertThat(object.get("really-big").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("really-big").asPrimitive().get()).isEqualTo(randomString);
}

@Test
public void testFourByteOffsets() {
// a string larger than 16777215 bytes to push the value offset size above 3 bytes
String randomString = RandomUtil.generateString(16_777_300, random);
SerializedPrimitive reallyBigString = VariantTestUtil.createString(randomString);

Map<String, VariantValue> data = Maps.newHashMap();
data.putAll(FIELDS);
data.put("really-big", reallyBigString);

ShreddedObject shredded = createShreddedObject(data);
VariantValue value = roundTripLargeBuffer(shredded, shredded.metadata());

assertThat(value.type()).isEqualTo(PhysicalType.OBJECT);
SerializedObject object = (SerializedObject) value;
assertThat(object.numFields()).isEqualTo(4);

assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT32);
assertThat(object.get("a").asPrimitive().get()).isEqualTo(34);
assertThat(object.get("b").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("b").asPrimitive().get()).isEqualTo("iceberg");
assertThat(object.get("c").type()).isEqualTo(PhysicalType.DECIMAL4);
assertThat(object.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21"));
assertThat(object.get("really-big").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("really-big").asPrimitive().get()).isEqualTo(randomString);
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
@SuppressWarnings({"unchecked", "rawtypes"})
Expand Down
166 changes: 166 additions & 0 deletions core/src/test/java/org/apache/iceberg/variants/TestValueArray.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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 static org.assertj.core.api.Assertions.assertThat;

import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.List;
import java.util.Random;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.util.RandomUtil;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;

public class TestValueArray {
private static final VariantMetadata EMPTY_METADATA = Variants.emptyMetadata();
private static final List<VariantValue> ELEMENTS =
ImmutableList.of(
Variants.of(34), Variants.of("iceberg"), Variants.of(new BigDecimal("12.21")));

private final Random random = new Random(871925);

@Test
public void testElementAccess() {
ValueArray arr = createArray(ELEMENTS);

assertThat(arr.numElements()).isEqualTo(3);
assertThat(arr.get(0)).isInstanceOf(VariantPrimitive.class);
assertThat(arr.get(0).asPrimitive().get()).isEqualTo(34);
assertThat(arr.get(1)).isInstanceOf(VariantPrimitive.class);
assertThat(arr.get(1).asPrimitive().get()).isEqualTo("iceberg");
assertThat(arr.get(2)).isInstanceOf(VariantPrimitive.class);
assertThat(arr.get(2).asPrimitive().get()).isEqualTo(new BigDecimal("12.21"));
}

@Test
public void testSerializationMinimalBuffer() {
ValueArray arr = createArray(ELEMENTS);

VariantValue value = roundTripMinimalBuffer(arr);

assertThat(value).isInstanceOf(SerializedArray.class);
SerializedArray actual = (SerializedArray) value;

assertThat(actual.numElements()).isEqualTo(3);
assertThat(actual.get(0)).isInstanceOf(VariantPrimitive.class);
assertThat(actual.get(0).asPrimitive().get()).isEqualTo(34);
assertThat(actual.get(1)).isInstanceOf(VariantPrimitive.class);
assertThat(actual.get(1).asPrimitive().get()).isEqualTo("iceberg");
assertThat(actual.get(2)).isInstanceOf(VariantPrimitive.class);
assertThat(actual.get(2).asPrimitive().get()).isEqualTo(new BigDecimal("12.21"));
}

@Test
public void testSerializationLargeBuffer() {
ValueArray arr = createArray(ELEMENTS);

VariantValue value = roundTripLargeBuffer(arr);

assertThat(value).isInstanceOf(SerializedArray.class);
SerializedArray actual = (SerializedArray) value;

assertThat(actual.numElements()).isEqualTo(3);
assertThat(actual.get(0)).isInstanceOf(VariantPrimitive.class);
assertThat(actual.get(0).asPrimitive().get()).isEqualTo(34);
assertThat(actual.get(1)).isInstanceOf(VariantPrimitive.class);
assertThat(actual.get(1).asPrimitive().get()).isEqualTo("iceberg");
assertThat(actual.get(2)).isInstanceOf(VariantPrimitive.class);
assertThat(actual.get(2).asPrimitive().get()).isEqualTo(new BigDecimal("12.21"));
}

@ParameterizedTest
@ValueSource(ints = {300, 70_000, 16_777_300})
public void testMultiByteOffsets(int len) {
// Use a string exceeding 255 bytes to test value offset sizes of 2, 3, and 4 bytes
String randomString = RandomUtil.generateString(len, random);
VariantPrimitive<String> bigString = Variants.of(randomString);

List<VariantValue> data = Lists.newArrayList();
data.addAll(ELEMENTS);
data.add(bigString);

ValueArray shredded = createArray(data);
VariantValue value = roundTripLargeBuffer(shredded);

assertThat(value.type()).isEqualTo(PhysicalType.ARRAY);
SerializedArray actualArray = (SerializedArray) value;
assertThat(actualArray.numElements()).isEqualTo(4);

assertThat(actualArray.get(0).type()).isEqualTo(PhysicalType.INT32);
assertThat(actualArray.get(0).asPrimitive().get()).isEqualTo(34);
assertThat(actualArray.get(1).type()).isEqualTo(PhysicalType.STRING);
assertThat(actualArray.get(1).asPrimitive().get()).isEqualTo("iceberg");
assertThat(actualArray.get(2).type()).isEqualTo(PhysicalType.DECIMAL4);
assertThat(actualArray.get(2).asPrimitive().get()).isEqualTo(new BigDecimal("12.21"));
assertThat(actualArray.get(3).type()).isEqualTo(PhysicalType.STRING);
assertThat(actualArray.get(3).asPrimitive().get()).isEqualTo(randomString);
}

@Test
public void testLargeArray() {
List<VariantValue> elements = Lists.newArrayList();
for (int i = 0; i < 10_000; i += 1) {
elements.add(Variants.of(RandomUtil.generateString(10, random)));
}

ValueArray arr = createArray(elements);
VariantValue value = roundTripLargeBuffer(arr);

assertThat(value.type()).isEqualTo(PhysicalType.ARRAY);
SerializedArray actualArray = (SerializedArray) value;
assertThat(actualArray.numElements()).isEqualTo(10_000);

for (int i = 0; i < 10_000; i++) {
VariantTestUtil.assertEqual(elements.get(i), actualArray.get(i));
}
}

private static VariantValue roundTripMinimalBuffer(ValueArray arr) {
ByteBuffer serialized = ByteBuffer.allocate(arr.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN);
arr.writeTo(serialized, 0);

return Variants.value(EMPTY_METADATA, serialized);
}

private static VariantValue roundTripLargeBuffer(ValueArray arr) {
ByteBuffer serialized =
ByteBuffer.allocate(1000 + arr.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN);
arr.writeTo(serialized, 300);

ByteBuffer slice = serialized.duplicate().order(ByteOrder.LITTLE_ENDIAN);
slice.position(300);
slice.limit(300 + arr.sizeInBytes());

return Variants.value(EMPTY_METADATA, slice);
}

private static ValueArray createArray(List<VariantValue> elements) {
ValueArray arr = new ValueArray();
for (VariantValue element : elements) {
arr.add(element);
}

return arr;
}
}
Loading
Loading