Skip to content

Use short string in Variant when possible #13284

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

Open
wants to merge 6 commits into
base: main
Choose a base branch
from
Open
Changes from 4 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
Original file line number Diff line number Diff line change
@@ -100,16 +100,22 @@ public void testStringDifferentLengths() {
assertThat(array.numElements()).isEqualTo(6);
assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(0).asPrimitive().get()).isEqualTo("a");
assertThat(array.get(0).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(1).asPrimitive().get()).isEqualTo("b");
assertThat(array.get(1).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(2).asPrimitive().get()).isEqualTo("c");
assertThat(array.get(2).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(3).asPrimitive().get()).isEqualTo("iceberg");
assertThat(array.get(3).asPrimitive().sizeInBytes()).isEqualTo(8);
assertThat(array.get(4).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(4).asPrimitive().get()).isEqualTo("d");
assertThat(array.get(4).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(5).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(5).asPrimitive().get()).isEqualTo("e");
assertThat(array.get(5).asPrimitive().sizeInBytes()).isEqualTo(2);

assertThatThrownBy(() -> array.get(6))
.isInstanceOf(ArrayIndexOutOfBoundsException.class)
@@ -178,12 +184,16 @@ public void testTwoByteOffsets() {
assertThat(array.numElements()).isEqualTo(4);
assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(0).asPrimitive().get()).isEqualTo(randomString);
assertThat(array.get(0).asPrimitive().sizeInBytes()).isEqualTo(5 + randomString.length());
assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(1).asPrimitive().get()).isEqualTo("a");
assertThat(array.get(1).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(2).asPrimitive().get()).isEqualTo("b");
assertThat(array.get(2).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(3).asPrimitive().get()).isEqualTo("c");
assertThat(array.get(3).asPrimitive().sizeInBytes()).isEqualTo(2);

assertThatThrownBy(() -> array.get(4))
.isInstanceOf(ArrayIndexOutOfBoundsException.class)
@@ -203,12 +213,16 @@ public void testThreeByteOffsets() {
assertThat(array.numElements()).isEqualTo(4);
assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(0).asPrimitive().get()).isEqualTo(randomString);
assertThat(array.get(0).asPrimitive().sizeInBytes()).isEqualTo(5 + randomString.length());
assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(1).asPrimitive().get()).isEqualTo("a");
assertThat(array.get(1).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(2).asPrimitive().get()).isEqualTo("b");
assertThat(array.get(2).asPrimitive().sizeInBytes()).isEqualTo(2);
assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING);
assertThat(array.get(3).asPrimitive().get()).isEqualTo("c");
assertThat(array.get(3).asPrimitive().sizeInBytes()).isEqualTo(2);

assertThatThrownBy(() -> array.get(4))
.isInstanceOf(ArrayIndexOutOfBoundsException.class)
Original file line number Diff line number Diff line change
@@ -189,15 +189,26 @@ public void testTwoByteOffsets() {
SerializedPrimitive bigString = VariantTestUtil.createString(randomString);

// note that order doesn't matter. fields are sorted by name
Map<String, VariantValue> data = ImmutableMap.of("big", bigString, "a", I1, "b", I2, "c", I3);
Map<String, VariantValue> data =
ImmutableMap.of(
"small",
VariantTestUtil.createShortString("iceberg"),
"big",
bigString,
"a",
I1,
"b",
I2,
"c",
I3);
ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* sort names */);
ByteBuffer value = VariantTestUtil.createObject(meta, data);

VariantMetadata metadata = VariantMetadata.from(meta);
SerializedObject object = SerializedObject.from(metadata, value, value.get(0));

assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
assertThat(object.numFields()).isEqualTo(4);
assertThat(object.numFields()).isEqualTo(5);

assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -207,6 +218,10 @@ public void testTwoByteOffsets() {
assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3);
assertThat(object.get("big").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("big").asPrimitive().get()).isEqualTo(randomString);
assertThat(object.get("big").asPrimitive().sizeInBytes()).isEqualTo(5 + randomString.length());
assertThat(object.get("small").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("small").asPrimitive().get()).isEqualTo("iceberg");
assertThat(object.get("small").asPrimitive().sizeInBytes()).isEqualTo(8);
}

@Test
@@ -217,15 +232,25 @@ public void testThreeByteOffsets() {

// note that order doesn't matter. fields are sorted by name
Map<String, VariantValue> data =
ImmutableMap.of("really-big", reallyBigString, "a", I1, "b", I2, "c", I3);
ImmutableMap.of(
"small",
VariantTestUtil.createShortString("iceberg"),
"really-big",
reallyBigString,
"a",
I1,
"b",
I2,
"c",
I3);
ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* sort names */);
ByteBuffer value = VariantTestUtil.createObject(meta, data);

VariantMetadata metadata = VariantMetadata.from(meta);
SerializedObject object = SerializedObject.from(metadata, value, value.get(0));

assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
assertThat(object.numFields()).isEqualTo(4);
assertThat(object.numFields()).isEqualTo(5);

assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -235,6 +260,11 @@ public void testThreeByteOffsets() {
assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3);
assertThat(object.get("really-big").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("really-big").asPrimitive().get()).isEqualTo(randomString);
assertThat(object.get("really-big").asPrimitive().sizeInBytes())
.isEqualTo(5 + randomString.length());
assertThat(object.get("small").type()).isEqualTo(PhysicalType.STRING);
assertThat(object.get("small").asPrimitive().get()).isEqualTo("iceberg");
assertThat(object.get("small").asPrimitive().sizeInBytes()).isEqualTo(8);
}

@ParameterizedTest
@@ -261,6 +291,8 @@ public void testLargeObject(boolean sortFieldNames) {
VariantValue fieldValue = object.get(entry.getKey());
assertThat(fieldValue.type()).isEqualTo(PhysicalType.STRING);
assertThat(fieldValue.asPrimitive().get()).isEqualTo(entry.getValue().get());
assertThat(fieldValue.asPrimitive().sizeInBytes())
.isEqualTo(5 + entry.getValue().get().toString().length());
}
}

Original file line number Diff line number Diff line change
@@ -442,6 +442,7 @@ public void testString() {

assertThat(value.type()).isEqualTo(PhysicalType.STRING);
assertThat(value.get()).isEqualTo("iceberg");
assertThat(value.sizeInBytes()).isEqualTo(12);
}

@Test
@@ -451,6 +452,7 @@ public void testShortString() {

assertThat(value.type()).isEqualTo(PhysicalType.STRING);
assertThat(value.get()).isEqualTo("iceberg");
assertThat(value.sizeInBytes()).isEqualTo(8);
}

@Test
Original file line number Diff line number Diff line change
@@ -107,6 +107,15 @@ static SerializedPrimitive createString(String string) {
return SerializedPrimitive.from(buffer, buffer.get(0));
}

/** Creates a short string primitive of max 63 chars to use only 1 header */
static SerializedShortString createShortString(String string) {
byte[] utf8 = string.getBytes(StandardCharsets.UTF_8);
ByteBuffer buffer = ByteBuffer.allocate(1 + utf8.length).order(ByteOrder.LITTLE_ENDIAN);
buffer.put(0, (byte) (VariantUtil.primitiveHeader(utf8.length) | 0b00000001));
writeBufferAbsolute(buffer, 1, ByteBuffer.wrap(utf8));
return SerializedShortString.from(buffer, buffer.get(0));
}

public static ByteBuffer variantBuffer(Map<String, VariantValue> data) {
ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* sort names */);
ByteBuffer value = VariantTestUtil.createObject(meta, data);
Original file line number Diff line number Diff line change
@@ -53,6 +53,8 @@ class PrimitiveWrapper<T> implements VariantPrimitive<T> {
private static final byte TIMESTAMPNTZ_NANOS_HEADER =
VariantUtil.primitiveHeader(Primitives.TYPE_TIMESTAMPNTZ_NANOS);
private static final byte UUID_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_UUID);
private static final int SHORT_STRING_LENGTH = 64;
private static final int SHORT_STRING_LENGTH_UMASK = 0b00000001;

private final PhysicalType type;
private final T value;
@@ -111,10 +113,13 @@ public int sizeInBytes() {
case BINARY:
return 5 + ((ByteBuffer) value).remaining(); // 1 header + 4 length + value length
case STRING:
byte[] valueBytes = ((String) value).getBytes(StandardCharsets.UTF_8);
if (null == buffer) {
this.buffer = ByteBuffer.wrap(((String) value).getBytes(StandardCharsets.UTF_8));
this.buffer = ByteBuffer.wrap(valueBytes);
}
if (valueBytes.length < SHORT_STRING_LENGTH) {
return 1 + buffer.remaining(); // 1 header + 1 length + value length
}

return 5 + buffer.remaining(); // 1 header + 4 length + value length
case UUID:
return 1 + 16; // 1 header + 16 length
@@ -208,15 +213,22 @@ public int writeTo(ByteBuffer outBuffer, int offset) {
VariantUtil.writeBufferAbsolute(outBuffer, offset + 5, binary);
return 5 + binary.remaining();
case STRING:
// TODO: use short string when possible
byte[] valueBytes = ((String) value).getBytes(StandardCharsets.UTF_8);
if (null == buffer) {
this.buffer = ByteBuffer.wrap(((String) value).getBytes(StandardCharsets.UTF_8));
this.buffer = ByteBuffer.wrap(valueBytes);
}
if (valueBytes.length < SHORT_STRING_LENGTH) {
outBuffer.put(
offset,
(byte) (VariantUtil.primitiveHeader(valueBytes.length) | SHORT_STRING_LENGTH_UMASK));
VariantUtil.writeBufferAbsolute(outBuffer, offset + 1, buffer);
return 1 + buffer.remaining();
} else {
outBuffer.put(offset, STRING_HEADER);
outBuffer.putInt(offset + 1, buffer.remaining());
VariantUtil.writeBufferAbsolute(outBuffer, offset + 5, buffer);
return 5 + buffer.remaining();
}

outBuffer.put(offset, STRING_HEADER);
outBuffer.putInt(offset + 1, buffer.remaining());
VariantUtil.writeBufferAbsolute(outBuffer, offset + 5, buffer);
return 5 + buffer.remaining();
case TIME:
outBuffer.put(offset, TIME_HEADER);
outBuffer.putLong(offset + 1, (Long) value);
Original file line number Diff line number Diff line change
@@ -23,6 +23,8 @@
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Random;
import org.apache.iceberg.util.RandomUtil;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.FieldSource;

@@ -57,7 +59,10 @@ public class TestPrimitiveWrapper {
Variants.of(new BigDecimal("9876543210.123456789")), // decimal16
Variants.of(new BigDecimal("-9876543210.123456789")), // decimal16
Variants.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})),
Variants.of("iceberg"),
Variants.of(
"icebergicebergicebergicebergicebergicebergicebergicebergiceberg"), // short string of
// 63 (9*7) chars
Variants.of(RandomUtil.generateString(64, new Random(1))), // string of 64 chars
};

@ParameterizedTest