Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

API: Deprecate NestedType.of in favor of builder #12227

Merged
merged 5 commits into from
Feb 14, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions api/src/main/java/org/apache/iceberg/types/Types.java
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
Expand Down
7 changes: 4 additions & 3 deletions api/src/test/java/org/apache/iceberg/TestSchema.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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 =
Expand All @@ -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) {
Expand Down
9 changes: 5 additions & 4 deletions api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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));
Expand All @@ -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())));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
27 changes: 15 additions & 12 deletions core/src/main/java/org/apache/iceberg/MetricsUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -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());
}
}

Expand Down
2 changes: 1 addition & 1 deletion core/src/test/java/org/apache/iceberg/TestDeleteFiles.java
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
36 changes: 20 additions & 16 deletions core/src/test/java/org/apache/iceberg/TestSchemaParser.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -85,30 +86,31 @@ public void testDocStrings() {

private static Stream<Arguments> 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()),
Expand All @@ -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());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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()));
Expand All @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -47,7 +48,11 @@ public class TestCreateTableRequest extends RequestResponseTestBase<CreateTableR
private static final String SAMPLE_LOCATION = "file://tmp/location/";
private static final Schema SAMPLE_SCHEMA =
new Schema(
required("id").withId(1).ofType(Types.IntegerType.get()).withWriteDefault(1).build(),
required("id")
.withId(1)
.ofType(Types.IntegerType.get())
.withWriteDefault(Literal.of(1))
.build(),
optional("data").withId(2).ofType(Types.StringType.get()).build());
private static final String SAMPLE_SCHEMA_JSON = SchemaParser.toJson(SAMPLE_SCHEMA);
private static final PartitionSpec SAMPLE_SPEC =
Expand Down
Loading