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 d1b4241bcba6..6620263f0d3f 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -540,10 +540,22 @@ public static NestedField required(int id, String name, Type type, String doc) { return new NestedField(false, id, name, type, doc, null, null); } + /** + * Create a nested field. + * + * @deprecated will be removed in 2.0.0; use {@link #builder()} instead. + */ + @Deprecated public static NestedField of(int id, boolean isOptional, String name, Type type) { return new NestedField(isOptional, id, name, type, null, null, null); } + /** + * Create a nested field. + * + * @deprecated will be removed in 2.0.0; use {@link #builder()} instead. + */ + @Deprecated public static NestedField of(int id, boolean isOptional, String name, Type type, String doc) { return new NestedField(isOptional, id, name, type, doc, null, null); } diff --git a/api/src/test/java/org/apache/iceberg/TestSchema.java b/api/src/test/java/org/apache/iceberg/TestSchema.java index 46db60852b3f..aca3c1ad6637 100644 --- a/api/src/test/java/org/apache/iceberg/TestSchema.java +++ b/api/src/test/java/org/apache/iceberg/TestSchema.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.stream.IntStream; import java.util.stream.Stream; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -50,8 +51,8 @@ public class TestSchema { Types.NestedField.required("has_default") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("--") - .withWriteDefault("--") + .withInitialDefault(Literal.of("--")) + .withWriteDefault(Literal.of("--")) .build()); private static final Schema WRITE_DEFAULT_SCHEMA = @@ -60,7 +61,7 @@ public class TestSchema { Types.NestedField.required("has_default") .withId(2) .ofType(Types.StringType.get()) - .withWriteDefault("--") + .withWriteDefault(Literal.of("--")) .build()); private Schema generateTypeSchema(Type type) { 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 36384d232af3..b7ecb5e7ec02 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java @@ -25,6 +25,7 @@ import java.util.Set; 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; @@ -598,8 +599,8 @@ public void testReassignOrRefreshIds() { Types.NestedField.required("c") .withId(11) .ofType(Types.IntegerType.get()) - .withInitialDefault(23) - .withWriteDefault(34) + .withInitialDefault(Literal.of(23)) + .withWriteDefault(Literal.of(34)) .build(), required(12, "B", Types.IntegerType.get())), Sets.newHashSet(10)); @@ -617,8 +618,8 @@ public void testReassignOrRefreshIds() { Types.NestedField.required("c") .withId(16) .ofType(Types.IntegerType.get()) - .withInitialDefault(23) - .withWriteDefault(34) + .withInitialDefault(Literal.of(23)) + .withWriteDefault(Literal.of(34)) .build(), required(15, "B", Types.IntegerType.get()))); diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java index 411f241e169f..ef33660dacbe 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java @@ -238,9 +238,7 @@ private static Types.NestedField getPhysicalType( // Use FixedSizeBinaryVector for binary backed decimal type = Types.FixedType.ofLength(primitive.getTypeLength()); } - physicalType = - Types.NestedField.of( - logicalType.fieldId(), logicalType.isOptional(), logicalType.name(), type); + physicalType = Types.NestedField.from(logicalType).ofType(type).build(); } return physicalType; diff --git a/core/src/main/java/org/apache/iceberg/MetricsUtil.java b/core/src/main/java/org/apache/iceberg/MetricsUtil.java index 2d23121bb0f9..f0c1970049cd 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsUtil.java +++ b/core/src/main/java/org/apache/iceberg/MetricsUtil.java @@ -346,18 +346,21 @@ public static Schema readableMetricsSchema(Schema dataTableSchema, Schema metada String colName = idToName.get(id); fields.add( - Types.NestedField.of( - nextId.incrementAndGet(), - true, - colName, - Types.StructType.of( - READABLE_METRIC_COLS.stream() - .map( - m -> - optional( - nextId.incrementAndGet(), m.name(), m.colType(field), m.doc())) - .collect(Collectors.toList())), - String.format("Metrics for column %s", colName))); + Types.NestedField.optional(colName) + .withId(nextId.incrementAndGet()) + .ofType( + Types.StructType.of( + READABLE_METRIC_COLS.stream() + .map( + m -> + optional( + nextId.incrementAndGet(), + m.name(), + m.colType(field), + m.doc())) + .collect(Collectors.toList()))) + .withDoc(String.format("Metrics for column %s", colName)) + .build()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index 41dfc3ed7d89..6ba65e6b7f00 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -352,7 +352,7 @@ public void testDeleteFilesOnIndependentBranches() { @TestTemplate public void testDeleteWithCollision() { - Schema schema = new Schema(Types.NestedField.of(0, false, "x", Types.StringType.get())); + Schema schema = new Schema(Types.NestedField.required(0, "x", Types.StringType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).identity("x").build(); Table collisionTable = TestTables.create(tableDir, "hashcollision", schema, spec, formatVersion); diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaParser.java b/core/src/test/java/org/apache/iceberg/TestSchemaParser.java index ebd197a68af0..108371416b14 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaParser.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaParser.java @@ -28,6 +28,7 @@ import java.util.UUID; import java.util.stream.Stream; import org.apache.iceberg.avro.AvroDataTest; +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.Type; @@ -85,30 +86,31 @@ public void testDocStrings() { private static Stream primitiveTypesAndDefaults() { return Stream.of( - Arguments.of(Types.BooleanType.get(), false), - Arguments.of(Types.IntegerType.get(), 34), - Arguments.of(Types.LongType.get(), 4900000000L), - Arguments.of(Types.FloatType.get(), 12.21F), - Arguments.of(Types.DoubleType.get(), -0.0D), - Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + Arguments.of(Types.BooleanType.get(), Literal.of(false)), + Arguments.of(Types.IntegerType.get(), Literal.of(34)), + Arguments.of(Types.LongType.get(), Literal.of(4900000000L)), + Arguments.of(Types.FloatType.get(), Literal.of(12.21F)), + Arguments.of(Types.DoubleType.get(), Literal.of(-0.0D)), + Arguments.of(Types.DateType.get(), Literal.of(DateTimeUtil.isoDateToDays("2024-12-17"))), // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), Arguments.of( Types.TimestampType.withZone(), - DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Literal.of(DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00"))), Arguments.of( Types.TimestampType.withoutZone(), - DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), - Arguments.of(Types.StringType.get(), "iceberg"), - Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Literal.of(DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999"))), + Arguments.of(Types.StringType.get(), Literal.of("iceberg")), + Arguments.of(Types.UUIDType.get(), Literal.of(UUID.randomUUID())), Arguments.of( - Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), - Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), - Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + Types.FixedType.ofLength(4), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d}))), + Arguments.of(Types.BinaryType.get(), Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b}))), + Arguments.of(Types.DecimalType.of(9, 2), Literal.of(new BigDecimal("12.34")))); } @ParameterizedTest @MethodSource("primitiveTypesAndDefaults") - public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) { + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal defaultValue) { Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -120,7 +122,9 @@ public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defau .build()); Schema serialized = SchemaParser.fromJson(SchemaParser.toJson(schema)); - assertThat(serialized.findField("col_with_default").initialDefault()).isEqualTo(defaultValue); - assertThat(serialized.findField("col_with_default").writeDefault()).isEqualTo(defaultValue); + assertThat(serialized.findField("col_with_default").initialDefault()) + .isEqualTo(defaultValue.value()); + assertThat(serialized.findField("col_with_default").writeDefault()) + .isEqualTo(defaultValue.value()); } } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java b/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java index 1cfe88206b1b..27a04f0645dd 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java @@ -28,6 +28,8 @@ import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.SingleValueParser; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; @@ -94,7 +96,7 @@ public void testDefaultAppliedWhenMissingColumn() throws IOException { Types.NestedField.optional("defaulted") .withId(1000) .ofType(type) - .withInitialDefault(defaultValue) + .withInitialDefault(Expressions.lit(defaultValue)) .build()); Record expectedRecord = new Record(AvroSchemaUtil.convert(readerSchema.asStruct())); @@ -119,7 +121,7 @@ public void testDefaultDoesNotOverrideExplicitValue() throws IOException { for (Object[] typeAndDefault : TYPES_WITH_DEFAULTS) { Type type = (Type) typeAndDefault[0]; String defaultValueJson = (String) typeAndDefault[1]; - Object defaultValue = SingleValueParser.fromJson(type, defaultValueJson); + Literal defaultValue = Expressions.lit(SingleValueParser.fromJson(type, defaultValueJson)); Schema readerSchema = new Schema( diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java index 77413ffe6bf6..e061a6986aa5 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java @@ -32,6 +32,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SortOrder; import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RequestResponseTestBase; @@ -47,7 +48,11 @@ public class TestCreateTableRequest extends RequestResponseTestBase primitiveTypesAndDefaults() { return Stream.of( - Arguments.of(Types.BooleanType.get(), false), - Arguments.of(Types.IntegerType.get(), 34), - Arguments.of(Types.LongType.get(), 4900000000L), - Arguments.of(Types.FloatType.get(), 12.21F), - Arguments.of(Types.DoubleType.get(), -0.0D), - Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), - Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), + Arguments.of(Types.BooleanType.get(), Literal.of(false)), + Arguments.of(Types.IntegerType.get(), Literal.of(34)), + Arguments.of(Types.LongType.get(), Literal.of(4900000000L)), + Arguments.of(Types.FloatType.get(), Literal.of(12.21F)), + Arguments.of(Types.DoubleType.get(), Literal.of(-0.0D)), + Arguments.of(Types.DateType.get(), Literal.of(DateTimeUtil.isoDateToDays("2024-12-17"))), + // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), Arguments.of( Types.TimestampType.withZone(), - DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Literal.of(DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00"))), Arguments.of( Types.TimestampType.withoutZone(), - DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), - Arguments.of(Types.StringType.get(), "iceberg"), - Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Literal.of(DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999"))), + Arguments.of(Types.StringType.get(), Literal.of("iceberg")), + Arguments.of(Types.UUIDType.get(), Literal.of(UUID.randomUUID())), Arguments.of( - Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), - Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), - Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + Types.FixedType.ofLength(4), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d}))), + Arguments.of(Types.BinaryType.get(), Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b}))), + Arguments.of(Types.DecimalType.of(9, 2), Literal.of(new BigDecimal("12.34")))); } @ParameterizedTest @MethodSource("primitiveTypesAndDefaults") - public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal defaultValue) throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 5b979e546d51..c99b74c14b44 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -68,12 +68,7 @@ class SortKeySerializer extends TypeSerializer { Types.NestedField sourceField = schema.findField(sortField.sourceId()); Type resultType = sortField.transform().getResultType(sourceField.type()); Types.NestedField transformedField = - Types.NestedField.of( - sourceField.fieldId(), - sourceField.isOptional(), - sourceField.name(), - resultType, - sourceField.doc()); + Types.NestedField.from(sourceField).ofType(resultType).build(); transformedFields[i] = transformedField; } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java index 1e5bdbbac3e4..891f2f02a241 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java @@ -45,12 +45,11 @@ static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { // case. To resolve the collision, field id is set to transform index and field name is set to // sourceFieldName_transformIndex Types.NestedField transformedField = - Types.NestedField.of( - i, - sourceField.isOptional(), - sourceField.name() + '_' + i, - transformedType, - sourceField.doc()); + Types.NestedField.from(sourceField) + .withId(i) + .withName(sourceField.name() + '_' + i) + .ofType(transformedType) + .build(); transformedFields.add(transformedField); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java index a2e0729fd506..b0dd56b45d67 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java @@ -276,11 +276,13 @@ Type toIcebergType(Schema valueSchema) { valueSchema.fields().stream() .map( field -> - NestedField.of( - nextId(), - config.schemaForceOptional() || field.schema().isOptional(), - field.name(), - toIcebergType(field.schema()))) + NestedField.builder() + .isOptional( + config.schemaForceOptional() || field.schema().isOptional()) + .withId(nextId()) + .ofType(toIcebergType(field.schema())) + .withName(field.name()) + .build()) .collect(Collectors.toList()); return StructType.of(structFields); case STRING: diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcToIcebergVisitor.java b/orc/src/main/java/org/apache/iceberg/orc/OrcToIcebergVisitor.java index 6992f88b870b..5cccbf6955e0 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcToIcebergVisitor.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcToIcebergVisitor.java @@ -43,7 +43,12 @@ public Optional record( .map(Optional::get) .collect(Collectors.toList())); return Optional.of( - Types.NestedField.of(icebergIdOpt.get(), isOptional, currentFieldName(), structType)); + Types.NestedField.builder() + .withId(icebergIdOpt.get()) + .isOptional(isOptional) + .withName(currentFieldName()) + .ofType(structType) + .build()); } @Override @@ -63,7 +68,12 @@ public Optional list( : Types.ListType.ofRequired(foundElement.fieldId(), foundElement.type()); return Optional.of( - Types.NestedField.of(icebergIdOpt.get(), isOptional, currentFieldName(), listTypeWithElem)); + Types.NestedField.builder() + .withId(icebergIdOpt.get()) + .isOptional(isOptional) + .withName(currentFieldName()) + .ofType(listTypeWithElem) + .build()); } @Override @@ -86,7 +96,12 @@ public Optional map( foundKey.fieldId(), foundValue.fieldId(), foundKey.type(), foundValue.type()); return Optional.of( - Types.NestedField.of(icebergIdOpt.get(), isOptional, currentFieldName(), mapTypeWithKV)); + Types.NestedField.builder() + .withId(icebergIdOpt.get()) + .isOptional(isOptional) + .withName(currentFieldName()) + .ofType(mapTypeWithKV) + .build()); } @Override @@ -98,17 +113,19 @@ public Optional primitive(TypeDescription primitive) { return Optional.empty(); } - final Types.NestedField foundField; - int icebergID = icebergIdOpt.get(); - String name = currentFieldName(); + Types.NestedField.Builder builder = + Types.NestedField.builder() + .withId(icebergIdOpt.get()) + .isOptional(isOptional) + .withName(currentFieldName()); switch (primitive.getCategory()) { case BOOLEAN: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.BooleanType.get()); + builder.ofType(Types.BooleanType.get()); break; case BYTE: case SHORT: case INT: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.IntegerType.get()); + builder.ofType(Types.IntegerType.get()); break; case LONG: String longAttributeValue = @@ -119,25 +136,25 @@ public Optional primitive(TypeDescription primitive) { : ORCSchemaUtil.LongType.valueOf(longAttributeValue); switch (longType) { case TIME: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.TimeType.get()); + builder.ofType(Types.TimeType.get()); break; case LONG: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.LongType.get()); + builder.ofType(Types.LongType.get()); break; default: throw new IllegalStateException("Invalid Long type found in ORC type attribute"); } break; case FLOAT: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.FloatType.get()); + builder.ofType(Types.FloatType.get()); break; case DOUBLE: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.DoubleType.get()); + builder.ofType(Types.DoubleType.get()); break; case STRING: case CHAR: case VARCHAR: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.StringType.get()); + builder.ofType(Types.StringType.get()); break; case BINARY: String binaryAttributeValue = @@ -148,44 +165,36 @@ public Optional primitive(TypeDescription primitive) { : ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue); switch (binaryType) { case UUID: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.UUIDType.get()); + builder.ofType(Types.UUIDType.get()); break; case FIXED: int fixedLength = Integer.parseInt(primitive.getAttributeValue(ORCSchemaUtil.ICEBERG_FIELD_LENGTH)); - foundField = - Types.NestedField.of( - icebergID, isOptional, name, Types.FixedType.ofLength(fixedLength)); + builder.ofType(Types.FixedType.ofLength(fixedLength)); break; case BINARY: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.BinaryType.get()); + builder.ofType(Types.BinaryType.get()); break; default: throw new IllegalStateException("Invalid Binary type found in ORC type attribute"); } break; case DATE: - foundField = Types.NestedField.of(icebergID, isOptional, name, Types.DateType.get()); + builder.ofType(Types.DateType.get()); break; case TIMESTAMP: - foundField = - Types.NestedField.of(icebergID, isOptional, name, Types.TimestampType.withoutZone()); + builder.ofType(Types.TimestampType.withoutZone()); break; case TIMESTAMP_INSTANT: - foundField = - Types.NestedField.of(icebergID, isOptional, name, Types.TimestampType.withZone()); + builder.ofType(Types.TimestampType.withZone()); break; case DECIMAL: - foundField = - Types.NestedField.of( - icebergID, - isOptional, - name, - Types.DecimalType.of(primitive.getPrecision(), primitive.getScale())); + builder.ofType(Types.DecimalType.of(primitive.getPrecision(), primitive.getScale())); break; default: throw new IllegalArgumentException("Can't handle " + primitive); } - return Optional.of(foundField); + + return Optional.of(builder.build()); } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java b/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java index 2d35f5861396..f42a9cffa81e 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java @@ -24,6 +24,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; import org.junit.jupiter.api.Test; @@ -184,7 +185,7 @@ public void testRequiredNestedFieldWithDefaultMissingInFile() { Types.NestedField.required("d") .withId(4) .ofType(Types.LongType.get()) - .withInitialDefault(34L) + .withInitialDefault(Literal.of(34L)) .build()))); assertThatThrownBy(() -> ORCSchemaUtil.buildOrcProjection(evolvedSchema, baseOrcSchema)) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index d7ecef758c47..4b5f83325f41 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -32,6 +32,7 @@ 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.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -321,7 +322,7 @@ public void testDefaultValues() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build()); @@ -331,17 +332,17 @@ public void testDefaultValues() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.required("missing_str") .withId(6) .ofType(Types.StringType.get()) - .withInitialDefault("orange") + .withInitialDefault(Literal.of("orange")) .build(), Types.NestedField.optional("missing_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()); writeAndValidate(writeSchema, expectedSchema); @@ -357,7 +358,7 @@ public void testNullDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build()); @@ -367,7 +368,7 @@ public void testNullDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("missing_date") .withId(3) @@ -388,7 +389,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested") @@ -403,7 +404,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested") .withId(3) @@ -413,7 +414,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("missing_inner_float") .withId(5) .ofType(Types.FloatType.get()) - .withInitialDefault(-0.0F) + .withInitialDefault(Literal.of(-0.0F)) .build())) .withDoc("Used to test nested field defaults") .build()); @@ -432,7 +433,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested_map") @@ -452,7 +453,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested_map") .withId(3) @@ -466,7 +467,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("value_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()))) .withDoc("Used to test nested field defaults") .build()); @@ -485,7 +486,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested_list") @@ -502,7 +503,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested_list") .withId(3) @@ -514,7 +515,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("element_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()))) .withDoc("Used to test nested field defaults") .build()); @@ -524,30 +525,31 @@ public void testListNestedDefaultValue() throws IOException { private static Stream primitiveTypesAndDefaults() { return Stream.of( - Arguments.of(Types.BooleanType.get(), false), - Arguments.of(Types.IntegerType.get(), 34), - Arguments.of(Types.LongType.get(), 4900000000L), - Arguments.of(Types.FloatType.get(), 12.21F), - Arguments.of(Types.DoubleType.get(), -0.0D), - Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + Arguments.of(Types.BooleanType.get(), Literal.of(false)), + Arguments.of(Types.IntegerType.get(), Literal.of(34)), + Arguments.of(Types.LongType.get(), Literal.of(4900000000L)), + Arguments.of(Types.FloatType.get(), Literal.of(12.21F)), + Arguments.of(Types.DoubleType.get(), Literal.of(-0.0D)), + Arguments.of(Types.DateType.get(), Literal.of(DateTimeUtil.isoDateToDays("2024-12-17"))), // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), Arguments.of( Types.TimestampType.withZone(), - DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Literal.of(DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00"))), // Arguments.of( // Types.TimestampType.withoutZone(), - // DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), - Arguments.of(Types.StringType.get(), "iceberg"), - Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + // Literal.of(DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999"))), + Arguments.of(Types.StringType.get(), Literal.of("iceberg")), + Arguments.of(Types.UUIDType.get(), Literal.of(UUID.randomUUID())), Arguments.of( - Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), - Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), - Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + Types.FixedType.ofLength(4), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d}))), + Arguments.of(Types.BinaryType.get(), Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b}))), + Arguments.of(Types.DecimalType.of(9, 2), Literal.of(new BigDecimal("12.34")))); } @ParameterizedTest @MethodSource("primitiveTypesAndDefaults") - public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal defaultValue) throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index ad969384c586..4e4000794ec4 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -31,6 +31,7 @@ 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.types.Type; import org.apache.iceberg.types.TypeUtil; @@ -302,7 +303,7 @@ public void testDefaultValues() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build()); @@ -312,17 +313,17 @@ public void testDefaultValues() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.required("missing_str") .withId(6) .ofType(Types.StringType.get()) - .withInitialDefault("orange") + .withInitialDefault(Literal.of("orange")) .build(), Types.NestedField.optional("missing_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()); writeAndValidate(writeSchema, expectedSchema); @@ -338,7 +339,7 @@ public void testNullDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build()); @@ -348,7 +349,7 @@ public void testNullDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("missing_date") .withId(3) @@ -369,7 +370,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested") @@ -384,7 +385,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested") .withId(3) @@ -394,7 +395,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("missing_inner_float") .withId(5) .ofType(Types.FloatType.get()) - .withInitialDefault(-0.0F) + .withInitialDefault(Literal.of(-0.0F)) .build())) .withDoc("Used to test nested field defaults") .build()); @@ -413,7 +414,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested_map") @@ -433,7 +434,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested_map") .withId(3) @@ -447,7 +448,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("value_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()))) .withDoc("Used to test nested field defaults") .build()); @@ -466,7 +467,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested_list") @@ -483,7 +484,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested_list") .withId(3) @@ -495,7 +496,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("element_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()))) .withDoc("Used to test nested field defaults") .build()); @@ -505,30 +506,31 @@ public void testListNestedDefaultValue() throws IOException { private static Stream primitiveTypesAndDefaults() { return Stream.of( - Arguments.of(Types.BooleanType.get(), false), - Arguments.of(Types.IntegerType.get(), 34), - Arguments.of(Types.LongType.get(), 4900000000L), - Arguments.of(Types.FloatType.get(), 12.21F), - Arguments.of(Types.DoubleType.get(), -0.0D), - Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + Arguments.of(Types.BooleanType.get(), Literal.of(false)), + Arguments.of(Types.IntegerType.get(), Literal.of(34)), + Arguments.of(Types.LongType.get(), Literal.of(4900000000L)), + Arguments.of(Types.FloatType.get(), Literal.of(12.21F)), + Arguments.of(Types.DoubleType.get(), Literal.of(-0.0D)), + Arguments.of(Types.DateType.get(), Literal.of(DateTimeUtil.isoDateToDays("2024-12-17"))), // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), Arguments.of( Types.TimestampType.withZone(), - DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Literal.of(DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00"))), Arguments.of( Types.TimestampType.withoutZone(), - DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), - Arguments.of(Types.StringType.get(), "iceberg"), - Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Literal.of(DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999"))), + Arguments.of(Types.StringType.get(), Literal.of("iceberg")), + Arguments.of(Types.UUIDType.get(), Literal.of(UUID.randomUUID())), Arguments.of( - Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), - Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), - Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + Types.FixedType.ofLength(4), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d}))), + Arguments.of(Types.BinaryType.get(), Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b}))), + Arguments.of(Types.DecimalType.of(9, 2), Literal.of(new BigDecimal("12.34")))); } @ParameterizedTest @MethodSource("primitiveTypesAndDefaults") - public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal defaultValue) throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index ad969384c586..4e4000794ec4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -31,6 +31,7 @@ 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.types.Type; import org.apache.iceberg.types.TypeUtil; @@ -302,7 +303,7 @@ public void testDefaultValues() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build()); @@ -312,17 +313,17 @@ public void testDefaultValues() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.required("missing_str") .withId(6) .ofType(Types.StringType.get()) - .withInitialDefault("orange") + .withInitialDefault(Literal.of("orange")) .build(), Types.NestedField.optional("missing_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()); writeAndValidate(writeSchema, expectedSchema); @@ -338,7 +339,7 @@ public void testNullDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build()); @@ -348,7 +349,7 @@ public void testNullDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("missing_date") .withId(3) @@ -369,7 +370,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested") @@ -384,7 +385,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested") .withId(3) @@ -394,7 +395,7 @@ public void testNestedDefaultValue() throws IOException { Types.NestedField.optional("missing_inner_float") .withId(5) .ofType(Types.FloatType.get()) - .withInitialDefault(-0.0F) + .withInitialDefault(Literal.of(-0.0F)) .build())) .withDoc("Used to test nested field defaults") .build()); @@ -413,7 +414,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested_map") @@ -433,7 +434,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested_map") .withId(3) @@ -447,7 +448,7 @@ public void testMapNestedDefaultValue() throws IOException { Types.NestedField.optional("value_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()))) .withDoc("Used to test nested field defaults") .build()); @@ -466,7 +467,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .withDoc("Should not produce default value") .build(), Types.NestedField.optional("nested_list") @@ -483,7 +484,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("data") .withId(2) .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") + .withInitialDefault(Literal.of("wrong!")) .build(), Types.NestedField.optional("nested_list") .withId(3) @@ -495,7 +496,7 @@ public void testListNestedDefaultValue() throws IOException { Types.NestedField.optional("element_int") .withId(7) .ofType(Types.IntegerType.get()) - .withInitialDefault(34) + .withInitialDefault(Literal.of(34)) .build()))) .withDoc("Used to test nested field defaults") .build()); @@ -505,30 +506,31 @@ public void testListNestedDefaultValue() throws IOException { private static Stream primitiveTypesAndDefaults() { return Stream.of( - Arguments.of(Types.BooleanType.get(), false), - Arguments.of(Types.IntegerType.get(), 34), - Arguments.of(Types.LongType.get(), 4900000000L), - Arguments.of(Types.FloatType.get(), 12.21F), - Arguments.of(Types.DoubleType.get(), -0.0D), - Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + Arguments.of(Types.BooleanType.get(), Literal.of(false)), + Arguments.of(Types.IntegerType.get(), Literal.of(34)), + Arguments.of(Types.LongType.get(), Literal.of(4900000000L)), + Arguments.of(Types.FloatType.get(), Literal.of(12.21F)), + Arguments.of(Types.DoubleType.get(), Literal.of(-0.0D)), + Arguments.of(Types.DateType.get(), Literal.of(DateTimeUtil.isoDateToDays("2024-12-17"))), // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), Arguments.of( Types.TimestampType.withZone(), - DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Literal.of(DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00"))), Arguments.of( Types.TimestampType.withoutZone(), - DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), - Arguments.of(Types.StringType.get(), "iceberg"), - Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Literal.of(DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999"))), + Arguments.of(Types.StringType.get(), Literal.of("iceberg")), + Arguments.of(Types.UUIDType.get(), Literal.of(UUID.randomUUID())), Arguments.of( - Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), - Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), - Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + Types.FixedType.ofLength(4), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d}))), + Arguments.of(Types.BinaryType.get(), Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b}))), + Arguments.of(Types.DecimalType.of(9, 2), Literal.of(new BigDecimal("12.34")))); } @ParameterizedTest @MethodSource("primitiveTypesAndDefaults") - public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal defaultValue) throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue();