Skip to content
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

API: Add UnknownType #12012

Merged
merged 7 commits into from
Jan 24, 2025
Merged
Show file tree
Hide file tree
Changes from 3 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
5 changes: 4 additions & 1 deletion api/src/main/java/org/apache/iceberg/Schema.java
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,10 @@ public class Schema implements Serializable {

@VisibleForTesting
static final Map<Type.TypeID, Integer> MIN_FORMAT_VERSIONS =
ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3, Type.TypeID.VARIANT, 3);
ImmutableMap.of(
Type.TypeID.TIMESTAMP_NANO, 3,
Type.TypeID.VARIANT, 3,
Type.TypeID.UNKNOWN, 3);

private final StructType struct;
private final int schemaId;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -539,7 +539,8 @@ private static String sanitize(Type type, Object value, long now, int today) {
case FIXED:
case BINARY:
case VARIANT:
// for boolean, uuid, decimal, fixed, variant, and binary, match the string result
case UNKNOWN:
// for boolean, uuid, decimal, fixed, variant, unknown, and binary, match the string result
return sanitizeSimpleString(value.toString());
Fokko marked this conversation as resolved.
Show resolved Hide resolved
}
throw new UnsupportedOperationException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,9 @@ class Identity<T> implements Transform<T, T> {
@Deprecated
public static <I> Identity<I> get(Type type) {
Preconditions.checkArgument(
type.typeId() != Type.TypeID.VARIANT, "Unsupported type for identity: %s", type);
type.typeId() != Type.TypeID.VARIANT && type.typeId() != Type.TypeID.UNKNOWN,
Fokko marked this conversation as resolved.
Show resolved Hide resolved
"Unsupported type for identity: %s",
type);

return new Identity<>(type);
}
Expand Down Expand Up @@ -93,6 +95,10 @@ public SerializableFunction<T, T> bind(Type type) {

@Override
public boolean canTransform(Type maybePrimitive) {
if (maybePrimitive.typeId() == Type.TypeID.UNKNOWN) {
return false;
Fokko marked this conversation as resolved.
Show resolved Hide resolved
}

return maybePrimitive.isPrimitiveType();
}

Expand Down
3 changes: 2 additions & 1 deletion api/src/main/java/org/apache/iceberg/types/Type.java
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,8 @@ enum TypeID {
STRUCT(StructLike.class),
LIST(List.class),
MAP(Map.class),
VARIANT(Object.class);
VARIANT(Object.class),
UNKNOWN(Object.class);

private final Class<?> javaClass;

Expand Down
3 changes: 3 additions & 0 deletions api/src/main/java/org/apache/iceberg/types/TypeUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -536,6 +536,9 @@ private static int estimateSize(Type type) {
case BINARY:
case VARIANT:
return 80;
case UNKNOWN:
// Consider Unknown as null
return 0;
case DECIMAL:
// 12 (header) + (12 + 12 + 4) (BigInteger) + 4 (scale) = 44 bytes
return 44;
Expand Down
19 changes: 19 additions & 0 deletions api/src/main/java/org/apache/iceberg/types/Types.java
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ private Types() {}
.put(StringType.get().toString(), StringType.get())
.put(UUIDType.get().toString(), UUIDType.get())
.put(BinaryType.get().toString(), BinaryType.get())
.put(UnknownType.get().toString(), UnknownType.get())
.buildOrThrow();

private static final Pattern FIXED = Pattern.compile("fixed\\[\\s*(\\d+)\\s*\\]");
Expand Down Expand Up @@ -447,6 +448,24 @@ public int hashCode() {
}
}

public static class UnknownType extends PrimitiveType {
private static final UnknownType INSTANCE = new UnknownType();

public static UnknownType get() {
return INSTANCE;
}

@Override
public TypeID typeId() {
return TypeID.UNKNOWN;
}

@Override
public String toString() {
return "unknown";
}
}

public static class DecimalType extends PrimitiveType {
public static DecimalType of(int precision, int scale) {
return new DecimalType(precision, scale);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,8 @@ public class TestPartitionSpecValidation {
NestedField.required(4, "d", Types.TimestampType.withZone()),
NestedField.required(5, "another_d", Types.TimestampType.withZone()),
NestedField.required(6, "s", Types.StringType.get()),
NestedField.required(7, "v", Types.VariantType.get()));
NestedField.required(7, "v", Types.VariantType.get()),
NestedField.required(8, "u", Types.UnknownType.get()));

@Test
public void testMultipleTimestampPartitions() {
Expand Down Expand Up @@ -325,4 +326,15 @@ public void testVariantUnsupported() {
.isInstanceOf(ValidationException.class)
.hasMessage("Cannot partition by non-primitive source field: variant");
}

@Test
public void testUnknownUnsupported() {
assertThatThrownBy(
() ->
PartitionSpec.builderFor(SCHEMA)
.add(8, 1005, "unknown_partition1", Transforms.bucket(5))
.build())
.isInstanceOf(ValidationException.class)
.hasMessage("Invalid source type unknown for transform: bucket[5]");
}
}
3 changes: 2 additions & 1 deletion api/src/test/java/org/apache/iceberg/TestSchema.java
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,8 @@ public class TestSchema {
ImmutableList.of(
Types.TimestampNanoType.withoutZone(),
Types.TimestampNanoType.withZone(),
Types.VariantType.get());
Types.VariantType.get(),
Types.UnknownType.get());

private static final Schema INITIAL_DEFAULT_SCHEMA =
new Schema(
Expand Down
14 changes: 14 additions & 0 deletions api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,20 @@ public void testVariantUnsupported() {
assertThat(bucket.canTransform(Types.VariantType.get())).isFalse();
}

@Test
public void testUnknownUnsupported() {
HonahX marked this conversation as resolved.
Show resolved Hide resolved
assertThatThrownBy(() -> Transforms.bucket(Types.UnknownType.get(), 3))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot bucket by type: unknown");

Transform<Object, Integer> bucket = Transforms.bucket(3);
assertThatThrownBy(() -> bucket.bind(Types.UnknownType.get()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot bucket by type: unknown");

assertThat(bucket.canTransform(Types.UnknownType.get())).isFalse();
}

private byte[] randomBytes(int length) {
byte[] bytes = new byte[length];
testRandom.nextBytes(bytes);
Expand Down
17 changes: 17 additions & 0 deletions api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java
Original file line number Diff line number Diff line change
Expand Up @@ -173,4 +173,21 @@ public void testVariantUnsupported() {

assertThat(Transforms.identity().canTransform(Types.VariantType.get())).isFalse();
}

@Test
public void testUnknownUnsupported() {
assertThatThrownBy(() -> Transforms.identity().bind(Types.UnknownType.get()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Cannot bind to unsupported type: unknown");

assertThatThrownBy(() -> Transforms.fromString(Types.UnknownType.get(), "identity"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Unsupported type for identity: unknown");

assertThatThrownBy(() -> Transforms.identity(Types.UnknownType.get()))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Unsupported type for identity: unknown");

assertThat(Transforms.identity().canTransform(Types.UnknownType.get())).isFalse();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,15 @@ public void testVariant() throws Exception {
.isEqualTo(variant);
}

@Test
public void testUnknown() throws Exception {
Types.UnknownType unknown = Types.UnknownType.get();
Type copy = TestHelpers.roundTripSerialize(unknown);
assertThat(copy)
.as("Unknown serialization should be equal to starting type")
.isEqualTo(unknown);
}

@Test
public void testSchema() throws Exception {
Schema schema =
Expand Down
4 changes: 2 additions & 2 deletions api/src/test/java/org/apache/iceberg/types/TestTypes.java
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public void fromPrimitiveString() {
assertThat(Types.fromPrimitiveString("Decimal(2,3)")).isEqualTo(Types.DecimalType.of(2, 3));

assertThatExceptionOfType(IllegalArgumentException.class)
.isThrownBy(() -> Types.fromPrimitiveString("Unknown"))
.withMessageContaining("Unknown");
.isThrownBy(() -> Types.fromPrimitiveString("abcdefghij"))
.withMessage("Cannot parse type string to primitive: abcdefghij");
}
}
16 changes: 16 additions & 0 deletions core/src/test/java/org/apache/iceberg/TestSortOrder.java
Original file line number Diff line number Diff line change
Expand Up @@ -342,6 +342,22 @@ public void testVariantUnsupported() {
.hasMessage("Unsupported type for identity: variant");
}

@TestTemplate
Fokko marked this conversation as resolved.
Show resolved Hide resolved
public void testUnknownUnsupported() {
Schema v3Schema =
new Schema(
Types.NestedField.required(3, "id", Types.LongType.get()),
Types.NestedField.required(4, "data", Types.StringType.get()),
Types.NestedField.required(
5,
"struct",
Types.StructType.of(Types.NestedField.optional(6, "u", Types.UnknownType.get()))));

assertThatThrownBy(() -> SortOrder.builderFor(v3Schema).withOrderId(10).asc("struct.u").build())
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("Unsupported type for identity: unknown");
Fokko marked this conversation as resolved.
Show resolved Hide resolved
}

@TestTemplate
public void testPreservingOrderSortedColumnNames() {
SortOrder order =
Expand Down