diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 30dfabba8690..43d3d0c41c32 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -160,6 +160,9 @@ + + + diff --git a/api/src/main/java/org/apache/iceberg/Accessors.java b/api/src/main/java/org/apache/iceberg/Accessors.java index 08233624f244..0b36730fbb4b 100644 --- a/api/src/main/java/org/apache/iceberg/Accessors.java +++ b/api/src/main/java/org/apache/iceberg/Accessors.java @@ -232,6 +232,11 @@ public Map> struct( return accessors; } + @Override + public Map> variant(Types.VariantType variant) { + return null; + } + @Override public Map> field( Types.NestedField field, Map> fieldResult) { diff --git a/api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java b/api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java index e58f76a8de56..75055cddc197 100644 --- a/api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java +++ b/api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java @@ -124,6 +124,11 @@ public Type map(Types.MapType map, Supplier keyFuture, Supplier valu } } + @Override + public Type variant(Types.VariantType variant) { + return variant; + } + @Override public Type primitive(Type.PrimitiveType primitive) { return primitive; diff --git a/api/src/main/java/org/apache/iceberg/types/AssignIds.java b/api/src/main/java/org/apache/iceberg/types/AssignIds.java index 68588f581adc..b2f72751eb89 100644 --- a/api/src/main/java/org/apache/iceberg/types/AssignIds.java +++ b/api/src/main/java/org/apache/iceberg/types/AssignIds.java @@ -92,6 +92,11 @@ public Type map(Types.MapType map, Supplier keyFuture, Supplier valu } } + @Override + public Type variant(Types.VariantType variant) { + return variant; + } + @Override public Type primitive(Type.PrimitiveType primitive) { return primitive; diff --git a/api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java b/api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java index 502e52c345e5..725f7f42562e 100644 --- a/api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java +++ b/api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java @@ -250,6 +250,16 @@ public List map( } } + @Override + public List variant(Types.VariantType readVariant) { + if (currentType.isVariantType()) { + return NO_ERRORS; + } + + // Currently promotion is not allowed to variant type + return ImmutableList.of(String.format(": %s cannot be read as a %s", currentType, readVariant)); + } + @Override public List primitive(Type.PrimitiveType readPrimitive) { if (currentType.equals(readPrimitive)) { diff --git a/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java index f0750f337e2e..64faebb48243 100644 --- a/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java +++ b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java @@ -77,9 +77,9 @@ public Type map(Types.MapType map, Type keyResult, Type valueResult) { } @Override - public Type variant() { - if (predicate.test(Types.VariantType.get())) { - return Types.VariantType.get(); + public Type variant(Types.VariantType variant) { + if (predicate.test(variant)) { + return variant; } return null; diff --git a/api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java b/api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java index a8a7de065ece..1ec70b8578bc 100644 --- a/api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java +++ b/api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java @@ -47,7 +47,9 @@ public Set struct(Types.StructType struct, List> fieldResu @Override public Set field(Types.NestedField field, Set fieldResult) { - if ((includeStructIds && field.type().isStructType()) || field.type().isPrimitiveType()) { + if ((includeStructIds && field.type().isStructType()) + || field.type().isPrimitiveType() + || field.type().isVariantType()) { fieldIds.add(field.fieldId()); } return fieldIds; @@ -72,4 +74,9 @@ public Set map(Types.MapType map, Set keyResult, Set } return fieldIds; } + + @Override + public Set variant(Types.VariantType variant) { + return null; + } } diff --git a/api/src/main/java/org/apache/iceberg/types/IndexById.java b/api/src/main/java/org/apache/iceberg/types/IndexById.java index 40280c5ed9dd..a7b96eb381f7 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexById.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexById.java @@ -64,4 +64,9 @@ public Map map( } return null; } + + @Override + public Map variant(Types.VariantType variant) { + return null; + } } diff --git a/api/src/main/java/org/apache/iceberg/types/IndexByName.java b/api/src/main/java/org/apache/iceberg/types/IndexByName.java index 131434c9a156..60258f5c5c3e 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexByName.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexByName.java @@ -177,7 +177,7 @@ public Map map( } @Override - public Map variant() { + public Map variant(Types.VariantType variant) { return nameToId; } diff --git a/api/src/main/java/org/apache/iceberg/types/IndexParents.java b/api/src/main/java/org/apache/iceberg/types/IndexParents.java index 952447ed2799..6e611d47e912 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexParents.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexParents.java @@ -77,7 +77,7 @@ public Map map( } @Override - public Map variant() { + public Map variant(Types.VariantType variant) { return idToParent; } diff --git a/api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java b/api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java index 42f0da38167d..928a65878d3a 100644 --- a/api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java +++ b/api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java @@ -33,6 +33,6 @@ class PrimitiveHolder implements Serializable { } Object readResolve() throws ObjectStreamException { - return Types.fromPrimitiveString(typeAsString); + return Types.fromTypeName(typeAsString); } } diff --git a/api/src/main/java/org/apache/iceberg/types/PruneColumns.java b/api/src/main/java/org/apache/iceberg/types/PruneColumns.java index daf2e6bbc0ca..56f01cf34bb5 100644 --- a/api/src/main/java/org/apache/iceberg/types/PruneColumns.java +++ b/api/src/main/java/org/apache/iceberg/types/PruneColumns.java @@ -159,6 +159,11 @@ public Type map(Types.MapType map, Type ignored, Type valueResult) { return null; } + @Override + public Type variant(Types.VariantType variant) { + return null; + } + @Override public Type primitive(Type.PrimitiveType primitive) { return null; diff --git a/api/src/main/java/org/apache/iceberg/types/ReassignDoc.java b/api/src/main/java/org/apache/iceberg/types/ReassignDoc.java index 9ce04a7bd103..328d81c42885 100644 --- a/api/src/main/java/org/apache/iceberg/types/ReassignDoc.java +++ b/api/src/main/java/org/apache/iceberg/types/ReassignDoc.java @@ -96,6 +96,11 @@ public Type map(Types.MapType map, Supplier keyTypeFuture, Supplier } } + @Override + public Type variant(Types.VariantType variant) { + return variant; + } + @Override public Type primitive(Type.PrimitiveType primitive) { return primitive; diff --git a/api/src/main/java/org/apache/iceberg/types/ReassignIds.java b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java index 565ceee2a901..3d114f093f6b 100644 --- a/api/src/main/java/org/apache/iceberg/types/ReassignIds.java +++ b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java @@ -157,6 +157,11 @@ public Type map(Types.MapType map, Supplier keyTypeFuture, Supplier } } + @Override + public Type variant(Types.VariantType variant) { + return variant; + } + @Override public Type primitive(Type.PrimitiveType primitive) { return primitive; // nothing to reassign diff --git a/api/src/main/java/org/apache/iceberg/types/Type.java b/api/src/main/java/org/apache/iceberg/types/Type.java index f4c6f22134a5..67e40df9e939 100644 --- a/api/src/main/java/org/apache/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -82,6 +82,10 @@ default Types.MapType asMapType() { throw new IllegalArgumentException("Not a map type: " + this); } + default Types.VariantType asVariantType() { + throw new IllegalArgumentException("Not a variant type: " + this); + } + default boolean isNestedType() { return false; } @@ -98,6 +102,10 @@ default boolean isMapType() { return false; } + default boolean isVariantType() { + return false; + } + default NestedType asNestedType() { throw new IllegalArgumentException("Not a nested type: " + this); } diff --git a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java index 39f2898757a6..4892696ab450 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -616,8 +616,16 @@ public T map(Types.MapType map, T keyResult, T valueResult) { return null; } + /** + * @deprecated will be removed in 2.0.0; use {@link #variant(Types.VariantType)} instead. + */ + @Deprecated public T variant() { - return null; + return variant(Types.VariantType.get()); + } + + public T variant(Types.VariantType variant) { + throw new UnsupportedOperationException("Unsupported type: variant"); } public T primitive(Type.PrimitiveType primitive) { @@ -684,7 +692,7 @@ public static T visit(Type type, SchemaVisitor visitor) { return visitor.map(map, keyResult, valueResult); case VARIANT: - return visitor.variant(); + return visitor.variant(type.asVariantType()); default: return visitor.primitive(type.asPrimitiveType()); @@ -712,6 +720,10 @@ public T map(Types.MapType map, Supplier keyResult, Supplier valueResult) return null; } + public T variant(Types.VariantType variant) { + throw new UnsupportedOperationException("Unsupported type: variant"); + } + public T primitive(Type.PrimitiveType primitive) { return null; } @@ -788,6 +800,9 @@ public static T visit(Type type, CustomOrderSchemaVisitor visitor) { new VisitFuture<>(map.keyType(), visitor), new VisitFuture<>(map.valueType(), visitor)); + case VARIANT: + return visitor.variant(type.asVariantType()); + default: return visitor.primitive(type.asPrimitiveType()); } diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index 6620263f0d3f..a866a31ea005 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -40,8 +40,8 @@ public class Types { private Types() {} - private static final ImmutableMap TYPES = - ImmutableMap.builder() + private static final ImmutableMap TYPES = + ImmutableMap.builder() .put(BooleanType.get().toString(), BooleanType.get()) .put(IntegerType.get().toString(), IntegerType.get()) .put(LongType.get().toString(), LongType.get()) @@ -57,13 +57,14 @@ private Types() {} .put(UUIDType.get().toString(), UUIDType.get()) .put(BinaryType.get().toString(), BinaryType.get()) .put(UnknownType.get().toString(), UnknownType.get()) + .put(VariantType.get().toString(), VariantType.get()) .buildOrThrow(); private static final Pattern FIXED = Pattern.compile("fixed\\[\\s*(\\d+)\\s*\\]"); private static final Pattern DECIMAL = Pattern.compile("decimal\\(\\s*(\\d+)\\s*,\\s*(\\d+)\\s*\\)"); - public static PrimitiveType fromPrimitiveString(String typeString) { + public static Type fromTypeName(String typeString) { String lowerTypeString = typeString.toLowerCase(Locale.ROOT); if (TYPES.containsKey(lowerTypeString)) { return TYPES.get(lowerTypeString); @@ -82,6 +83,15 @@ public static PrimitiveType fromPrimitiveString(String typeString) { throw new IllegalArgumentException("Cannot parse type string to primitive: " + typeString); } + public static PrimitiveType fromPrimitiveString(String typeString) { + Type type = fromTypeName(typeString); + if (type.isPrimitiveType()) { + return type.asPrimitiveType(); + } + + throw new IllegalArgumentException("Cannot parse type string: variant is not a primitive type"); + } + public static class BooleanType extends PrimitiveType { private static final BooleanType INSTANCE = new BooleanType(); @@ -431,6 +441,16 @@ public String toString() { return "variant"; } + @Override + public boolean isVariantType() { + return true; + } + + @Override + public VariantType asVariantType() { + return this; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -692,6 +712,10 @@ private NestedField( Literal writeDefault) { Preconditions.checkNotNull(name, "Name cannot be null"); Preconditions.checkNotNull(type, "Type cannot be null"); + Preconditions.checkArgument( + isOptional || !type.equals(UnknownType.get()), + "Cannot create required field with unknown type: %s", + name); this.isOptional = isOptional; this.id = id; this.name = name; @@ -706,7 +730,10 @@ private static Literal castDefault(Literal defaultValue, Type type) { throw new IllegalArgumentException( String.format("Invalid default value for %s: %s (must be null)", type, defaultValue)); } else if (defaultValue != null) { - return defaultValue.to(type); + Literal typedDefault = defaultValue.to(type); + Preconditions.checkArgument( + typedDefault != null, "Cannot cast default value to %s: %s", type, defaultValue); + return typedDefault; } return null; diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java index ba7010f196a7..125b0b519fbc 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java @@ -37,7 +37,7 @@ public class TestPartitionSpecValidation { NestedField.required(5, "another_d", Types.TimestampType.withZone()), NestedField.required(6, "s", Types.StringType.get()), NestedField.required(7, "v", Types.VariantType.get()), - NestedField.required(8, "u", Types.UnknownType.get())); + NestedField.optional(8, "u", Types.UnknownType.get())); @Test public void testMultipleTimestampPartitions() { diff --git a/api/src/test/java/org/apache/iceberg/TestSchema.java b/api/src/test/java/org/apache/iceberg/TestSchema.java index aca3c1ad6637..6b41baa4714f 100644 --- a/api/src/test/java/org/apache/iceberg/TestSchema.java +++ b/api/src/test/java/org/apache/iceberg/TestSchema.java @@ -31,6 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.FieldSource; @@ -42,8 +43,7 @@ public class TestSchema { ImmutableList.of( Types.TimestampNanoType.withoutZone(), Types.TimestampNanoType.withZone(), - Types.VariantType.get(), - Types.UnknownType.get()); + Types.VariantType.get()); private static final Schema INITIAL_DEFAULT_SCHEMA = new Schema( @@ -123,6 +123,56 @@ private static Stream supportedTypes() { .mapToObj(supportedVersion -> Arguments.of(type, supportedVersion))); } + @Test + public void testUnknownSupport() { + // this needs a different schema because it cannot be used in required fields + Schema schemaWithUnknown = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "top", Types.UnknownType.get()), + Types.NestedField.optional( + 3, "arr", Types.ListType.ofOptional(4, Types.UnknownType.get())), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of( + Types.NestedField.optional(6, "inner_op", Types.UnknownType.get()), + Types.NestedField.optional( + 7, + "inner_map", + Types.MapType.ofOptional( + 8, 9, Types.StringType.get(), Types.UnknownType.get())), + Types.NestedField.optional( + 10, + "struct_arr", + Types.StructType.of( + Types.NestedField.optional(11, "deep", Types.UnknownType.get())))))); + + assertThatThrownBy(() -> Schema.checkCompatibility(schemaWithUnknown, 2)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid schema for v%s:\n" + + "- Invalid type for top: %s is not supported until v%s\n" + + "- Invalid type for arr.element: %s is not supported until v%s\n" + + "- Invalid type for struct.inner_op: %s is not supported until v%s\n" + + "- Invalid type for struct.inner_map.value: %s is not supported until v%s\n" + + "- Invalid type for struct.struct_arr.deep: %s is not supported until v%s", + 2, + Types.UnknownType.get(), + MIN_FORMAT_VERSIONS.get(Type.TypeID.UNKNOWN), + Types.UnknownType.get(), + MIN_FORMAT_VERSIONS.get(Type.TypeID.UNKNOWN), + Types.UnknownType.get(), + MIN_FORMAT_VERSIONS.get(Type.TypeID.UNKNOWN), + Types.UnknownType.get(), + MIN_FORMAT_VERSIONS.get(Type.TypeID.UNKNOWN), + Types.UnknownType.get(), + MIN_FORMAT_VERSIONS.get(Type.TypeID.UNKNOWN)); + + assertThatCode(() -> Schema.checkCompatibility(schemaWithUnknown, 3)) + .doesNotThrowAnyException(); + } + @ParameterizedTest @MethodSource("supportedTypes") public void testTypeSupported(Type type, int supportedVersion) { diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java index 251f1942e9dd..7069d891c38d 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -129,9 +129,9 @@ public class TestInclusiveMetricsEvaluator { Row.of(), 50, // any value counts, including nulls - ImmutableMap.of(3, 20L), + ImmutableMap.of(3, 50L), // null value counts - ImmutableMap.of(3, 2L), + ImmutableMap.of(3, 0L), // nan value counts null, // lower bounds @@ -145,9 +145,9 @@ public class TestInclusiveMetricsEvaluator { Row.of(), 50, // any value counts, including nulls - ImmutableMap.of(3, 20L), + ImmutableMap.of(3, 50L), // null value counts - ImmutableMap.of(3, 2L), + ImmutableMap.of(3, 0L), // nan value counts null, // lower bounds @@ -161,9 +161,9 @@ public class TestInclusiveMetricsEvaluator { Row.of(), 50, // any value counts, including nulls - ImmutableMap.of(3, 20L), + ImmutableMap.of(3, 50L), // null value counts - ImmutableMap.of(3, 2L), + ImmutableMap.of(3, 0L), // nan value counts null, // lower bounds @@ -171,6 +171,22 @@ public class TestInclusiveMetricsEvaluator { // upper bounds ImmutableMap.of(3, toByteBuffer(StringType.get(), "イロハニホヘト"))); + private static final DataFile FILE_5 = + new TestDataFile( + "file_4.avro", + Row.of(), + 50, + // any value counts, including nulls + ImmutableMap.of(3, 50L), + // null value counts + ImmutableMap.of(3, 0L), + // nan value counts + null, + // lower bounds + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc")), + // upper bounds + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abcdefghi"))); + @Test public void testAllNulls() { boolean shouldRead = new InclusiveMetricsEvaluator(SCHEMA, notNull("all_nulls")).eval(FILE); @@ -731,6 +747,14 @@ public void testStringNotStartsWith() { new InclusiveMetricsEvaluator(SCHEMA, notStartsWith("required", aboveMax), true) .eval(FILE_4); assertThat(shouldRead).as("Should read: range matches").isTrue(); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notStartsWith("required", "abc"), true).eval(FILE_5); + assertThat(shouldRead).as("Should not read: all strings start with prefix").isFalse(); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notStartsWith("required", "abcd"), true).eval(FILE_5); + assertThat(shouldRead).as("Should not read: lower shorter than prefix, cannot match").isTrue(); } @Test diff --git a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java index 2d02da5346a7..debb9c9dc1d6 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java +++ b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java @@ -22,10 +22,15 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import java.util.Arrays; import java.util.List; +import java.util.stream.Stream; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Type.PrimitiveType; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public class TestReadabilityChecks { private static final Type.PrimitiveType[] PRIMITIVES = @@ -112,6 +117,40 @@ private void testDisallowPrimitiveToStruct(PrimitiveType from, Schema fromSchema .contains("cannot be read as a struct"); } + @Test + public void testVariantToVariant() { + Schema fromSchema = new Schema(required(1, "from_field", Types.VariantType.get())); + List errors = + CheckCompatibility.writeCompatibilityErrors( + new Schema(required(1, "to_field", Types.VariantType.get())), fromSchema); + assertThat(errors).as("Should produce 0 error messages").isEmpty(); + } + + private static Stream incompatibleTypesToVariant() { + return Stream.of( + Stream.of( + Arguments.of(Types.StructType.of(required(1, "from", Types.IntegerType.get()))), + Arguments.of( + Types.MapType.ofRequired( + 1, 2, Types.StringType.get(), Types.IntegerType.get())), + Arguments.of(Types.ListType.ofRequired(1, Types.StringType.get()))), + Arrays.stream(PRIMITIVES).map(type -> Arguments.of(type))) + .flatMap(s -> s); + } + + @ParameterizedTest + @MethodSource("incompatibleTypesToVariant") + public void testIncompatibleTypesToVariant(Type from) { + Schema fromSchema = new Schema(required(3, "from_field", from)); + List errors = + CheckCompatibility.writeCompatibilityErrors( + new Schema(required(3, "to_field", Types.VariantType.get())), fromSchema); + assertThat(errors).hasSize(1); + assertThat(errors.get(0)) + .as("Should complain that other type to variant is not allowed") + .contains("cannot be read as a variant"); + } + @Test public void testRequiredSchemaField() { Schema write = new Schema(optional(1, "from_field", Types.IntegerType.get())); diff --git a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java index a222e8e66b8e..790f59587c59 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -46,6 +46,7 @@ public void testIdentityTypes() throws Exception { Types.StringType.get(), Types.UUIDType.get(), Types.BinaryType.get(), + Types.UnknownType.get() }; for (Type type : identityPrimitives) { @@ -136,15 +137,6 @@ 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 = diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java index b7ecb5e7ec02..078c0180b5e7 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java @@ -23,13 +23,19 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; import org.apache.iceberg.Schema; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types.IntegerType; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public class TestTypeUtil { @Test @@ -646,4 +652,95 @@ public void testReassignOrRefreshIdsCaseInsensitive() { required(2, "FIELD2", Types.IntegerType.get()))); assertThat(actualSchema.asStruct()).isEqualTo(expectedSchema.asStruct()); } + + private static Stream testTypes() { + return Stream.of( + Arguments.of(Types.UnknownType.get()), + Arguments.of(Types.VariantType.get()), + Arguments.of(Types.TimestampNanoType.withoutZone()), + Arguments.of(Types.TimestampNanoType.withZone())); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testAssignIdsWithType(Type testType) { + Types.StructType sourceType = + Types.StructType.of(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + Type expectedType = + Types.StructType.of(required(10, "id", IntegerType.get()), optional(11, "data", testType)); + + Type assignedType = TypeUtil.assignIds(sourceType, oldId -> oldId + 10); + assertThat(assignedType).isEqualTo(expectedType); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testAssignFreshIdsWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + + Schema assignedSchema = TypeUtil.assignFreshIds(schema, new AtomicInteger(10)::incrementAndGet); + Schema expectedSchema = + new Schema(required(11, "id", IntegerType.get()), optional(12, "data", testType)); + assertThat(assignedSchema.asStruct()).isEqualTo(expectedSchema.asStruct()); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testReassignIdsWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + Schema sourceSchema = + new Schema(required(1, "id", IntegerType.get()), optional(2, "data", testType)); + + Schema reassignedSchema = TypeUtil.reassignIds(schema, sourceSchema); + assertThat(reassignedSchema.asStruct()).isEqualTo(sourceSchema.asStruct()); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testIndexByIdWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + + Map indexByIds = TypeUtil.indexById(schema.asStruct()); + assertThat(indexByIds.get(1).type()).isEqualTo(testType); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testIndexNameByIdWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + + Map indexNameByIds = TypeUtil.indexNameById(schema.asStruct()); + assertThat(indexNameByIds.get(1)).isEqualTo("data"); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testProjectWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + + Schema expectedSchema = new Schema(optional(1, "data", testType)); + Schema projectedSchema = TypeUtil.project(schema, Sets.newHashSet(1)); + assertThat(projectedSchema.asStruct()).isEqualTo(expectedSchema.asStruct()); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testGetProjectedIdsWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + + Set projectedIds = TypeUtil.getProjectedIds(schema); + assertThat(Set.of(0, 1)).isEqualTo(projectedIds); + } + + @ParameterizedTest + @MethodSource("testTypes") + public void testReassignDocWithType(Type testType) { + Schema schema = new Schema(required(0, "id", IntegerType.get()), optional(1, "data", testType)); + Schema docSourceSchema = + new Schema( + required(0, "id", IntegerType.get(), "id"), optional(1, "data", testType, "data")); + + Schema reassignedSchema = TypeUtil.reassignDoc(schema, docSourceSchema); + assertThat(reassignedSchema.asStruct()).isEqualTo(docSourceSchema.asStruct()); + } } diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypes.java b/api/src/test/java/org/apache/iceberg/types/TestTypes.java index b464317c2f87..f8ee4e2ccbd4 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypes.java @@ -27,6 +27,30 @@ public class TestTypes { + @Test + public void fromTypeName() { + assertThat(Types.fromTypeName("boolean")).isSameAs(Types.BooleanType.get()); + assertThat(Types.fromTypeName("BooLean")).isSameAs(Types.BooleanType.get()); + + assertThat(Types.fromTypeName("timestamp")).isSameAs(Types.TimestampType.withoutZone()); + assertThat(Types.fromTypeName("timestamptz")).isSameAs(Types.TimestampType.withZone()); + assertThat(Types.fromTypeName("timestamp_ns")).isSameAs(Types.TimestampNanoType.withoutZone()); + assertThat(Types.fromTypeName("timestamptz_ns")).isSameAs(Types.TimestampNanoType.withZone()); + + assertThat(Types.fromTypeName("Fixed[ 3 ]")).isEqualTo(Types.FixedType.ofLength(3)); + + assertThat(Types.fromTypeName("Decimal( 2 , 3 )")).isEqualTo(Types.DecimalType.of(2, 3)); + + assertThat(Types.fromTypeName("Decimal(2,3)")).isEqualTo(Types.DecimalType.of(2, 3)); + + assertThat(Types.fromTypeName("variant")).isSameAs(Types.VariantType.get()); + assertThat(Types.fromTypeName("Variant")).isSameAs(Types.VariantType.get()); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> Types.fromTypeName("abcdefghij")) + .withMessage("Cannot parse type string to primitive: abcdefghij"); + } + @Test public void fromPrimitiveString() { assertThat(Types.fromPrimitiveString("boolean")).isSameAs(Types.BooleanType.get()); @@ -45,6 +69,13 @@ public void fromPrimitiveString() { assertThat(Types.fromPrimitiveString("Decimal(2,3)")).isEqualTo(Types.DecimalType.of(2, 3)); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> Types.fromPrimitiveString("variant")) + .withMessage("Cannot parse type string: variant is not a primitive type"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> Types.fromPrimitiveString("Variant")) + .withMessage("Cannot parse type string: variant is not a primitive type"); + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> Types.fromPrimitiveString("abcdefghij")) .withMessage("Cannot parse type string to primitive: abcdefghij"); diff --git a/baseline.gradle b/baseline.gradle index 07889d4e348c..25dab14afb70 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -153,9 +153,11 @@ subprojects { '-Xep:UnicodeEscape:ERROR', // Palantir's UnnecessarilyQualified may throw during analysis '-Xep:UnnecessarilyQualified:OFF', + '-Xep:UnnecessaryLambdaArgumentParentheses:ERROR', '-Xep:UnnecessaryLongToIntConversion:ERROR', '-Xep:UnnecessaryMethodReference:ERROR', '-Xep:UnusedMethod:ERROR', + '-Xep:UnusedTypeParameter:ERROR', '-Xep:UnusedVariable:ERROR', '-Xep:UseEnumSwitch:ERROR', ) diff --git a/core/src/main/java/org/apache/iceberg/SchemaParser.java b/core/src/main/java/org/apache/iceberg/SchemaParser.java index 64d3f8795db0..d7c756795711 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaParser.java +++ b/core/src/main/java/org/apache/iceberg/SchemaParser.java @@ -145,8 +145,8 @@ static void toJson(Type.PrimitiveType primitive, JsonGenerator generator) throws } static void toJson(Type type, JsonGenerator generator) throws IOException { - if (type.isPrimitiveType()) { - toJson(type.asPrimitiveType(), generator); + if (type.isPrimitiveType() || type.isVariantType()) { + generator.writeString(type.toString()); } else { Type.NestedType nested = type.asNestedType(); switch (type.typeId()) { @@ -181,7 +181,7 @@ public static String toJson(Schema schema, boolean pretty) { private static Type typeFromJson(JsonNode json) { if (json.isTextual()) { - return Types.fromPrimitiveString(json.asText()); + return Types.fromTypeName(json.asText()); } else if (json.isObject()) { JsonNode typeObj = json.get(TYPE); if (typeObj != null) { diff --git a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java index 4b6d7f6cdd6e..db02a0e96e10 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -742,6 +742,11 @@ public Type map(Types.MapType map, Type kResult, Type valueResult) { } } + @Override + public Type variant(Types.VariantType variant) { + return variant; + } + @Override public Type primitive(Type.PrimitiveType primitive) { return primitive; diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 25a6cc96ce39..ea61f300b666 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -133,6 +133,11 @@ static TableMetadata newTableMetadata( int freshSortOrderId = sortOrder.isUnsorted() ? sortOrder.orderId() : INITIAL_SORT_ORDER_ID; SortOrder freshSortOrder = freshSortOrder(freshSortOrderId, freshSchema, sortOrder); + // configure row lineage using table properties + Boolean rowLineage = + PropertyUtil.propertyAsBoolean( + properties, TableProperties.ROW_LINEAGE, DEFAULT_ROW_LINEAGE); + // Validate the metrics configuration. Note: we only do this on new tables to we don't // break existing tables. MetricsConfig.fromProperties(properties).validateReferencedColumns(schema); @@ -146,6 +151,7 @@ static TableMetadata newTableMetadata( .setDefaultSortOrder(freshSortOrder) .setLocation(location) .setProperties(properties) + .setRowLineage(rowLineage) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index ba8f2f78c5e9..4ac0364a15f2 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -216,11 +216,9 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw // write properties map JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator); - if (metadata.currentSnapshot() != null) { - generator.writeNumberField(CURRENT_SNAPSHOT_ID, metadata.currentSnapshot().snapshotId()); - } else { - generator.writeNullField(CURRENT_SNAPSHOT_ID); - } + generator.writeNumberField( + CURRENT_SNAPSHOT_ID, + metadata.currentSnapshot() != null ? metadata.currentSnapshot().snapshotId() : -1); if (metadata.rowLineageEnabled()) { generator.writeBooleanField(ROW_LINEAGE, metadata.rowLineageEnabled()); diff --git a/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java b/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java index a3817b8ad911..72b2a6a783bf 100644 --- a/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java +++ b/core/src/main/java/org/apache/iceberg/mapping/MappingUtil.java @@ -304,6 +304,11 @@ public MappedFields map(Types.MapType map, MappedFields keyResult, MappedFields MappedField.of(map.valueId(), "value", valueResult)); } + @Override + public MappedFields variant(Types.VariantType variant) { + return null; // no mapping because variant has no nested fields with IDs + } + @Override public MappedFields primitive(Type.PrimitiveType primitive) { return null; // no mapping because primitives have no nested fields diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 980a7addda1a..1cde2b8ad43c 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -444,12 +444,15 @@ public void renameTable(SessionContext context, TableIdentifier from, TableIdent @Override public boolean tableExists(SessionContext context, TableIdentifier identifier) { - Endpoint.check(endpoints, Endpoint.V1_TABLE_EXISTS); - try { checkIdentifierIsValid(identifier); - client.head(paths.table(identifier), headers(context), ErrorHandlers.tableErrorHandler()); - return true; + if (endpoints.contains(Endpoint.V1_TABLE_EXISTS)) { + client.head(paths.table(identifier), headers(context), ErrorHandlers.tableErrorHandler()); + return true; + } else { + // fallback in order to work with 1.7.x and older servers + return super.tableExists(context, identifier); + } } catch (NoSuchTableException e) { return false; } @@ -665,13 +668,16 @@ public List listNamespaces(SessionContext context, Namespace namespac @Override public boolean namespaceExists(SessionContext context, Namespace namespace) { - Endpoint.check(endpoints, Endpoint.V1_NAMESPACE_EXISTS); - try { checkNamespaceIsValid(namespace); - client.head( - paths.namespace(namespace), headers(context), ErrorHandlers.namespaceErrorHandler()); - return true; + if (endpoints.contains(Endpoint.V1_NAMESPACE_EXISTS)) { + client.head( + paths.namespace(namespace), headers(context), ErrorHandlers.namespaceErrorHandler()); + return true; + } else { + // fallback in order to work with 1.7.x and older servers + return super.namespaceExists(context, namespace); + } } catch (NoSuchNamespaceException e) { return false; } @@ -1239,12 +1245,15 @@ public List listViews(SessionContext context, Namespace namespa @Override public boolean viewExists(SessionContext context, TableIdentifier identifier) { - Endpoint.check(endpoints, Endpoint.V1_VIEW_EXISTS); - try { checkViewIdentifierIsValid(identifier); - client.head(paths.view(identifier), headers(context), ErrorHandlers.viewErrorHandler()); - return true; + if (endpoints.contains(Endpoint.V1_VIEW_EXISTS)) { + client.head(paths.view(identifier), headers(context), ErrorHandlers.viewErrorHandler()); + return true; + } else { + // fallback in order to work with 1.7.x and older servers + return super.viewExists(context, identifier); + } } catch (NoSuchViewException e) { return false; } diff --git a/core/src/main/java/org/apache/iceberg/schema/SchemaWithPartnerVisitor.java b/core/src/main/java/org/apache/iceberg/schema/SchemaWithPartnerVisitor.java index 9b2226f5714d..694bfb2f6242 100644 --- a/core/src/main/java/org/apache/iceberg/schema/SchemaWithPartnerVisitor.java +++ b/core/src/main/java/org/apache/iceberg/schema/SchemaWithPartnerVisitor.java @@ -107,6 +107,9 @@ public static T visit( return visitor.map(map, partner, keyResult, valueResult); + case VARIANT: + return visitor.variant(type.asVariantType(), partner); + default: return visitor.primitive(type.asPrimitiveType(), partner); } @@ -160,6 +163,10 @@ public R map(Types.MapType map, P partner, R keyResult, R valueResult) { return null; } + public R variant(Types.VariantType variant, P partner) { + throw new UnsupportedOperationException("Unsupported type: variant"); + } + public R primitive(Type.PrimitiveType primitive, P partner) { return null; } diff --git a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java index b7ac23816a02..c3b9a50b2081 100644 --- a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java +++ b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java @@ -142,6 +142,11 @@ public Boolean map( return false; } + @Override + public Boolean variant(Types.VariantType variant, Integer partnerId) { + return partnerId == null; + } + @Override public Boolean primitive(Type.PrimitiveType primitive, Integer partnerId) { return partnerId == null; diff --git a/core/src/main/java/org/apache/iceberg/types/FixupTypes.java b/core/src/main/java/org/apache/iceberg/types/FixupTypes.java index 23fccddda3d9..1e4c0b597a6a 100644 --- a/core/src/main/java/org/apache/iceberg/types/FixupTypes.java +++ b/core/src/main/java/org/apache/iceberg/types/FixupTypes.java @@ -147,6 +147,12 @@ public Type map(Types.MapType map, Supplier keyTypeFuture, Supplier } } + @Override + public Type variant(Types.VariantType variant) { + // nothing to fix up + return variant; + } + @Override public Type primitive(Type.PrimitiveType primitive) { if (sourceType.equals(primitive)) { diff --git a/core/src/main/java/org/apache/iceberg/variants/Variants.java b/core/src/main/java/org/apache/iceberg/variants/Variants.java index 7b14c4c38d0f..96ef0bbb5ba6 100644 --- a/core/src/main/java/org/apache/iceberg/variants/Variants.java +++ b/core/src/main/java/org/apache/iceberg/variants/Variants.java @@ -52,6 +52,10 @@ enum BasicType { ARRAY } + public static VariantMetadata emptyMetadata() { + return SerializedMetadata.EMPTY_V1_METADATA; + } + public static VariantMetadata metadata(ByteBuffer metadata) { return SerializedMetadata.from(metadata); } @@ -89,59 +93,59 @@ public static VariantPrimitive ofNull() { return new PrimitiveWrapper<>(PhysicalType.NULL, null); } - static VariantPrimitive of(boolean value) { + public static VariantPrimitive of(boolean value) { return new PrimitiveWrapper<>(PhysicalType.BOOLEAN_TRUE, value); } - static VariantPrimitive of(byte value) { + public static VariantPrimitive of(byte value) { return new PrimitiveWrapper<>(PhysicalType.INT8, value); } - static VariantPrimitive of(short value) { + public static VariantPrimitive of(short value) { return new PrimitiveWrapper<>(PhysicalType.INT16, value); } - static VariantPrimitive of(int value) { + public static VariantPrimitive of(int value) { return new PrimitiveWrapper<>(PhysicalType.INT32, value); } - static VariantPrimitive of(long value) { + public static VariantPrimitive of(long value) { return new PrimitiveWrapper<>(PhysicalType.INT64, value); } - static VariantPrimitive of(float value) { + public static VariantPrimitive of(float value) { return new PrimitiveWrapper<>(PhysicalType.FLOAT, value); } - static VariantPrimitive of(double value) { + public static VariantPrimitive of(double value) { return new PrimitiveWrapper<>(PhysicalType.DOUBLE, value); } - static VariantPrimitive ofDate(int value) { + public static VariantPrimitive ofDate(int value) { return new PrimitiveWrapper<>(PhysicalType.DATE, value); } - static VariantPrimitive ofIsoDate(String value) { + public static VariantPrimitive ofIsoDate(String value) { return ofDate(DateTimeUtil.isoDateToDays(value)); } - static VariantPrimitive ofTimestamptz(long value) { + public static VariantPrimitive ofTimestamptz(long value) { return new PrimitiveWrapper<>(PhysicalType.TIMESTAMPTZ, value); } - static VariantPrimitive ofIsoTimestamptz(String value) { + public static VariantPrimitive ofIsoTimestamptz(String value) { return ofTimestamptz(DateTimeUtil.isoTimestamptzToMicros(value)); } - static VariantPrimitive ofTimestampntz(long value) { + public static VariantPrimitive ofTimestampntz(long value) { return new PrimitiveWrapper<>(PhysicalType.TIMESTAMPNTZ, value); } - static VariantPrimitive ofIsoTimestampntz(String value) { + public static VariantPrimitive ofIsoTimestampntz(String value) { return ofTimestampntz(DateTimeUtil.isoTimestampToMicros(value)); } - static VariantPrimitive of(BigDecimal value) { + public static VariantPrimitive of(BigDecimal value) { int bitLength = value.unscaledValue().bitLength(); if (bitLength < 32) { return new PrimitiveWrapper<>(PhysicalType.DECIMAL4, value); @@ -154,11 +158,11 @@ static VariantPrimitive of(BigDecimal value) { throw new UnsupportedOperationException("Unsupported decimal precision: " + value.precision()); } - static VariantPrimitive of(ByteBuffer value) { + public static VariantPrimitive of(ByteBuffer value) { return new PrimitiveWrapper<>(PhysicalType.BINARY, value); } - static VariantPrimitive of(String value) { + public static VariantPrimitive of(String value) { return new PrimitiveWrapper<>(PhysicalType.STRING, value); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java index 0fb9ee880f6b..3a72f654038a 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageMetadata.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.primitives.Ints; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterEach; @@ -306,6 +307,20 @@ public void testEnableRowLineageViaProperty() { assertThat(table.ops().current().rowLineageEnabled()).isTrue(); } + @TestTemplate + public void testEnableRowLineageViaPropertyAtTableCreation() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(TableMetadata.MIN_FORMAT_VERSION_ROW_LINEAGE); + + TestTables.TestTable table = + TestTables.create( + tableDir, + "test", + TEST_SCHEMA, + ImmutableMap.of(TableProperties.ROW_LINEAGE, "true"), + formatVersion); + assertThat(table.ops().current().rowLineageEnabled()).isTrue(); + } + private final AtomicInteger fileNum = new AtomicInteger(0); private DataFile fileWithRows(long numRows) { diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaParser.java b/core/src/test/java/org/apache/iceberg/TestSchemaParser.java index 108371416b14..acdba85adf55 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaParser.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaParser.java @@ -127,4 +127,14 @@ public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal d assertThat(serialized.findField("col_with_default").writeDefault()) .isEqualTo(defaultValue.value()); } + + @Test + public void testVariantType() throws IOException { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.VariantType.get())); + + writeAndValidate(schema); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java index 3a61ce8a1513..aa478f85260e 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java @@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.BinaryType; import org.apache.iceberg.types.Types.BooleanType; @@ -43,13 +43,16 @@ import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampNanoType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; +import org.apache.iceberg.types.Types.UnknownType; +import org.apache.iceberg.types.Types.VariantType; import org.junit.jupiter.api.Test; public class TestSchemaUnionByFieldName { - private static List primitiveTypes() { + private static List primitiveTypes() { return Lists.newArrayList( StringType.get(), TimeType.get(), @@ -64,11 +67,15 @@ private static List primitiveTypes() { FixedType.ofLength(10), DecimalType.of(10, 2), LongType.get(), - FloatType.get()); + FloatType.get(), + VariantType.get(), + UnknownType.get(), + TimestampNanoType.withoutZone(), + TimestampNanoType.withZone()); } private static NestedField[] primitiveFields( - Integer initialValue, List primitiveTypes) { + Integer initialValue, List primitiveTypes) { AtomicInteger atomicInteger = new AtomicInteger(initialValue); return primitiveTypes.stream() .map( @@ -76,7 +83,7 @@ private static NestedField[] primitiveFields( optional( atomicInteger.incrementAndGet(), type.toString(), - Types.fromPrimitiveString(type.toString()))) + Types.fromTypeName(type.toString()))) .toArray(NestedField[]::new); } @@ -104,7 +111,7 @@ public void testAddFieldWithDefault() { @Test public void testAddTopLevelListOfPrimitives() { - for (PrimitiveType primitiveType : primitiveTypes()) { + for (Type primitiveType : primitiveTypes()) { Schema newSchema = new Schema(optional(1, "aList", Types.ListType.ofOptional(2, primitiveType))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); @@ -114,7 +121,13 @@ public void testAddTopLevelListOfPrimitives() { @Test public void testAddTopLevelMapOfPrimitives() { - for (PrimitiveType primitiveType : primitiveTypes()) { + for (Type primitiveType : primitiveTypes()) { + if (primitiveType.equals(UnknownType.get())) { + // The UnknownType has to be optional, and this conflicts with the map key that must be + // required + continue; + } + Schema newSchema = new Schema( optional(1, "aMap", Types.MapType.ofOptional(2, 3, primitiveType, primitiveType))); @@ -125,7 +138,7 @@ public void testAddTopLevelMapOfPrimitives() { @Test public void testAddTopLevelStructOfPrimitives() { - for (PrimitiveType primitiveType : primitiveTypes()) { + for (Type primitiveType : primitiveTypes()) { Schema currentSchema = new Schema( optional(1, "aStruct", Types.StructType.of(optional(2, "primitive", primitiveType)))); @@ -136,7 +149,7 @@ public void testAddTopLevelStructOfPrimitives() { @Test public void testAddNestedPrimitive() { - for (PrimitiveType primitiveType : primitiveTypes()) { + for (Type primitiveType : primitiveTypes()) { Schema currentSchema = new Schema(optional(1, "aStruct", Types.StructType.of())); Schema newSchema = new Schema( diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index 38cf6da18a3e..d1591f80d836 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -2439,4 +2439,47 @@ public void testMoveDeletedNestedStructFieldToFirst() { assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } + + @Test + public void testAddUnknown() { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + Schema expected = + new Schema( + required(1, "id", Types.LongType.get()), optional(2, "unk", Types.UnknownType.get())); + + Schema actual = + new SchemaUpdate(schema, schema.highestFieldId()) + .addColumn("unk", Types.UnknownType.get()) + .apply(); + + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); + } + + @Test + public void testAddUnknownNonNullDefault() { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + + assertThatThrownBy( + () -> + new SchemaUpdate(schema, schema.highestFieldId()) + .allowIncompatibleChanges() + .addColumn("unk", Types.UnknownType.get(), Literal.of("string!")) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot cast default value to unknown: \"string!\""); + } + + @Test + public void testAddRequiredUnknown() { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + + assertThatThrownBy( + () -> + new SchemaUpdate(schema, schema.highestFieldId()) + .allowIncompatibleChanges() + .addRequiredColumn("unk", Types.UnknownType.get()) + .apply()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot create required field with unknown type: unk"); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestTables.java b/core/src/test/java/org/apache/iceberg/TestTables.java index eeff5db8e5a6..de4aff8142b9 100644 --- a/core/src/test/java/org/apache/iceberg/TestTables.java +++ b/core/src/test/java/org/apache/iceberg/TestTables.java @@ -52,6 +52,26 @@ public static TestTable create( return create(temp, name, schema, spec, SortOrder.unsorted(), formatVersion); } + public static TestTable create( + File temp, String name, Schema schema, Map properties, int formatVersion) { + TestTableOperations ops = new TestTableOperations(name, temp); + if (ops.current() != null) { + throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp); + } + + ops.commit( + null, + newTableMetadata( + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + temp.toString(), + properties, + formatVersion)); + + return new TestTable(ops, name); + } + public static TestTable create( File temp, String name, diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java b/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java index d30a93d50d49..3ebf4d9242ab 100644 --- a/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java +++ b/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java @@ -289,4 +289,16 @@ public void testMappingFindByName() { "location", MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12, "longitude")))); } + + @Test + public void testMappingVariantType() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), required(2, "data", Types.VariantType.get())); + + MappedFields expected = MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")); + + NameMapping mapping = MappingUtil.create(schema); + assertThat(mapping.asMappedFields()).isEqualTo(expected); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 6a5f22075c6e..696240bb6da2 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -66,6 +66,7 @@ import org.apache.iceberg.exceptions.ServiceFailureException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -2505,6 +2506,127 @@ public void testNamespaceExistsViaHEADRequest() { any()); } + @Test + public void testNamespaceExistsFallbackToGETRequest() { + RESTCatalogAdapter adapter = + Mockito.spy( + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPRequest request, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders) { + if ("v1/config".equals(request.path())) { + return castResponse( + responseType, + ConfigResponse.builder() + // server indicates support of loading a namespace only via GET, which is + // what older REST servers would send back too + .withEndpoints(ImmutableList.of(Endpoint.V1_LOAD_NAMESPACE)) + .build()); + } + + return super.execute(request, responseType, errorHandler, responseHeaders); + } + }); + + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of()); + + assertThat(catalog.namespaceExists(Namespace.of("non-existing"))).isFalse(); + + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.GET, "v1/config", Map.of(), Map.of()), + eq(ConfigResponse.class), + any(), + any()); + + // verifies that the namespace is loaded via a GET instead of HEAD (V1_NAMESPACE_EXISTS) + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.GET, "v1/namespaces/non-existing", Map.of(), Map.of()), + any(), + any(), + any()); + } + + @Test + public void testTableExistsViaHEADRequest() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of()); + + assertThat(catalog.tableExists(TABLE)).isFalse(); + + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.GET, "v1/config", Map.of(), Map.of()), + eq(ConfigResponse.class), + any(), + any()); + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.HEAD, "v1/namespaces/newdb/tables/table", Map.of(), Map.of()), + any(), + any(), + any()); + } + + @Test + public void testTableExistsFallbackToGETRequest() { + RESTCatalogAdapter adapter = + Mockito.spy( + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPRequest request, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders) { + if ("v1/config".equals(request.path())) { + return castResponse( + responseType, + ConfigResponse.builder() + // server indicates support of loading a table only via GET, which is + // what older REST servers would send back too + .withEndpoints(ImmutableList.of(Endpoint.V1_LOAD_TABLE)) + .build()); + } + + return super.execute(request, responseType, errorHandler, responseHeaders); + } + }); + + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of()); + + assertThat(catalog.tableExists(TABLE)).isFalse(); + + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.GET, "v1/config", Map.of(), Map.of()), + eq(ConfigResponse.class), + any(), + any()); + + // verifies that the table is loaded via a GET instead of HEAD (V1_LOAD_TABLE) + Mockito.verify(adapter) + .execute( + reqMatcher( + HTTPMethod.GET, + "v1/namespaces/newdb/tables/table", + Map.of(), + Map.of("snapshots", "all")), + any(), + any(), + any()); + } + private RESTCatalog catalog(RESTCatalogAdapter adapter) { RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java index 8dfbf0df6dd7..73ff5f052a11 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -38,6 +38,7 @@ import org.apache.iceberg.catalog.SessionCatalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.HTTPRequest.HTTPMethod; import org.apache.iceberg.rest.responses.ConfigResponse; @@ -243,6 +244,51 @@ public void viewExistsViaHEADRequest() { any()); } + @Test + public void viewExistsFallbackToGETRequest() { + RESTCatalogAdapter adapter = + Mockito.spy( + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPRequest request, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders) { + if ("v1/config".equals(request.path())) { + return castResponse( + responseType, + ConfigResponse.builder() + // server indicates support of loading a view only via GET, which is + // what older REST servers would send back too + .withEndpoints(ImmutableList.of(Endpoint.V1_LOAD_VIEW)) + .build()); + } + + return super.execute(request, responseType, errorHandler, responseHeaders); + } + }); + + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of()); + + assertThat(catalog.viewExists(TableIdentifier.of("ns", "view"))).isFalse(); + + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.GET, "v1/config", Map.of(), Map.of()), + eq(ConfigResponse.class), + any(), + any()); + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.GET, "v1/namespaces/ns/views/view", Map.of(), Map.of()), + any(), + any(), + any()); + } + @Override protected RESTCatalog catalog() { return restCatalog; diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java index b4879dd39f51..cc6f4cfc74d7 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java @@ -107,7 +107,7 @@ public void roundTripSerde() { + " \"fields\" : [ ]\n" + " } ],\n" + " \"properties\" : { },\n" - + " \"current-snapshot-id\" : null,\n" + + " \"current-snapshot-id\" : -1,\n" + " \"refs\" : { },\n" + " \"snapshots\" : [ ],\n" + " \"statistics\" : [ ],\n" @@ -180,7 +180,7 @@ public void roundTripSerdeWithConfig() { + " \"fields\" : [ ]\n" + " } ],\n" + " \"properties\" : { },\n" - + " \"current-snapshot-id\" : null,\n" + + " \"current-snapshot-id\" : -1,\n" + " \"refs\" : { },\n" + " \"snapshots\" : [ ],\n" + " \"statistics\" : [ ],\n" @@ -283,7 +283,7 @@ public void roundTripSerdeWithCredentials() { + " \"fields\" : [ ]\n" + " } ],\n" + " \"properties\" : { },\n" - + " \"current-snapshot-id\" : null,\n" + + " \"current-snapshot-id\" : -1,\n" + " \"refs\" : { },\n" + " \"snapshots\" : [ ],\n" + " \"statistics\" : [ ],\n" diff --git a/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java b/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java index 576e06a9d1c5..e2973b78b9ea 100644 --- a/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java +++ b/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.variants; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.charset.StandardCharsets; @@ -27,10 +29,55 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public class VariantTestUtil { private VariantTestUtil() {} + public static void assertEqual(VariantMetadata expected, VariantMetadata actual) { + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); + assertThat(actual.dictionarySize()) + .as("Dictionary size should match") + .isEqualTo(expected.dictionarySize()); + + for (int i = 0; i < expected.dictionarySize(); i += 1) { + assertThat(actual.get(i)).isEqualTo(expected.get(i)); + } + } + + public static void assertEqual(VariantValue expected, VariantValue actual) { + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); + assertThat(actual.type()).as("Variant type should match").isEqualTo(expected.type()); + + if (expected.type() == PhysicalType.OBJECT) { + VariantObject expectedObject = expected.asObject(); + VariantObject actualObject = actual.asObject(); + assertThat(actualObject.numFields()) + .as("Variant object num fields should match") + .isEqualTo(expectedObject.numFields()); + for (String fieldName : expectedObject.fieldNames()) { + assertEqual(expectedObject.get(fieldName), actualObject.get(fieldName)); + } + + } else if (expected.type() == PhysicalType.ARRAY) { + VariantArray expectedArray = expected.asArray(); + VariantArray actualArray = actual.asArray(); + assertThat(actualArray.numElements()) + .as("Variant array num element should match") + .isEqualTo(expectedArray.numElements()); + for (int i = 0; i < expectedArray.numElements(); i += 1) { + assertEqual(expectedArray.get(i), actualArray.get(i)); + } + + } else { + assertThat(actual.asPrimitive().get()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + } + } + private static byte primitiveHeader(int primitiveType) { return (byte) (primitiveType << 2); } @@ -60,7 +107,11 @@ static SerializedPrimitive createString(String string) { return SerializedPrimitive.from(buffer, buffer.get(0)); } - static ByteBuffer createMetadata(Collection fieldNames, boolean sortNames) { + public static ByteBuffer emptyMetadata() { + return createMetadata(ImmutableList.of(), true); + } + + public static ByteBuffer createMetadata(Collection fieldNames, boolean sortNames) { if (fieldNames.isEmpty()) { return SerializedMetadata.EMPTY_V1_BUFFER; } @@ -108,7 +159,7 @@ static ByteBuffer createMetadata(Collection fieldNames, boolean sortName return buffer; } - static ByteBuffer createObject(ByteBuffer metadataBuffer, Map data) { + public static ByteBuffer createObject(ByteBuffer metadataBuffer, Map data) { // create the metadata to look up field names VariantMetadata metadata = Variants.metadata(metadataBuffer); diff --git a/docs/docs/amoro.md b/docs/docs/amoro.md new file mode 100644 index 000000000000..f065d7e693a2 --- /dev/null +++ b/docs/docs/amoro.md @@ -0,0 +1,69 @@ +--- +title: "Apache Amoro" +--- + + +# Apache Amoro With Iceberg + +**[Apache Amoro(incubating)](https://amoro.apache.org)** is a Lakehouse management system built on open data lake formats. Working with compute engines including Flink, Spark, and Trino, Amoro brings pluggable and +**[Table Maintenance](https://amoro.apache.org/docs/latest/self-optimizing/)** features for a Lakehouse to provide out-of-the-box data warehouse experience, and helps data platforms or products easily build infra-decoupled, stream-and-batch-fused and lake-native architecture. +**[AMS](https://amoro.apache.org/docs/latest/#architecture)(Amoro Management Service)** provides Lakehouse management features, like self-optimizing, data expiration, etc. It also provides a unified catalog service for all compute engines, which can also be combined with existing metadata services like HMS(Hive Metastore). + +## Auto Self-optimizing + +Amoro has introduced a Self-optimizing mechanism to +create an out-of-the-box Streaming Lakehouse management service that is as user-friendly as a traditional database or data warehouse. Self-optimizing involves various procedures such as file compaction, deduplication, and sorting. + +The architecture and working mechanism of Self-optimizing are shown in the figure below: + +![Self-optimizing architecture](https://amoro.apache.org/docs/latest/images/concepts/self-optimizing_arch.png) + +The Optimizer is a component responsible for executing Self-optimizing tasks. It is a resident process managed by [AMS](https://amoro.apache.org/docs/latest/#architecture). AMS is responsible for +detecting and planning Self-optimizing tasks for tables, and then scheduling them to Optimizers for distributed execution in real-time. Finally, AMS +is responsible for submitting the optimizing results. Amoro achieves physical isolation of Optimizers through the Optimizer Group. + +The core features of [Amoro Self Optimizing](https://amoro.apache.org/docs/latest/self-optimizing/) are: + +- Automated, Asynchronous and Transparent — Continuous background detecting of file changes, asynchronous distributed execution of optimizing tasks, + transparent and imperceptible to users +- Resource Isolation and Sharing — Allow resources to be isolated and shared at the table level, as well as setting resource quotas +- Flexible and Scalable Deployment — Optimizers support various deployment methods and convenient scaling + +## Table Format + +Apache Amoro supports all catalog types supported by Iceberg, including common catalog: [REST](https://editor-next.swagger.io/?url=https://raw.githubusercontent.com/apache/iceberg/main/open-api/rest-catalog-open-api.yaml), Hadoop, Hive, Glue, JDBC, Nessie and other third-party catalog. +Amoro supports all storage types supported by Iceberg, including common store: Hadoop, S3, GCS, ECS, OSS, and so on. + +At the same time, we also provide a unique form based on Apache Iceberg, including mixed-Iceberg Format and mixed-Hive Format, so that you can quickly upgrade to the iceberg+hive Mixed table while compatible with the original Hive data + +### Iceberg Format + +Starting from Apache Amoro v0.4, Iceberg format including v1 and v2 is supported. Users only need to register Iceberg’s catalog in Amoro to host the table for Amoro maintenance. Amoro maintains the performance and economic availability of Iceberg tables with minimal read/write costs through means such as small file merging, eq-delete file conversion to pos-delete files, +duplicate data elimination, and file cleaning, and Amoro has no intrusive impact on the functionality of Iceberg. + +### Mixed-Iceberg Format + +[Mixed-Iceberg Format](https://amoro.apache.org/docs/latest/mixed-iceberg-format/) is similar to that of clustered indexes in databases. Each TableStore can use different table formats. Mixed-Iceberg format provides high freshness OLAP through merge-on-read between BaseStore and ChangeStore. To provide high-performance merge-on-read, BaseStore and ChangeStore use completely consistent partition and layout, and both support auto-bucket. + +- BaseStore — stores the stock data of the table, usually generated by batch computing or optimizing processes, and is more friendly to ReadStore for reading. +- ChangeStore — stores the flow and change data of the table, usually written in real-time by streaming computing, and can also be used for downstream CDC consumption, and is more friendly to WriteStore for writing. +- LogStore — serves as a cache layer for ChangeStore to accelerate stream processing. Amoro manages the consistency between LogStore and ChangeStore. + +### Mixed-Hive Format + +[Mixed-Hive](https://amoro.apache.org/docs/latest/mixed-hive-format/) format is a format that has better compatibility with Hive than Mixed-Iceberg format. Mixed-Hive format uses a Hive table as the BaseStore and an Iceberg table as the ChangeStore. diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index c58c68c5873c..288922c36413 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -974,4 +974,94 @@ CALL catalog_name.system.compute_table_stats(table => 'my_table', snapshot_id => Collect statistics of the snapshot with id `snap1` of table `my_table` for columns `col1` and `col2` ```sql CALL catalog_name.system.compute_table_stats(table => 'my_table', snapshot_id => 'snap1', columns => array('col1', 'col2')); -``` \ No newline at end of file +``` + +## Table Replication + +The `rewrite_table_path` procedure prepares an Iceberg table for copying to another location. + +### `rewrite_table_path` + +Stages a copy of the Iceberg table's metadata files where every absolute path source prefix is replaced by the specified target prefix. +This can be the starting point to fully or incrementally copy an Iceberg table to a new location. + +!!! info + This procedure only stages rewritten metadata files and prepares a list of files to copy. The actual file copy is not included in this procedure. + + +| Argument Name | Required? | default | Type | Description | +|--------------------|-----------|------------------------------------------------|--------|------------------------------------------------------------------------| +| `table` | ✔️ | | string | Name of the table | +| `source_prefix` | ✔️ | | string | The existing prefix to be replaced | +| `target_prefix` | ✔️ | | string | The replacement prefix for `source_prefix` | +| `start_version` | | first metadata.json in table's metadata log | string | The name or path of the chronologically first metadata.json to rewrite | +| `end_version` | | latest metadata.json in table's metadata log | string | The name or path of the chronologically last metadata.json to rewrite | +| `staging_location` | | new directory under table's metadata directory | string | The output location for newly rewritten metadata files | + + +#### Modes of operation + +* Full Rewrite: A full rewrite will rewrite all reachable metadata files (this includes metadata.json, manifest lists, manifests, and position delete files), and will return all reachable files in the `file_list_location`. This is the default mode of operation for this procedure. + +* Incremental Rewrite: Optionally, `start_version` and `end_version` can be provided to limit the scope to an incremental rewrite. An incremental rewrite will only rewrite metadata files added between `start_version` and `end_version`, and will only return files added in this range in the `file_list_location`. + + +#### Output + +| Output Name | Type | Description | +|----------------------|--------|-------------------------------------------------------------------| +| `latest_version` | string | Name of the latest metadata file rewritten by this procedure | +| `file_list_location` | string | Path to a CSV file containing a mapping of source to target paths | + +##### File List +The file contains the copy plan for all files added to the table between `start_version` and `end_version`. + +For each file, it specifies: + +* Source Path: The original file path in the table, or the staging location if the file has been rewritten + +* Target Path: The path with the replacement prefix + +The following example shows a copy plan for three files: + +```csv +sourcepath/datafile1.parquet,targetpath/datafile1.parquet +sourcepath/datafile2.parquet,targetpath/datafile2.parquet +stagingpath/manifest.avro,targetpath/manifest.avro +``` + +#### Examples + +This example fully rewrites metadata paths of `my_table` from source location in HDFS to a target location in S3. +It will produce a new set of metadata in the default staging location under the table's metadata directory. + +```sql +CALL catalog_name.system.rewrite_table_path( + table => 'db.my_table', + source_prefix => "hdfs://nn:8020/path/to/source_table", + target_prefix => "s3a://bucket/prefix/db.db/my_table" +); +``` + +This example incrementally rewrites metadata paths of `my_table` between metadata versions `v2.metadata.json` and `v20.metadata.json`, +with new metadata files written to an explicit staging location. + +```sql +CALL catalog_name.system.rewrite_table_path( + table => 'db.my_table', + source_prefix => "s3a://bucketOne/prefix/db.db/my_table", + target_prefix => "s3a://bucketTwo/prefix/db.db/my_table", + start_version => "v2.metadata.json", + end_version => "v20.metadata.json", + staging_location => "s3a://bucketStaging/my_table" +); +``` + +Once the rewrite completes, third-party tools ( +eg. [Distcp](https://hadoop.apache.org/docs/current/hadoop-distcp/DistCp.html)) can copy the newly created +metadata files and data files to the target location. + +Lastly, the [register_table](#register_table) procedure can be used to register the copied table in the target location with a catalog. + +!!! warning + Iceberg tables with partition statistics files are not currently supported for path rewrite. \ No newline at end of file diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index bc8809595d6e..7273a1946896 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -58,6 +58,7 @@ nav: - Presto: https://prestodb.io/docs/current/connector/iceberg.html - Dremio: https://docs.dremio.com/data-formats/apache-iceberg/ - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog + - Amoro: amoro.md - Amazon Athena: https://docs.aws.amazon.com/athena/latest/ug/querying-iceberg.html - Amazon EMR: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html - Amazon Data Firehose: https://docs.aws.amazon.com/firehose/latest/dev/apache-iceberg-destination.html diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 2bb96fde3146..162060a3c646 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.27.3" awaitility = "4.2.2" -awssdk-bom = "2.30.16" +awssdk-bom = "2.30.21" azuresdk-bom = "1.2.31" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" @@ -70,7 +70,7 @@ microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.102.5" -netty-buffer = "4.1.117.Final" +netty-buffer = "4.1.118.Final" object-client-bundle = "3.3.2" orc = "1.9.5" parquet = "1.15.0" diff --git a/open-api/Makefile b/open-api/Makefile index d60496cc4b94..6bf7365f2afc 100644 --- a/open-api/Makefile +++ b/open-api/Makefile @@ -24,7 +24,7 @@ lint: generate: datamodel-codegen \ --enum-field-as-literal all \ - --target-python-version 3.8 \ + --target-python-version 3.9 \ --use-schema-description \ --field-constraints \ --input rest-catalog-open-api.yaml \ diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 3dcd0d02e03b..ee36b5196c7c 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.27.2 +datamodel-code-generator==0.28.1 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index b3999264a4a6..2571961cb192 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -507,6 +507,10 @@ class PlanStatus(BaseModel): class RegisterTableRequest(BaseModel): name: str metadata_location: str = Field(..., alias='metadata-location') + overwrite: Optional[bool] = Field( + False, + description='Whether to overwrite table metadata if the table already exists', + ) class TokenType(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 7e3620981a44..a6e56c54c338 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -3479,6 +3479,10 @@ components: type: string metadata-location: type: string + overwrite: + description: Whether to overwrite table metadata if the table already exists + type: boolean + default: false CreateViewRequest: type: object diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java index 70e6b3ff447e..7e972e6cedff 100644 --- a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.data.parquet; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; @@ -26,7 +27,9 @@ import org.apache.iceberg.parquet.ParquetSchemaUtil; import org.apache.iceberg.parquet.ParquetValueReader; import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetVariantVisitor; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.parquet.VariantReaderBuilder; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -431,6 +434,16 @@ public ParquetValueReader primitive( } } + @Override + public ParquetValueReader variant(Types.VariantType iVariant, ParquetValueReader reader) { + return reader; + } + + @Override + public ParquetVariantVisitor> variantVisitor() { + return new VariantReaderBuilder(type, Arrays.asList(currentPath())); + } + MessageType type() { return type; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 7677a53de0c1..ae25305629ef 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -108,6 +108,7 @@ import org.apache.parquet.avro.AvroWriteSupport; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.column.ParquetProperties.WriterVersion; +import org.apache.parquet.conf.PlainParquetConfiguration; import org.apache.parquet.crypto.FileDecryptionProperties; import org.apache.parquet.crypto.FileEncryptionProperties; import org.apache.parquet.hadoop.ParquetFileReader; @@ -283,7 +284,7 @@ private WriteBuilder createContextFunc( return this; } - private void setBloomFilterConfig( + private void setBloomFilterConfig( Context context, MessageType parquetSchema, BiConsumer withBloomFilterEnabled, @@ -1251,7 +1252,7 @@ public CloseableIterable build() { } optionsBuilder = HadoopReadOptions.builder(conf); } else { - optionsBuilder = ParquetReadOptions.builder(); + optionsBuilder = ParquetReadOptions.builder(new PlainParquetConfiguration()); } for (Map.Entry entry : properties.entrySet()) { @@ -1291,7 +1292,7 @@ public CloseableIterable build() { } else { Function> readBuilder = readerFuncWithSchema != null - ? (fileType) -> readerFuncWithSchema.apply(schema, fileType) + ? fileType -> readerFuncWithSchema.apply(schema, fileType) : readerFunc; return new org.apache.iceberg.parquet.ParquetReader<>( file, schema, options, readBuilder, mapping, filter, reuseContainers, caseSensitive); @@ -1324,7 +1325,9 @@ public CloseableIterable build() { // TODO: should not need to get the schema to push down before opening the file. // Parquet should allow setting a filter inside its read support ParquetReadOptions decryptOptions = - ParquetReadOptions.builder().withDecryption(fileDecryptionProperties).build(); + ParquetReadOptions.builder(new PlainParquetConfiguration()) + .withDecryption(fileDecryptionProperties) + .build(); MessageType type; try (ParquetFileReader schemaReader = ParquetFileReader.open(ParquetIO.file(file), decryptOptions)) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index a0dc54c1cdd9..68a9aa979fdf 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -27,6 +27,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.parquet.io.InvalidRecordException; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -75,6 +76,32 @@ private static Schema convertInternal( converter.getAliases()); } + /** + * Returns true if the name identifies a field in the struct/group. + * + * @param group a GroupType + * @param name a String name + * @return true if the group contains a field with the given name + */ + public static boolean hasField(GroupType group, String name) { + return fieldType(group, name) != null; + } + + /** + * Returns the Type of the named field in the struct/group, or null. + * + * @param group a GroupType + * @param name a String name + * @return the Type of the field in the group, or null if it is not present. + */ + public static Type fieldType(GroupType group, String name) { + try { + return group.getType(name); + } catch (InvalidRecordException ignored) { + return null; + } + } + public static MessageType pruneColumns(MessageType fileSchema, Schema expectedSchema) { // column order must match the incoming type, so it doesn't matter that the ids are unordered Set selectedIds = TypeUtil.getProjectedIds(expectedSchema); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java index 73ce83b9bfdd..bb8930085924 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java @@ -63,6 +63,14 @@ public static ParquetValueReader unboxed(ColumnDescriptor desc) { return new UnboxedReader<>(desc); } + public static ParquetValueReader intsAsByte(ColumnDescriptor desc) { + return new IntAsByteReader(desc); + } + + public static ParquetValueReader intsAsShort(ColumnDescriptor desc) { + return new IntAsShortReader(desc); + } + public static ParquetValueReader strings(ColumnDescriptor desc) { return new StringReader(desc); } @@ -390,6 +398,28 @@ public String read(String reuse) { } } + private static class IntAsByteReader extends UnboxedReader { + private IntAsByteReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Byte read(Byte ignored) { + return (byte) readInteger(); + } + } + + private static class IntAsShortReader extends UnboxedReader { + private IntAsShortReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Short read(Short ignored) { + return (short) readInteger(); + } + } + public static class IntAsLongReader extends UnboxedReader { public IntAsLongReader(ColumnDescriptor desc) { super(desc); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantReaders.java new file mode 100644 index 000000000000..b0e132afad5f --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantReaders.java @@ -0,0 +1,424 @@ +/* + * 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.parquet; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.commons.lang3.stream.Streams; +import org.apache.iceberg.parquet.ParquetValueReaders.PrimitiveReader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.variants.PhysicalType; +import org.apache.iceberg.variants.ShreddedObject; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantObject; +import org.apache.iceberg.variants.VariantValue; +import org.apache.iceberg.variants.Variants; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; + +public class ParquetVariantReaders { + private ParquetVariantReaders() {} + + public interface VariantValueReader extends ParquetValueReader { + @Override + default VariantValue read(VariantValue reuse) { + throw new UnsupportedOperationException("Variants must be read using read(VariantMetadata)"); + } + + /** Reads a variant value */ + VariantValue read(VariantMetadata metadata); + } + + private static final VariantValue MISSING = null; + + @SuppressWarnings("unchecked") + public static ParquetValueReader variant( + ParquetValueReader metadata, ParquetValueReader value) { + return new VariantReader( + (ParquetValueReader) metadata, (VariantValueReader) value); + } + + public static ParquetValueReader metadata(ColumnDescriptor desc) { + return new VariantMetadataReader(desc); + } + + public static VariantValueReader serialized(ColumnDescriptor desc) { + return new SerializedVariantReader(desc); + } + + public static VariantValueReader shredded( + int valueDefinitionLevel, + ParquetValueReader valueReader, + int typedDefinitionLevel, + ParquetValueReader typedReader) { + return new ShreddedVariantReader( + valueDefinitionLevel, + (VariantValueReader) valueReader, + typedDefinitionLevel, + (VariantValueReader) typedReader); + } + + public static VariantValueReader objects( + int valueDefinitionLevel, + ParquetValueReader valueReader, + int typedDefinitionLevel, + List fieldNames, + List fieldReaders) { + return new ShreddedObjectReader( + valueDefinitionLevel, + (VariantValueReader) valueReader, + typedDefinitionLevel, + fieldNames, + fieldReaders); + } + + public static VariantValueReader asVariant(PhysicalType type, ParquetValueReader reader) { + return new ValueAsVariantReader<>(type, reader); + } + + private abstract static class DelegatingValueReader implements ParquetValueReader { + private final ParquetValueReader reader; + + private DelegatingValueReader(ParquetValueReader reader) { + this.reader = reader; + } + + protected S readFromDelegate(S reuse) { + return reader.read(reuse); + } + + @Override + public TripleIterator column() { + return reader.column(); + } + + @Override + public List> columns() { + return reader.columns(); + } + + @Override + public void setPageSource(PageReadStore pageStore) { + reader.setPageSource(pageStore); + } + } + + private static ByteBuffer readBinary(ColumnIterator column) { + ByteBuffer data = column.nextBinary().toByteBuffer(); + byte[] array = new byte[data.remaining()]; + data.get(array, 0, data.remaining()); + return ByteBuffer.wrap(array).order(ByteOrder.LITTLE_ENDIAN); + } + + private static class VariantMetadataReader extends PrimitiveReader { + private VariantMetadataReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public VariantMetadata read(VariantMetadata reuse) { + return Variants.metadata(readBinary(column)); + } + } + + private static class SerializedVariantReader extends PrimitiveReader + implements VariantValueReader { + private SerializedVariantReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public VariantValue read(VariantMetadata metadata) { + return Variants.value(metadata, readBinary(column)); + } + } + + private static class ValueAsVariantReader extends DelegatingValueReader + implements VariantValueReader { + private final PhysicalType type; + + private ValueAsVariantReader(PhysicalType type, ParquetValueReader reader) { + super(reader); + this.type = type; + } + + @Override + public VariantValue read(VariantMetadata ignored) { + return Variants.of(type, readFromDelegate(null)); + } + } + + /** + * A Variant reader that combines value and typed_value columns from Parquet. + * + *

This reader does not handle merging partially shredded objects. To handle shredded objects, + * use {@link ShreddedObjectReader}. + */ + private static class ShreddedVariantReader implements VariantValueReader { + private final int valueDefinitionLevel; + private final VariantValueReader valueReader; + private final int typeDefinitionLevel; + private final VariantValueReader typedReader; + private final TripleIterator column; + private final List> children; + + private ShreddedVariantReader( + int valueDefinitionLevel, + VariantValueReader valueReader, + int typeDefinitionLevel, + VariantValueReader typedReader) { + this.valueDefinitionLevel = valueDefinitionLevel; + this.valueReader = valueReader; + this.typeDefinitionLevel = typeDefinitionLevel; + this.typedReader = typedReader; + this.column = valueReader != null ? valueReader.column() : typedReader.column(); + this.children = children(valueReader, typedReader); + } + + @Override + public VariantValue read(VariantMetadata metadata) { + VariantValue value = ParquetVariantReaders.read(metadata, valueReader, valueDefinitionLevel); + VariantValue typed = ParquetVariantReaders.read(metadata, typedReader, typeDefinitionLevel); + + if (typed != null) { + Preconditions.checkArgument( + value == null, + "Invalid variant, conflicting value and typed_value: value=%s typed_value=%s", + value, + typed); + return typed; + } + + return value; + } + + @Override + public TripleIterator column() { + return column; + } + + @Override + public List> columns() { + return children; + } + + @Override + public void setPageSource(PageReadStore pageStore) { + if (valueReader != null) { + valueReader.setPageSource(pageStore); + } + + if (typedReader != null) { + typedReader.setPageSource(pageStore); + } + } + } + + /** + * A Variant reader that combines value and partially shredded object columns. + * + *

This reader handles partially shredded objects. For shredded values, use {@link + * ShreddedVariantReader} instead. + */ + private static class ShreddedObjectReader implements VariantValueReader { + private final int valueDefinitionLevel; + private final VariantValueReader valueReader; + private final int fieldsDefinitionLevel; + private final String[] fieldNames; + private final VariantValueReader[] fieldReaders; + private final TripleIterator valueColumn; + private final TripleIterator fieldColumn; + private final List> children; + + private ShreddedObjectReader( + int valueDefinitionLevel, + VariantValueReader valueReader, + int fieldsDefinitionLevel, + List fieldNames, + List fieldReaders) { + this.valueDefinitionLevel = valueDefinitionLevel; + this.valueReader = valueReader; + this.fieldsDefinitionLevel = fieldsDefinitionLevel; + this.fieldNames = fieldNames.toArray(String[]::new); + this.fieldReaders = fieldReaders.toArray(VariantValueReader[]::new); + this.fieldColumn = this.fieldReaders[0].column(); + this.valueColumn = valueReader != null ? valueReader.column() : fieldColumn; + this.children = children(Iterables.concat(Arrays.asList(valueReader), fieldReaders)); + } + + @Override + public VariantValue read(VariantMetadata metadata) { + boolean isObject = fieldColumn.currentDefinitionLevel() > fieldsDefinitionLevel; + VariantValue value = ParquetVariantReaders.read(metadata, valueReader, valueDefinitionLevel); + + if (isObject) { + ShreddedObject object; + if (value == MISSING) { + object = Variants.object(metadata); + } else { + Preconditions.checkArgument( + value.type() == PhysicalType.OBJECT, + "Invalid variant, non-object value with shredded fields: %s", + value); + object = Variants.object(metadata, (VariantObject) value); + } + + for (int i = 0; i < fieldReaders.length; i += 1) { + // each field is a ShreddedVariantReader or ShreddedObjectReader that handles DL + String name = fieldNames[i]; + VariantValue fieldValue = fieldReaders[i].read(metadata); + if (fieldValue == MISSING) { + object.remove(name); + } else { + object.put(name, fieldValue); + } + } + + return object; + } + + // for non-objects, advance the field iterators + for (VariantValueReader reader : fieldReaders) { + for (TripleIterator child : reader.columns()) { + child.nextNull(); + } + } + + return value; + } + + @Override + public TripleIterator column() { + return valueColumn; + } + + @Override + public List> columns() { + return children; + } + + @Override + public void setPageSource(PageReadStore pageStore) { + if (valueReader != null) { + valueReader.setPageSource(pageStore); + } + + for (VariantValueReader reader : fieldReaders) { + reader.setPageSource(pageStore); + } + } + } + + private static class VariantReader implements ParquetValueReader { + private final ParquetValueReader metadataReader; + private final VariantValueReader valueReader; + private final TripleIterator column; + private final List> children; + + private VariantReader( + ParquetValueReader metadataReader, VariantValueReader valueReader) { + this.metadataReader = metadataReader; + this.valueReader = valueReader; + // metadata is always non-null so its column can be used for the variant + this.column = metadataReader.column(); + this.children = children(metadataReader, valueReader); + } + + @Override + public Variant read(Variant ignored) { + VariantMetadata metadata = metadataReader.read(null); + VariantValue value = valueReader.read(metadata); + if (value == MISSING) { + return new Variant() { + @Override + public VariantMetadata metadata() { + return metadata; + } + + @Override + public VariantValue value() { + return Variants.ofNull(); + } + }; + } + + return new Variant() { + @Override + public VariantMetadata metadata() { + return metadata; + } + + @Override + public VariantValue value() { + return value; + } + }; + } + + @Override + public TripleIterator column() { + return column; + } + + @Override + public List> columns() { + return children; + } + + @Override + public void setPageSource(PageReadStore pageStore) { + metadataReader.setPageSource(pageStore); + valueReader.setPageSource(pageStore); + } + } + + private static VariantValue read( + VariantMetadata metadata, VariantValueReader reader, int definitionLevel) { + if (reader != null) { + if (reader.column().currentDefinitionLevel() > definitionLevel) { + return reader.read(metadata); + } + + for (TripleIterator child : reader.columns()) { + child.nextNull(); + } + } + + return MISSING; + } + + private static List> children(ParquetValueReader... readers) { + return children(Arrays.asList(readers)); + } + + private static List> children(Iterable> readers) { + return ImmutableList.copyOf( + Iterables.concat( + Iterables.transform( + Streams.of(readers).filter(Objects::nonNull).collect(Collectors.toList()), + ParquetValueReader::columns))); + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantVisitor.java new file mode 100644 index 000000000000..71d2eb26627b --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantVisitor.java @@ -0,0 +1,285 @@ +/* + * 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.parquet; + +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation.ListLogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; + +public abstract class ParquetVariantVisitor { + static final String METADATA = "metadata"; + static final String VALUE = "value"; + static final String TYPED_VALUE = "typed_value"; + + /** + * Handles the root variant column group. + * + *

The value and typed_value results are combined by calling {@link #value}. + * + *

+   *   group v (VARIANT) { <-- metadata result and combined value and typed_value result
+   *     required binary metadata;
+   *     optional binary value;
+   *     optional ... typed_value;
+   *   }
+   * 
+ */ + public R variant(GroupType variant, R metadataResult, R valueResult) { + return null; + } + + /** + * Handles a serialized variant metadata column. + * + *
+   *   group v (VARIANT) {
+   *     required binary metadata; <-- this column
+   *     optional binary value;
+   *     optional ... typed_value;
+   *   }
+   * 
+ */ + public R metadata(PrimitiveType metadata) { + return null; + } + + /** + * Handles a serialized variant value column. + * + *
+   *   group variant_value_pair {
+   *     optional binary value; <-- this column
+   *     optional ... typed_value;
+   *   }
+   * 
+ */ + public R serialized(PrimitiveType value) { + return null; + } + + /** + * Handles a shredded primitive typed_value column. + * + *
+   *   group variant_value_pair {
+   *     optional binary value;
+   *     optional int32 typed_value; <-- this column when it is any primitive
+   *   }
+   * 
+ */ + public R primitive(PrimitiveType primitive) { + return null; + } + + /** + * Handles a variant value result and typed_value result pair. + * + *

The value and typed_value pair may be nested in an object field, array element, or in the + * root group of a variant. + * + *

This method is also called when the typed_value field is missing. + * + *

+   *   group variant_value_pair { <-- value result and typed_value result
+   *     optional binary value;
+   *     optional ... typed_value;
+   *   }
+   * 
+ */ + public R value(GroupType value, R valueResult, R typedResult) { + return null; + } + + /** + * Handles a shredded object value result and a list of field value results. + * + *

Each field's value and typed_value results are combined by calling {@link #value}. + * + *

+   *   group variant_value_pair {  <-- value result and typed_value field results
+   *     optional binary value;
+   *     optional group typed_value {
+   *       required group a {
+   *         optional binary value;
+   *         optional binary typed_value (UTF8);
+   *       }
+   *       ...
+   *     }
+   *   }
+   * 
+ */ + public R object(GroupType object, R valueResult, List fieldResults) { + return null; + } + + /** + * Handles a shredded array value result and an element value result. + * + *

The element's value and typed_value results are combined by calling {@link #value}. + * + *

+   *   group variant_value_pair {  <-- value result and element result
+   *     optional binary value;
+   *     optional group typed_value (LIST) {
+   *       repeated group list {
+   *         required group element {
+   *           optional binary value;
+   *           optional binary typed_value (UTF8);
+   *         }
+   *       }
+   *     }
+   *   }
+   * 
+ */ + public R array(GroupType array, R valueResult, R elementResult) { + return null; + } + + /** Handler called before visiting any primitive or group type. */ + public void beforeField(Type type) {} + + /** Handler called after visiting any primitive or group type. */ + public void afterField(Type type) {} + + public static R visit(GroupType type, ParquetVariantVisitor visitor) { + Preconditions.checkArgument( + ParquetSchemaUtil.hasField(type, METADATA), "Invalid variant, missing metadata: %s", type); + + Type metadataType = type.getType(METADATA); + Preconditions.checkArgument( + isBinary(metadataType), "Invalid variant metadata, expecting BINARY: %s", metadataType); + + R metadataResult = + withBeforeAndAfter( + () -> visitor.metadata(metadataType.asPrimitiveType()), metadataType, visitor); + R valueResult = visitValue(type, visitor); + + return visitor.variant(type, metadataResult, valueResult); + } + + private static R visitValue(GroupType valueGroup, ParquetVariantVisitor visitor) { + R valueResult; + if (ParquetSchemaUtil.hasField(valueGroup, VALUE)) { + Type valueType = valueGroup.getType(VALUE); + Preconditions.checkArgument( + isBinary(valueType), "Invalid variant value, expecting BINARY: %s", valueType); + + valueResult = + withBeforeAndAfter( + () -> visitor.serialized(valueType.asPrimitiveType()), valueType, visitor); + } else { + Preconditions.checkArgument( + ParquetSchemaUtil.hasField(valueGroup, TYPED_VALUE), + "Invalid variant, missing both value and typed_value: %s", + valueGroup); + + valueResult = null; + } + + if (ParquetSchemaUtil.hasField(valueGroup, TYPED_VALUE)) { + Type typedValueType = valueGroup.getType(TYPED_VALUE); + + if (typedValueType.isPrimitive()) { + R typedResult = + withBeforeAndAfter( + () -> visitor.primitive(typedValueType.asPrimitiveType()), typedValueType, visitor); + + return visitor.value(valueGroup, valueResult, typedResult); + + } else if (typedValueType.getLogicalTypeAnnotation() instanceof ListLogicalTypeAnnotation) { + R elementResult = + withBeforeAndAfter( + () -> visitArray(typedValueType.asGroupType(), visitor), typedValueType, visitor); + + return visitor.array(valueGroup, valueResult, elementResult); + + } else { + List results = + withBeforeAndAfter( + () -> visitObjectFields(typedValueType.asGroupType(), visitor), + typedValueType, + visitor); + + return visitor.object(valueGroup, valueResult, results); + } + } + + // there was no typed_value field, but the value result must be handled + return visitor.value(valueGroup, valueResult, null); + } + + private static R visitArray(GroupType array, ParquetVariantVisitor visitor) { + Preconditions.checkArgument( + array.getFieldCount() == 1, + "Invalid variant array: does not contain single repeated field: %s", + array); + + Type repeated = array.getFields().get(0); + Preconditions.checkArgument( + repeated.isRepetition(Type.Repetition.REPEATED), + "Invalid variant array: inner group is not repeated"); + + // 3-level structure is required; element is always the only child of the repeated field + return withBeforeAndAfter( + () -> visitElement(repeated.asGroupType().getType(0), visitor), repeated, visitor); + } + + private static R visitElement(Type element, ParquetVariantVisitor visitor) { + return withBeforeAndAfter(() -> visitValue(element.asGroupType(), visitor), element, visitor); + } + + private static List visitObjectFields(GroupType fields, ParquetVariantVisitor visitor) { + List results = Lists.newArrayList(); + for (Type fieldType : fields.getFields()) { + Preconditions.checkArgument( + !fieldType.isPrimitive(), "Invalid shredded object field, not a group: %s", fieldType); + R fieldResult = + withBeforeAndAfter( + () -> visitValue(fieldType.asGroupType(), visitor), fieldType, visitor); + results.add(fieldResult); + } + + return results; + } + + @FunctionalInterface + private interface Action { + R invoke(); + } + + private static R withBeforeAndAfter( + Action task, Type type, ParquetVariantVisitor visitor) { + visitor.beforeField(type); + try { + return task.invoke(); + } finally { + visitor.afterField(type); + } + } + + private static boolean isBinary(Type type) { + return type.isPrimitive() + && type.asPrimitiveType().getPrimitiveTypeName() == PrimitiveTypeName.BINARY; + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index d48485305e8a..50f28c90e3da 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -26,6 +26,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types.ListType; import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.LogicalTypeAnnotation; @@ -57,7 +58,7 @@ public Type message(StructType expected, MessageType message, List fields) hasChange = true; builder.addField(field); } else { - if (isStruct(originalField)) { + if (isStruct(originalField, expected.field(fieldId))) { hasChange = true; builder.addField(originalField.asGroupType().withNewFields(Collections.emptyList())); } else { @@ -152,6 +153,11 @@ public Type map(MapType expected, GroupType map, Type key, Type value) { return null; } + @Override + public Type variant(org.apache.iceberg.types.Types.VariantType expected, Type variant) { + return variant; + } + @Override public Type primitive( org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { @@ -162,8 +168,8 @@ private Integer getId(Type type) { return type.getId() == null ? null : type.getId().intValue(); } - private boolean isStruct(Type field) { - if (field.isPrimitive()) { + private boolean isStruct(Type field, NestedField expected) { + if (field.isPrimitive() || expected.type().isVariantType()) { return false; } else { GroupType groupType = field.asGroupType(); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java b/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java index 54f11500489b..cb40693e24a4 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java @@ -86,6 +86,9 @@ public Type field(NestedField field) { if (field.type().isPrimitiveType()) { return primitive(field.type().asPrimitiveType(), repetition, id, name); + } else if (field.type().isVariantType()) { + return variant(repetition, id, name); + } else { NestedType nested = field.type().asNestedType(); if (nested.isStructType()) { @@ -117,6 +120,17 @@ public GroupType map(MapType map, Type.Repetition repetition, int id, String nam .named(AvroSchemaUtil.makeCompatibleName(name)); } + public Type variant(Type.Repetition repetition, int id, String originalName) { + String name = AvroSchemaUtil.makeCompatibleName(originalName); + return Types.buildGroup(repetition) + .id(id) + .required(BINARY) + .named("metadata") + .required(BINARY) + .named("value") + .named(name); + } + public Type primitive( PrimitiveType primitive, Type.Repetition repetition, int id, String originalName) { String name = AvroSchemaUtil.makeCompatibleName(originalName); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java index e0c07d31755e..23e72b91c9e3 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java @@ -24,8 +24,10 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.ListLogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.MapLogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; @@ -54,94 +56,14 @@ public static T visit( } else { // if not a primitive, the typeId must be a group GroupType group = type.asGroupType(); - OriginalType annotation = group.getOriginalType(); - if (annotation != null) { - switch (annotation) { - case LIST: - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid list: does not contain single repeated field: %s", - group); - - Type repeatedElement = group.getFields().get(0); - Preconditions.checkArgument( - repeatedElement.isRepetition(Type.Repetition.REPEATED), - "Invalid list: inner group is not repeated"); - - Type listElement = ParquetSchemaUtil.determineListElementType(group); - Types.ListType list = null; - Types.NestedField element = null; - if (iType != null) { - list = iType.asListType(); - element = list.fields().get(0); - } - - if (listElement.isRepetition(Type.Repetition.REPEATED)) { - return visitTwoLevelList(list, element, group, listElement, visitor); - } else { - return visitThreeLevelList(list, element, group, listElement, visitor); - } - - case MAP: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid map: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid map: does not contain single repeated field: %s", - group); - - GroupType repeatedKeyValue = group.getType(0).asGroupType(); - Preconditions.checkArgument( - repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), - "Invalid map: inner group is not repeated"); - Preconditions.checkArgument( - repeatedKeyValue.getFieldCount() <= 2, - "Invalid map: repeated group does not have 2 fields"); - - Types.MapType map = null; - Types.NestedField keyField = null; - Types.NestedField valueField = null; - if (iType != null) { - map = iType.asMapType(); - keyField = map.fields().get(0); - valueField = map.fields().get(1); - } - - visitor.fieldNames.push(repeatedKeyValue.getName()); - try { - T keyResult = null; - T valueResult = null; - switch (repeatedKeyValue.getFieldCount()) { - case 2: - // if there are 2 fields, both key and value are projected - keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); - valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); - break; - case 1: - // if there is just one, use the name to determine what it is - Type keyOrValue = repeatedKeyValue.getType(0); - if (keyOrValue.getName().equalsIgnoreCase("key")) { - keyResult = visitField(keyField, keyOrValue, visitor); - // value result remains null - } else { - valueResult = visitField(valueField, keyOrValue, visitor); - // key result remains null - } - break; - default: - // both results will remain null - } - - return visitor.map(map, group, keyResult, valueResult); - - } finally { - visitor.fieldNames.pop(); - } - - default: - } + LogicalTypeAnnotation annotation = group.getLogicalTypeAnnotation(); + if (annotation instanceof ListLogicalTypeAnnotation) { + return visitList(iType, group, visitor); + } else if (annotation instanceof MapLogicalTypeAnnotation) { + return visitMap(iType, group, visitor); + } else if (iType != null && iType.isVariantType()) { + // when Parquet has a VARIANT logical type, use it here + return visitVariant(iType.asVariantType(), group, visitor); } Types.StructType struct = iType != null ? iType.asStructType() : null; @@ -149,6 +71,93 @@ public static T visit( } } + private static T visitList( + org.apache.iceberg.types.Type iType, GroupType group, TypeWithSchemaVisitor visitor) { + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid list: does not contain single repeated field: %s", + group); + + Type repeatedElement = group.getFields().get(0); + Preconditions.checkArgument( + repeatedElement.isRepetition(Type.Repetition.REPEATED), + "Invalid list: inner group is not repeated"); + + Type listElement = ParquetSchemaUtil.determineListElementType(group); + Types.ListType list = null; + Types.NestedField element = null; + if (iType != null) { + list = iType.asListType(); + element = list.fields().get(0); + } + + if (listElement.isRepetition(Type.Repetition.REPEATED)) { + return visitTwoLevelList(list, element, group, listElement, visitor); + } else { + return visitThreeLevelList(list, element, group, listElement, visitor); + } + } + + private static T visitMap( + org.apache.iceberg.types.Type iType, GroupType group, TypeWithSchemaVisitor visitor) { + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid map: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid map: does not contain single repeated field: %s", + group); + + GroupType repeatedKeyValue = group.getType(0).asGroupType(); + Preconditions.checkArgument( + repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), + "Invalid map: inner group is not repeated"); + Preconditions.checkArgument( + repeatedKeyValue.getFieldCount() <= 2, + "Invalid map: repeated group does not have 2 fields"); + + Types.MapType map = null; + Types.NestedField keyField = null; + Types.NestedField valueField = null; + if (iType != null) { + map = iType.asMapType(); + keyField = map.fields().get(0); + valueField = map.fields().get(1); + } + + visitor.fieldNames.push(repeatedKeyValue.getName()); + try { + T keyResult = null; + T valueResult = null; + switch (repeatedKeyValue.getFieldCount()) { + case 2: + // if there are 2 fields, both key and value are projected + keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); + valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); + break; + case 1: + // if there is just one, use the name to determine what it is + Type keyOrValue = repeatedKeyValue.getType(0); + if (keyOrValue.getName().equalsIgnoreCase("key")) { + keyResult = visitField(keyField, keyOrValue, visitor); + // value result remains null + } else { + valueResult = visitField(valueField, keyOrValue, visitor); + // key result remains null + } + break; + default: + // both results will remain null + } + + return visitor.map(map, group, keyResult, valueResult); + + } finally { + visitor.fieldNames.pop(); + } + } + private static T visitTwoLevelList( Types.ListType iListType, Types.NestedField iListElement, @@ -201,6 +210,17 @@ private static List visitFields( return results; } + private static T visitVariant( + Types.VariantType variant, GroupType group, TypeWithSchemaVisitor visitor) { + ParquetVariantVisitor variantVisitor = visitor.variantVisitor(); + if (variantVisitor != null) { + T variantResult = ParquetVariantVisitor.visit(group, variantVisitor); + return visitor.variant(variant, variantResult); + } else { + return visitor.variant(variant, null); + } + } + public T message(Types.StructType iStruct, MessageType message, List fields) { return null; } @@ -217,11 +237,19 @@ public T map(Types.MapType iMap, GroupType map, T key, T value) { return null; } + public T variant(Types.VariantType iVariant, T result) { + throw new UnsupportedOperationException("Not implemented for variant"); + } + public T primitive( org.apache.iceberg.types.Type.PrimitiveType iPrimitive, PrimitiveType primitive) { return null; } + public ParquetVariantVisitor variantVisitor() { + return null; + } + protected String[] currentPath() { return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VariantReaderBuilder.java b/parquet/src/main/java/org/apache/iceberg/parquet/VariantReaderBuilder.java new file mode 100644 index 000000000000..7655be8709fb --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VariantReaderBuilder.java @@ -0,0 +1,260 @@ +/* + * 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.parquet; + +import java.util.Deque; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.parquet.ParquetVariantReaders.VariantValueReader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.variants.PhysicalType; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation.DateLogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.IntLogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.LogicalTypeAnnotationVisitor; +import org.apache.parquet.schema.LogicalTypeAnnotation.StringLogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.TimestampLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class VariantReaderBuilder extends ParquetVariantVisitor> { + private final MessageType schema; + private final Iterable basePath; + private final Deque fieldNames = Lists.newLinkedList(); + + public VariantReaderBuilder(MessageType schema, Iterable basePath) { + this.schema = schema; + this.basePath = basePath; + } + + @Override + public void beforeField(Type type) { + fieldNames.addLast(type.getName()); + } + + @Override + public void afterField(Type type) { + fieldNames.removeLast(); + } + + private String[] currentPath() { + return Streams.concat(Streams.stream(basePath), fieldNames.stream()).toArray(String[]::new); + } + + private String[] path(String name) { + return Streams.concat(Streams.stream(basePath), fieldNames.stream(), Stream.of(name)) + .toArray(String[]::new); + } + + @Override + public ParquetValueReader variant( + GroupType variant, ParquetValueReader metadataReader, ParquetValueReader valueReader) { + return ParquetVariantReaders.variant(metadataReader, valueReader); + } + + @Override + public ParquetValueReader metadata(PrimitiveType metadata) { + ColumnDescriptor desc = schema.getColumnDescription(currentPath()); + return ParquetVariantReaders.metadata(desc); + } + + @Override + public VariantValueReader serialized(PrimitiveType value) { + ColumnDescriptor desc = schema.getColumnDescription(currentPath()); + return ParquetVariantReaders.serialized(desc); + } + + @Override + public VariantValueReader primitive(PrimitiveType primitive) { + ColumnDescriptor desc = schema.getColumnDescription(currentPath()); + + if (primitive.getLogicalTypeAnnotation() != null) { + Optional reader = + primitive.getLogicalTypeAnnotation().accept(new LogicalTypeToVariantReader(desc)); + if (reader.isPresent()) { + return reader.get(); + } + + } else { + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + return ParquetVariantReaders.asVariant( + PhysicalType.BINARY, ParquetValueReaders.byteBuffers(desc)); + case BOOLEAN: + // the actual boolean type will be fixed in PrimitiveWrapper + return ParquetVariantReaders.asVariant( + PhysicalType.BOOLEAN_TRUE, ParquetValueReaders.unboxed(desc)); + case INT32: + return ParquetVariantReaders.asVariant( + PhysicalType.INT32, ParquetValueReaders.unboxed(desc)); + case INT64: + return ParquetVariantReaders.asVariant( + PhysicalType.INT64, ParquetValueReaders.unboxed(desc)); + case FLOAT: + return ParquetVariantReaders.asVariant( + PhysicalType.FLOAT, ParquetValueReaders.unboxed(desc)); + case DOUBLE: + return ParquetVariantReaders.asVariant( + PhysicalType.DOUBLE, ParquetValueReaders.unboxed(desc)); + } + } + + // note that both FIXED_LEN_BYTE_ARRAY and INT96 are not valid Variant primitives + throw new UnsupportedOperationException("Unsupported shredded value type: " + primitive); + } + + @Override + public VariantValueReader value( + GroupType group, ParquetValueReader valueReader, ParquetValueReader typedReader) { + int valueDL = + valueReader != null ? schema.getMaxDefinitionLevel(path(VALUE)) - 1 : Integer.MAX_VALUE; + int typedDL = + typedReader != null + ? schema.getMaxDefinitionLevel(path(TYPED_VALUE)) - 1 + : Integer.MAX_VALUE; + return ParquetVariantReaders.shredded(valueDL, valueReader, typedDL, typedReader); + } + + @Override + public VariantValueReader object( + GroupType group, + ParquetValueReader valueReader, + List> fieldResults) { + int valueDL = + valueReader != null ? schema.getMaxDefinitionLevel(path(VALUE)) - 1 : Integer.MAX_VALUE; + int fieldsDL = schema.getMaxDefinitionLevel(path(TYPED_VALUE)) - 1; + + List shreddedFieldNames = + group.getType(TYPED_VALUE).asGroupType().getFields().stream() + .map(Type::getName) + .collect(Collectors.toList()); + List fieldReaders = + fieldResults.stream().map(VariantValueReader.class::cast).collect(Collectors.toList()); + + return ParquetVariantReaders.objects( + valueDL, valueReader, fieldsDL, shreddedFieldNames, fieldReaders); + } + + @Override + public VariantValueReader array( + GroupType array, ParquetValueReader valueResult, ParquetValueReader elementResult) { + throw new UnsupportedOperationException("Array is not yet supported"); + } + + private static class LogicalTypeToVariantReader + implements LogicalTypeAnnotationVisitor { + private final ColumnDescriptor desc; + + private LogicalTypeToVariantReader(ColumnDescriptor desc) { + this.desc = desc; + } + + @Override + public Optional visit(StringLogicalTypeAnnotation ignored) { + VariantValueReader reader = + ParquetVariantReaders.asVariant(PhysicalType.STRING, ParquetValueReaders.strings(desc)); + + return Optional.of(reader); + } + + @Override + public Optional visit(DecimalLogicalTypeAnnotation logical) { + PhysicalType variantType = variantDecimalType(desc.getPrimitiveType()); + VariantValueReader reader = + ParquetVariantReaders.asVariant(variantType, ParquetValueReaders.bigDecimals(desc)); + + return Optional.of(reader); + } + + @Override + public Optional visit(DateLogicalTypeAnnotation ignored) { + VariantValueReader reader = + ParquetVariantReaders.asVariant(PhysicalType.DATE, ParquetValueReaders.unboxed(desc)); + + return Optional.of(reader); + } + + @Override + public Optional visit(TimestampLogicalTypeAnnotation logical) { + PhysicalType variantType = + logical.isAdjustedToUTC() ? PhysicalType.TIMESTAMPTZ : PhysicalType.TIMESTAMPNTZ; + + VariantValueReader reader = + ParquetVariantReaders.asVariant(variantType, ParquetValueReaders.timestamps(desc)); + + return Optional.of(reader); + } + + @Override + public Optional visit(IntLogicalTypeAnnotation logical) { + if (!logical.isSigned()) { + // unsigned ints are not allowed for shredded fields + throw new UnsupportedOperationException("Unsupported shredded value type: " + logical); + } + + VariantValueReader reader; + switch (logical.getBitWidth()) { + case 64: + reader = + ParquetVariantReaders.asVariant( + PhysicalType.INT64, ParquetValueReaders.unboxed(desc)); + break; + case 32: + reader = + ParquetVariantReaders.asVariant( + PhysicalType.INT32, ParquetValueReaders.unboxed(desc)); + break; + case 16: + reader = + ParquetVariantReaders.asVariant( + PhysicalType.INT16, ParquetValueReaders.intsAsShort(desc)); + break; + case 8: + reader = + ParquetVariantReaders.asVariant( + PhysicalType.INT8, ParquetValueReaders.intsAsByte(desc)); + break; + default: + throw new IllegalArgumentException("Invalid bit width for int: " + logical.getBitWidth()); + } + + return Optional.of(reader); + } + + private static PhysicalType variantDecimalType(PrimitiveType primitive) { + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return PhysicalType.DECIMAL16; + case INT64: + return PhysicalType.DECIMAL8; + case INT32: + return PhysicalType.DECIMAL4; + } + + throw new IllegalArgumentException("Invalid primitive type for decimal: " + primitive); + } + } +} diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java new file mode 100644 index 000000000000..1936fe3935fe --- /dev/null +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java @@ -0,0 +1,1107 @@ +/* + * 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.parquet; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.InternalReader; +import org.apache.iceberg.inmemory.InMemoryOutputFile; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.VariantType; +import org.apache.iceberg.variants.PhysicalType; +import org.apache.iceberg.variants.ShreddedObject; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantObject; +import org.apache.iceberg.variants.VariantPrimitive; +import org.apache.iceberg.variants.VariantTestUtil; +import org.apache.iceberg.variants.VariantValue; +import org.apache.iceberg.variants.Variants; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.avro.AvroWriteSupport; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.assertj.core.api.Assumptions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.FieldSource; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestVariantReaders { + private static final Schema SCHEMA = + new Schema( + NestedField.required(1, "id", IntegerType.get()), + NestedField.required(2, "var", VariantType.get())); + + private static final LogicalTypeAnnotation STRING = LogicalTypeAnnotation.stringType(); + + private static final ByteBuffer TEST_METADATA_BUFFER = + VariantTestUtil.createMetadata(ImmutableList.of("a", "b", "c", "d", "e"), true); + private static final ByteBuffer TEST_OBJECT_BUFFER = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, + ImmutableMap.of( + "a", Variants.ofNull(), + "d", Variants.of("iceberg"))); + + private static final VariantMetadata EMPTY_METADATA = + Variants.metadata(VariantTestUtil.emptyMetadata()); + private static final VariantMetadata TEST_METADATA = Variants.metadata(TEST_METADATA_BUFFER); + private static final VariantObject TEST_OBJECT = + (VariantObject) Variants.value(TEST_METADATA, TEST_OBJECT_BUFFER); + + private static final VariantPrimitive[] PRIMITIVES = + new VariantPrimitive[] { + Variants.ofNull(), + Variants.of(true), + Variants.of(false), + Variants.of((byte) 34), + Variants.of((byte) -34), + Variants.of((short) 1234), + Variants.of((short) -1234), + Variants.of(12345), + Variants.of(-12345), + Variants.of(9876543210L), + Variants.of(-9876543210L), + Variants.of(10.11F), + Variants.of(-10.11F), + Variants.of(14.3D), + Variants.of(-14.3D), + Variants.ofIsoDate("2024-11-07"), + Variants.ofIsoDate("1957-11-07"), + Variants.ofIsoTimestamptz("2024-11-07T12:33:54.123456+00:00"), + Variants.ofIsoTimestamptz("1957-11-07T12:33:54.123456+00:00"), + Variants.ofIsoTimestampntz("2024-11-07T12:33:54.123456"), + Variants.ofIsoTimestampntz("1957-11-07T12:33:54.123456"), + Variants.of(new BigDecimal("123456.7890")), // decimal4 + Variants.of(new BigDecimal("-123456.7890")), // decimal4 + Variants.of(new BigDecimal("1234567890.987654321")), // decimal8 + Variants.of(new BigDecimal("-1234567890.987654321")), // decimal8 + 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"), + }; + + private static Stream metadataAndValues() { + Stream primitives = + Stream.of(PRIMITIVES).map(variant -> Arguments.of(EMPTY_METADATA, variant)); + Stream object = Stream.of(Arguments.of(TEST_METADATA, TEST_OBJECT)); + return Streams.concat(primitives, object); + } + + @ParameterizedTest + @MethodSource("metadataAndValues") + public void testUnshreddedVariants(VariantMetadata metadata, VariantValue expected) + throws IOException { + GroupType variantType = variant("var", 2); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record(variantType, Map.of("metadata", serialize(metadata), "value", serialize(expected))); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(metadata, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @ParameterizedTest + @MethodSource("metadataAndValues") + public void testUnshreddedVariantsWithShreddedSchema( + VariantMetadata metadata, VariantValue expected) throws IOException { + // the variant's Parquet schema has a shredded field that is unused by all data values + GroupType variantType = variant("var", 2, shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record(variantType, Map.of("metadata", serialize(metadata), "value", serialize(expected))); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(metadata, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @ParameterizedTest + @FieldSource("PRIMITIVES") + public void testShreddedVariantPrimitives(VariantPrimitive primitive) throws IOException { + Assumptions.assumeThat(primitive.type() != PhysicalType.NULL) + .as("Null is not a shredded type") + .isTrue(); + + GroupType variantType = variant("var", 2, shreddedType(primitive)); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + VariantTestUtil.emptyMetadata(), + "typed_value", + toAvroValue(primitive))); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(primitive, actualVariant.value()); + } + + @Test + public void testNullValueAndNullTypedValue() throws IOException { + GroupType variantType = variant("var", 2, shreddedPrimitive(PrimitiveTypeName.INT32)); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record(variantType, Map.of("metadata", VariantTestUtil.emptyMetadata())); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(Variants.ofNull(), actualVariant.value()); + } + + @Test + public void testMissingValueColumn() throws IOException { + GroupType variantType = + Types.buildGroup(Type.Repetition.REQUIRED) + .id(2) + .required(PrimitiveTypeName.BINARY) + .named("metadata") + .addField(shreddedPrimitive(PrimitiveTypeName.INT32)) + .named("var"); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record(variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", 34)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(Variants.of(34), actualVariant.value()); + } + + @Test + public void testValueAndTypedValueConflict() throws IOException { + GroupType variantType = variant("var", 2, shreddedPrimitive(PrimitiveTypeName.INT32)); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + VariantTestUtil.emptyMetadata(), + "value", + serialize(Variants.of("str")), + "typed_value", + 34)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + assertThatThrownBy(() -> writeAndRead(parquetSchema, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid variant, conflicting value and typed_value"); + } + + @Test + public void testUnsignedInteger() { + GroupType variantType = + variant( + "var", + 2, + shreddedPrimitive(PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(32, false))); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record(variantType, Map.of("metadata", VariantTestUtil.emptyMetadata())); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + assertThatThrownBy(() -> writeAndRead(parquetSchema, record)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Unsupported shredded value type: INTEGER(32,false)"); + } + + @Test + public void testFixedLengthByteArray() { + GroupType variantType = + variant( + "var", + 2, + Types.optional(PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY).length(4).named("typed_value")); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record(variantType, Map.of("metadata", VariantTestUtil.emptyMetadata())); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + assertThatThrownBy(() -> writeAndRead(parquetSchema, record)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + "Unsupported shredded value type: optional fixed_len_byte_array(4) typed_value"); + } + + @Test + public void testShreddedObject() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.ofNull()))); + GenericRecord recordB = record(fieldB, Map.of("typed_value", "")); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.ofNull()); + expected.put("b", Variants.of("")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testShreddedObjectMissingValueColumn() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = + Types.buildGroup(Type.Repetition.REQUIRED) + .id(2) + .required(PrimitiveTypeName.BINARY) + .named("metadata") + .addField(objectFields) + .named("var"); + + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.of((short) 1234)))); + GenericRecord recordB = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.of((short) 1234)); + expected.put("b", Variants.of("iceberg")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testShreddedObjectMissingField() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.of(false)))); + // value and typed_value are null, but a struct for b is required + GenericRecord recordB = record(fieldB, Map.of()); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.of(false)); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testEmptyShreddedObject() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of()); // missing + GenericRecord recordB = record(fieldB, Map.of()); // missing + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testShreddedObjectMissingFieldValueColumn() throws IOException { + // field groups do not have value + GroupType fieldA = + Types.buildGroup(Type.Repetition.REQUIRED) + .addField(shreddedPrimitive(PrimitiveTypeName.INT32)) + .named("a"); + GroupType fieldB = + Types.buildGroup(Type.Repetition.REQUIRED) + .addField(shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)) + .named("b"); + GroupType objectFields = + Types.buildGroup(Type.Repetition.OPTIONAL).addFields(fieldA, fieldB).named("typed_value"); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of()); // typed_value=null + GenericRecord recordB = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("b", Variants.of("iceberg")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testShreddedObjectMissingTypedValue() throws IOException { + // field groups do not have typed_value + GroupType fieldA = + Types.buildGroup(Type.Repetition.REQUIRED) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .named("a"); + GroupType fieldB = + Types.buildGroup(Type.Repetition.REQUIRED) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .named("b"); + GroupType objectFields = + Types.buildGroup(Type.Repetition.OPTIONAL).addFields(fieldA, fieldB).named("typed_value"); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of()); // value=null + GenericRecord recordB = record(fieldB, Map.of("value", serialize(Variants.of("iceberg")))); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("b", Variants.of("iceberg")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testShreddedObjectWithinShreddedObject() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType innerFields = objectFields(fieldA, fieldB); + GroupType fieldC = field("c", innerFields); + GroupType fieldD = field("d", shreddedPrimitive(PrimitiveTypeName.DOUBLE)); + GroupType outerFields = objectFields(fieldC, fieldD); + GroupType variantType = variant("var", 2, outerFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of("typed_value", 34)); + GenericRecord recordB = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord inner = record(innerFields, Map.of("a", recordA, "b", recordB)); + GenericRecord recordC = record(fieldC, Map.of("typed_value", inner)); + GenericRecord recordD = record(fieldD, Map.of("typed_value", -0.0D)); + GenericRecord outer = record(outerFields, Map.of("c", recordC, "d", recordD)); + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", outer)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expectedInner = Variants.object(TEST_METADATA); + expectedInner.put("a", Variants.of(34)); + expectedInner.put("b", Variants.of("iceberg")); + ShreddedObject expectedOuter = Variants.object(TEST_METADATA); + expectedOuter.put("c", expectedInner); + expectedOuter.put("d", Variants.of(-0.0D)); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedOuter, actualVariant.value()); + } + + @Test + public void testShreddedObjectWithOptionalFieldStructs() throws IOException { + // fields use an incorrect OPTIONAL struct of value and typed_value to test definition levels + GroupType fieldA = + Types.buildGroup(Type.Repetition.OPTIONAL) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedPrimitive(PrimitiveTypeName.INT32)) + .named("a"); + GroupType fieldB = + Types.buildGroup(Type.Repetition.OPTIONAL) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)) + .named("b"); + GroupType fieldC = + Types.buildGroup(Type.Repetition.OPTIONAL) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedPrimitive(PrimitiveTypeName.DOUBLE)) + .named("c"); + GroupType fieldD = + Types.buildGroup(Type.Repetition.OPTIONAL) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedPrimitive(PrimitiveTypeName.BOOLEAN)) + .named("d"); + GroupType objectFields = + Types.buildGroup(Type.Repetition.OPTIONAL) + .addFields(fieldA, fieldB, fieldC, fieldD) + .named("typed_value"); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.of(34)))); + GenericRecord recordB = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord recordC = record(fieldC, Map.of()); // c.value and c.typed_value are missing + GenericRecord fields = + record(objectFields, Map.of("a", recordA, "b", recordB, "c", recordC)); // d is missing + GenericRecord variant = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + // the expected value is the shredded field value + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.of(34)); + expected.put("b", Variants.of("iceberg")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testPartiallyShreddedObject() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + ShreddedObject baseObject = Variants.object(TEST_METADATA); + baseObject.put("d", Variants.ofIsoDate("2024-01-30")); + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.ofNull()))); + GenericRecord recordB = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + TEST_METADATA_BUFFER, + "value", + serialize(baseObject), + "typed_value", + fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.ofNull()); + expected.put("b", Variants.of("iceberg")); + expected.put("d", Variants.ofIsoDate("2024-01-30")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testPartiallyShreddedObjectFieldConflict() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + ByteBuffer baseObjectBuffer = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("b", Variants.ofIsoDate("2024-01-30"))); // conflict + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.ofNull()))); + GenericRecord recordB = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + TEST_METADATA_BUFFER, + "value", + baseObjectBuffer, + "typed_value", + fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + // the expected value is the shredded field value + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.ofNull()); + expected.put("b", Variants.of("iceberg")); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testPartiallyShreddedObjectMissingFieldConflict() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + ByteBuffer baseObjectBuffer = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("b", Variants.ofIsoDate("2024-01-30"))); // conflict + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.ofNull()))); + // value and typed_value are null, but a struct for b is required + GenericRecord recordB = record(fieldB, Map.of()); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + TEST_METADATA_BUFFER, + "value", + baseObjectBuffer, + "typed_value", + fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + // the expected value is the shredded field value + ShreddedObject expected = Variants.object(TEST_METADATA); + expected.put("a", Variants.ofNull()); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expected, actualVariant.value()); + } + + @Test + public void testNonObjectWithNullShreddedFields() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord variant = + record( + variantType, + Map.of("metadata", TEST_METADATA_BUFFER, "value", serialize(Variants.of(34)))); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + Record actual = writeAndRead(parquetSchema, record); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(Variants.of(34), actualVariant.value()); + } + + @Test + public void testNonObjectWithNonNullShreddedFields() { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of("value", serialize(Variants.ofNull()))); + GenericRecord recordB = record(fieldB, Map.of("value", serialize(Variants.of(9876543210L)))); + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + TEST_METADATA_BUFFER, + "value", + serialize(Variants.of(34)), + "typed_value", + fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + assertThatThrownBy(() -> writeAndRead(parquetSchema, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid variant, non-object value with shredded fields"); + } + + @Test + public void testEmptyPartiallyShreddedObjectConflict() { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType objectFields = objectFields(fieldA, fieldB); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord recordA = record(fieldA, Map.of()); // missing + GenericRecord recordB = record(fieldB, Map.of()); // missing + GenericRecord fields = record(objectFields, Map.of("a", recordA, "b", recordB)); + GenericRecord variant = + record( + variantType, + Map.of( + "metadata", + TEST_METADATA_BUFFER, + "value", + serialize(Variants.ofNull()), // conflicting non-object + "typed_value", + fields)); + GenericRecord record = record(parquetSchema, Map.of("id", 1, "var", variant)); + + assertThatThrownBy(() -> writeAndRead(parquetSchema, record)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid variant, non-object value with shredded fields"); + } + + @Test + public void testMixedRecords() throws IOException { + // tests multiple rows to check that Parquet columns are correctly advanced + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType innerFields = objectFields(fieldA, fieldB); + GroupType fieldC = field("c", innerFields); + GroupType fieldD = field("d", shreddedPrimitive(PrimitiveTypeName.DOUBLE)); + GroupType outerFields = objectFields(fieldC, fieldD); + GroupType variantType = variant("var", 2, outerFields); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord zero = record(parquetSchema, Map.of("id", 0)); + + GenericRecord a1 = record(fieldA, Map.of()); // missing + GenericRecord b1 = record(fieldB, Map.of("typed_value", "iceberg")); + GenericRecord inner1 = record(innerFields, Map.of("a", a1, "b", b1)); + GenericRecord c1 = record(fieldC, Map.of("typed_value", inner1)); + GenericRecord d1 = record(fieldD, Map.of()); // missing + GenericRecord outer1 = record(outerFields, Map.of("c", c1, "d", d1)); + GenericRecord variant1 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", outer1)); + GenericRecord one = record(parquetSchema, Map.of("id", 1, "var", variant1)); + + ShreddedObject expectedC1 = Variants.object(TEST_METADATA); + expectedC1.put("b", Variants.of("iceberg")); + ShreddedObject expectedOne = Variants.object(TEST_METADATA); + expectedOne.put("c", expectedC1); + + GenericRecord c2 = record(fieldC, Map.of("value", serialize(Variants.of((byte) 8)))); + GenericRecord d2 = record(fieldD, Map.of("typed_value", -0.0D)); + GenericRecord outer2 = record(outerFields, Map.of("c", c2, "d", d2)); + GenericRecord variant2 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", outer2)); + GenericRecord two = record(parquetSchema, Map.of("id", 2, "var", variant2)); + + ShreddedObject expectedTwo = Variants.object(TEST_METADATA); + expectedTwo.put("c", Variants.of((byte) 8)); + expectedTwo.put("d", Variants.of(-0.0D)); + + GenericRecord a3 = record(fieldA, Map.of("typed_value", 34)); + GenericRecord b3 = record(fieldB, Map.of("value", serialize(Variants.of("")))); + GenericRecord inner3 = record(innerFields, Map.of("a", a3, "b", b3)); + GenericRecord c3 = record(fieldC, Map.of("typed_value", inner3)); + GenericRecord d3 = record(fieldD, Map.of("typed_value", 0.0D)); + GenericRecord outer3 = record(outerFields, Map.of("c", c3, "d", d3)); + GenericRecord variant3 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", outer3)); + GenericRecord three = record(parquetSchema, Map.of("id", 3, "var", variant3)); + + ShreddedObject expectedC3 = Variants.object(TEST_METADATA); + expectedC3.put("a", Variants.of(34)); + expectedC3.put("b", Variants.of("")); + ShreddedObject expectedThree = Variants.object(TEST_METADATA); + expectedThree.put("c", expectedC3); + expectedThree.put("d", Variants.of(0.0D)); + + List records = writeAndRead(parquetSchema, List.of(zero, one, two, three)); + + Record actualZero = records.get(0); + assertThat(actualZero.getField("id")).isEqualTo(0); + assertThat(actualZero.getField("var")).isNull(); + + Record actualOne = records.get(1); + assertThat(actualOne.getField("id")).isEqualTo(1); + assertThat(actualOne.getField("var")).isInstanceOf(Variant.class); + + Variant actualOneVariant = (Variant) actualOne.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualOneVariant.metadata()); + VariantTestUtil.assertEqual(expectedOne, actualOneVariant.value()); + + Record actualTwo = records.get(2); + assertThat(actualTwo.getField("id")).isEqualTo(2); + assertThat(actualTwo.getField("var")).isInstanceOf(Variant.class); + + Variant actualTwoVariant = (Variant) actualTwo.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualTwoVariant.metadata()); + VariantTestUtil.assertEqual(expectedTwo, actualTwoVariant.value()); + + Record actualThree = records.get(3); + assertThat(actualThree.getField("id")).isEqualTo(3); + assertThat(actualThree.getField("var")).isInstanceOf(Variant.class); + + Variant actualThreeVariant = (Variant) actualThree.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualThreeVariant.metadata()); + VariantTestUtil.assertEqual(expectedThree, actualThreeVariant.value()); + } + + private static ByteBuffer serialize(VariantValue value) { + ByteBuffer buffer = ByteBuffer.allocate(value.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + value.writeTo(buffer, 0); + return buffer; + } + + private static ByteBuffer serialize(VariantMetadata metadata) { + ByteBuffer buffer = ByteBuffer.allocate(metadata.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + metadata.writeTo(buffer, 0); + return buffer; + } + + /** Creates an Avro record from a map of field name to value. */ + private static GenericRecord record(GroupType type, Map fields) { + GenericRecord record = new GenericData.Record(avroSchema(type)); + for (Map.Entry entry : fields.entrySet()) { + record.put(entry.getKey(), entry.getValue()); + } + return record; + } + + /** + * This is a custom Parquet writer builder that injects a specific Parquet schema and then uses + * the Avro object model. This ensures that the Parquet file's schema is exactly what was passed. + */ + private static class TestWriterBuilder + extends ParquetWriter.Builder { + private MessageType parquetSchema = null; + + protected TestWriterBuilder(OutputFile outputFile) { + super(ParquetIO.file(outputFile)); + } + + TestWriterBuilder withFileType(MessageType schema) { + this.parquetSchema = schema; + return self(); + } + + @Override + protected TestWriterBuilder self() { + return this; + } + + @Override + protected WriteSupport getWriteSupport(Configuration conf) { + return new AvroWriteSupport<>(parquetSchema, avroSchema(parquetSchema), GenericData.get()); + } + } + + static Record writeAndRead(MessageType parquetSchema, GenericRecord record) throws IOException { + return Iterables.getOnlyElement(writeAndRead(parquetSchema, List.of(record))); + } + + static List writeAndRead(MessageType parquetSchema, List records) + throws IOException { + OutputFile outputFile = new InMemoryOutputFile(); + + try (ParquetWriter writer = + new TestWriterBuilder(outputFile).withFileType(parquetSchema).build()) { + for (GenericRecord record : records) { + writer.write(record); + } + } + + try (CloseableIterable reader = + Parquet.read(outputFile.toInputFile()) + .project(SCHEMA) + .createReaderFunc(fileSchema -> InternalReader.create(SCHEMA, fileSchema)) + .build()) { + return Lists.newArrayList(reader); + } + } + + private static MessageType parquetSchema(Type variantType) { + return Types.buildMessage() + .required(PrimitiveTypeName.INT32) + .id(1) + .named("id") + .addField(variantType) + .named("table"); + } + + private static GroupType variant(String name, int fieldId) { + return Types.buildGroup(Type.Repetition.REQUIRED) + .id(fieldId) + .required(PrimitiveTypeName.BINARY) + .named("metadata") + .required(PrimitiveTypeName.BINARY) + .named("value") + .named(name); + } + + private static void checkShreddedType(Type shreddedType) { + Preconditions.checkArgument( + shreddedType.getName().equals("typed_value"), + "Invalid shredded type name: %s should be typed_value", + shreddedType.getName()); + Preconditions.checkArgument( + shreddedType.isRepetition(Type.Repetition.OPTIONAL), + "Invalid shredded type repetition: %s should be OPTIONAL", + shreddedType.getRepetition()); + } + + private static Type shreddedPrimitive(PrimitiveTypeName primitive) { + return Types.optional(primitive).named("typed_value"); + } + + private static Type shreddedPrimitive( + PrimitiveTypeName primitive, LogicalTypeAnnotation annotation) { + return Types.optional(primitive).as(annotation).named("typed_value"); + } + + private static Type shreddedType(VariantValue value) { + switch (value.type()) { + case BOOLEAN_TRUE: + case BOOLEAN_FALSE: + return shreddedPrimitive(PrimitiveTypeName.BOOLEAN); + case INT8: + return shreddedPrimitive(PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(8)); + case INT16: + return shreddedPrimitive(PrimitiveTypeName.INT32, LogicalTypeAnnotation.intType(16)); + case INT32: + return shreddedPrimitive(PrimitiveTypeName.INT32); + case INT64: + return shreddedPrimitive(PrimitiveTypeName.INT64); + case FLOAT: + return shreddedPrimitive(PrimitiveTypeName.FLOAT); + case DOUBLE: + return shreddedPrimitive(PrimitiveTypeName.DOUBLE); + case DECIMAL4: + BigDecimal decimal4 = (BigDecimal) value.asPrimitive().get(); + return shreddedPrimitive( + PrimitiveTypeName.INT32, LogicalTypeAnnotation.decimalType(decimal4.scale(), 9)); + case DECIMAL8: + BigDecimal decimal8 = (BigDecimal) value.asPrimitive().get(); + return shreddedPrimitive( + PrimitiveTypeName.INT64, LogicalTypeAnnotation.decimalType(decimal8.scale(), 18)); + case DECIMAL16: + BigDecimal decimal16 = (BigDecimal) value.asPrimitive().get(); + return shreddedPrimitive( + PrimitiveTypeName.BINARY, LogicalTypeAnnotation.decimalType(decimal16.scale(), 38)); + case DATE: + return shreddedPrimitive(PrimitiveTypeName.INT32, LogicalTypeAnnotation.dateType()); + case TIMESTAMPTZ: + return shreddedPrimitive( + PrimitiveTypeName.INT64, LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)); + case TIMESTAMPNTZ: + return shreddedPrimitive( + PrimitiveTypeName.INT64, LogicalTypeAnnotation.timestampType(false, TimeUnit.MICROS)); + case BINARY: + return shreddedPrimitive(PrimitiveTypeName.BINARY); + case STRING: + return shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + } + + throw new UnsupportedOperationException("Unsupported shredding type: " + value.type()); + } + + private static Object toAvroValue(VariantPrimitive variant) { + switch (variant.type()) { + case DECIMAL4: + return ((BigDecimal) variant.get()).unscaledValue().intValueExact(); + case DECIMAL8: + return ((BigDecimal) variant.get()).unscaledValue().longValueExact(); + case DECIMAL16: + return ((BigDecimal) variant.get()).unscaledValue().toByteArray(); + default: + return variant.get(); + } + } + + private static GroupType variant(String name, int fieldId, Type shreddedType) { + checkShreddedType(shreddedType); + return Types.buildGroup(Type.Repetition.OPTIONAL) + .id(fieldId) + .required(PrimitiveTypeName.BINARY) + .named("metadata") + .optional(PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedType) + .named(name); + } + + private static void checkField(GroupType fieldType) { + Preconditions.checkArgument( + fieldType.isRepetition(Type.Repetition.REQUIRED), + "Invalid field type repetition: %s should be REQUIRED", + fieldType.getRepetition()); + } + + private static GroupType objectFields(GroupType... fields) { + for (GroupType fieldType : fields) { + checkField(fieldType); + } + + return Types.buildGroup(Type.Repetition.OPTIONAL).addFields(fields).named("typed_value"); + } + + private static GroupType field(String name, Type shreddedType) { + checkShreddedType(shreddedType); + return Types.buildGroup(Type.Repetition.REQUIRED) + .optional(PrimitiveTypeName.BINARY) + .named("value") + .addField(shreddedType) + .named(name); + } + + private static org.apache.avro.Schema avroSchema(GroupType schema) { + if (schema instanceof MessageType) { + return new AvroSchemaConverter().convert((MessageType) schema); + + } else { + MessageType wrapped = Types.buildMessage().addField(schema).named("table"); + org.apache.avro.Schema avro = + new AvroSchemaConverter().convert(wrapped).getFields().get(0).schema(); + switch (avro.getType()) { + case RECORD: + return avro; + case UNION: + return avro.getTypes().get(1); + } + + throw new IllegalArgumentException("Invalid converted type: " + avro); + } + } +} diff --git a/site/docs/about.md b/site/docs/about.md index b377ca1d1a22..f67fbf3fb8c5 100644 --- a/site/docs/about.md +++ b/site/docs/about.md @@ -22,7 +22,7 @@ Iceberg is a high-performance format for huge analytic tables. Iceberg brings th
  • - + Learn More
  • diff --git a/site/docs/concepts/catalog.md b/site/docs/concepts/catalog.md deleted file mode 100644 index ee0de3b93836..000000000000 --- a/site/docs/concepts/catalog.md +++ /dev/null @@ -1,49 +0,0 @@ ---- -title: "Iceberg Catalogs" ---- - - -# Iceberg Catalogs - -## Overview - -You may think of Iceberg as a format for managing data in a single table, but the Iceberg library needs a way to keep track of those tables by name. Tasks like creating, dropping, and renaming tables are the responsibility of a catalog. Catalogs manage a collection of tables that are usually grouped into namespaces. The most important responsibility of a catalog is tracking a table's current metadata, which is provided by the catalog when you load a table. - -The first step when using an Iceberg client is almost always initializing and configuring a catalog. The configured catalog is then used by compute engines to execute catalog operations. Multiple types of compute engines using a shared Iceberg catalog allows them to share a common data layer. - -A catalog is almost always configured through the processing engine which passes along a set of properties during initialization. Different processing engines have different ways to configure a catalog. When configuring a catalog, it’s always best to refer to the [Iceberg documentation](../docs/latest/configuration.md#catalog-properties) as well as the docs for the specific processing engine being used. Ultimately, these configurations boil down to a common set of catalog properties that will be passed to configure the Iceberg catalog. - -## Catalog Implementations - -Iceberg catalogs are flexible and can be implemented using almost any backend system. They can be plugged into any Iceberg runtime, and allow any processing engine that supports Iceberg to load the tracked Iceberg tables. Iceberg also comes with a number of catalog implementations that are ready to use out of the box. - -This includes: - -* REST: a server-side catalog that’s exposed through a REST API -* Hive Metastore: tracks namespaces and tables using a Hive metastore -* JDBC: tracks namespaces and tables in a simple JDBC database -* Nessie: a transactional catalog that tracks namespaces and tables in a database with git-like version control - -There are more catalog types in addition to the ones listed here as well as custom catalogs that are developed to include specialized functionality. - -## Decoupling Using the REST Catalog - -The REST catalog was introduced in the Iceberg 0.14.0 release and provides greater control over how Iceberg catalogs are implemented. Instead of using technology-specific logic contained in the catalog clients, the implementation details of a REST catalog lives on the catalog server. If you’re familiar with Hive, this is somewhat similar to the Hive thrift service that allows access to a hive server over a single port. The server-side logic can be written in any language and use any custom technology, as long as the API follows the [Iceberg REST Open API specification](https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml). - -A great benefit of the REST catalog is that it allows you to use a single client to talk to any catalog backend. This increased flexibility makes -it easier to make custom catalogs compatible with engines like Athena or Starburst without requiring the inclusion of a Jar into the classpath. diff --git a/site/docs/terms.md b/site/docs/terms.md index 1ab2e5b30d9d..ebca6d478d3d 100644 --- a/site/docs/terms.md +++ b/site/docs/terms.md @@ -20,31 +20,61 @@ title: "Terms" # Terms -### Snapshot +## Catalog + +### Overview + +You may think of Iceberg as a format for managing data in a single table, but the Iceberg library needs a way to keep track of those tables by name. Tasks like creating, dropping, and renaming tables are the responsibility of a catalog. Catalogs manage a collection of tables that are usually grouped into namespaces. The most important responsibility of a catalog is tracking a table's current metadata, which is provided by the catalog when you load a table. + +The first step when using an Iceberg client is almost always initializing and configuring a catalog. The configured catalog is then used by compute engines to execute catalog operations. Multiple types of compute engines using a shared Iceberg catalog allows them to share a common data layer. + +A catalog is almost always configured through the processing engine which passes along a set of properties during initialization. Different processing engines have different ways to configure a catalog. When configuring a catalog, it’s always best to refer to the [Iceberg documentation](../docs/latest/configuration.md#catalog-properties) as well as the docs for the specific processing engine being used. Ultimately, these configurations boil down to a common set of catalog properties that will be passed to configure the Iceberg catalog. + +### Catalog Implementations + +Iceberg catalogs are flexible and can be implemented using almost any backend system. They can be plugged into any Iceberg runtime, and allow any processing engine that supports Iceberg to load the tracked Iceberg tables. Iceberg also comes with a number of catalog implementations that are ready to use out of the box. + +This includes: + +* REST: a server-side catalog that’s exposed through a REST API +* Hive Metastore: tracks namespaces and tables using a Hive metastore +* JDBC: tracks namespaces and tables in a simple JDBC database +* Nessie: a transactional catalog that tracks namespaces and tables in a database with git-like version control + +There are more catalog types in addition to the ones listed here as well as custom catalogs that are developed to include specialized functionality. + +### Decoupling Using the REST Catalog + +The REST catalog was introduced in the Iceberg 0.14.0 release and provides greater control over how Iceberg catalogs are implemented. Instead of using technology-specific logic contained in the catalog clients, the implementation details of a REST catalog lives on the catalog server. If you’re familiar with Hive, this is somewhat similar to the Hive thrift service that allows access to a hive server over a single port. The server-side logic can be written in any language and use any custom technology, as long as the API follows the [Iceberg REST Open API specification](https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml). + +A great benefit of the REST catalog is that it allows you to use a single client to talk to any catalog backend. This increased flexibility makes +it easier to make custom catalogs compatible with engines like Athena or Starburst without requiring the inclusion of a Jar into the classpath. + +## Snapshot A **snapshot** is the state of a table at some time. Each snapshot lists all of the data files that make up the table's contents at the time of the snapshot. Data files are stored across multiple [manifest](#manifest-file) files, and the manifests for a snapshot are listed in a single [manifest list](#manifest-list) file. -### Manifest list +## Manifest list A **manifest list** is a metadata file that lists the [manifests](#manifest-file) that make up a table snapshot. Each manifest file in the manifest list is stored with information about its contents, like partition value ranges, used to speed up metadata operations. -### Manifest file +## Manifest file A **manifest file** is a metadata file that lists a subset of data files that make up a snapshot. Each data file in a manifest is stored with a [partition tuple](#partition-tuple), column-level stats, and summary information used to prune splits during [scan planning](docs/latest/performance.md#scan-planning). -### Partition spec +## Partition spec A **partition spec** is a description of how to [partition](docs/latest/partitioning.md) data in a table. A spec consists of a list of source columns and transforms. A transform produces a partition value from a source value. For example, `date(ts)` produces the date associated with a timestamp column named `ts`. -### Partition tuple +## Partition tuple A **partition tuple** is a tuple or struct of partition data stored with each data file. @@ -52,7 +82,7 @@ All values in a partition tuple are the same for all rows stored in a data file. Iceberg stores partition values unmodified, unlike Hive tables that convert values to and from strings in file system paths and keys. -### Snapshot log (history table) +## Snapshot log (history table) The **snapshot log** is a metadata log of how the table's current snapshot has changed over time. diff --git a/site/nav.yml b/site/nav.yml index eb247cdaeb34..a3a5194c351a 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -43,15 +43,8 @@ nav: - Project: - Community: community.md - Contributing: contribute.md - - REST Catalog Spec: https://editor-next.swagger.io/?url=https://raw.githubusercontent.com/apache/iceberg/main/open-api/rest-catalog-open-api.yaml - - Table Spec: spec.md - - View spec: view-spec.md - - Puffin spec: puffin-spec.md - - AES GCM Stream spec: gcm-stream-spec.md - - Implementation status: status.md - Multi-engine support: multi-engine-support.md - How to release: how-to-release.md - - Terms: terms.md - ASF: - Sponsorship: https://www.apache.org/foundation/thanks.html - Events: https://www.apache.org/events/current-event.html @@ -59,5 +52,11 @@ nav: - License: https://www.apache.org/licenses/ - Security: https://www.apache.org/security/ - Sponsors: https://www.apache.org/foundation/thanks.html - - Concepts: - - Catalogs: concepts/catalog.md + - Specification: + - Terms: terms.md + - REST Catalog Spec: https://editor-next.swagger.io/?url=https://raw.githubusercontent.com/apache/iceberg/main/open-api/rest-catalog-open-api.yaml + - Table Spec: spec.md + - View spec: view-spec.md + - Puffin spec: puffin-spec.md + - AES GCM Stream spec: gcm-stream-spec.md + - Implementation status: status.md diff --git a/site/requirements.txt b/site/requirements.txt index 595fd0fc30b3..be65f4ac0574 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.10.1 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.6.3 +mkdocs-material==9.6.4 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java index af0fa84f67a1..ad8a4beb55d0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -565,6 +565,11 @@ public String map(Types.MapType map, String keyResult, String valueResult) { return "map<" + keyResult + ", " + valueResult + ">"; } + @Override + public String variant(Types.VariantType variant) { + return "variant"; + } + @Override public String primitive(Type.PrimitiveType primitive) { switch (primitive.typeId()) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index c9382fafd8ae..4c9e5c5e4a96 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -239,7 +239,7 @@ private boolean versionInFilePath(String path, String version) { } private String jobDesc() { - if (startVersionName != null) { + if (startVersionName == null) { return String.format( "Replacing path prefixes '%s' with '%s' in the metadata files of table %s," + "up to version '%s'.", diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java index 6f900ffebb10..e4e66abfefa0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java @@ -105,12 +105,13 @@ public void testDescribeSchema() { 3, "pairs", Types.MapType.ofOptional(4, 5, Types.StringType.get(), Types.LongType.get())), - required(6, "time", Types.TimestampType.withoutZone())); + required(6, "time", Types.TimestampType.withoutZone()), + required(7, "v", Types.VariantType.get())); assertThat(Spark3Util.describe(schema)) .as("Schema description isn't correct.") .isEqualTo( - "struct not null,pairs: map,time: timestamp not null>"); + "struct not null,pairs: map,time: timestamp not null,v: variant not null>"); } @Test