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

[IcebergIO] Use InternalRecordWrapper partition util #33701

Merged
merged 3 commits into from
Jan 23, 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
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 2
"modification": 3
}
1 change: 1 addition & 0 deletions sdks/java/io/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ dependencies {
implementation "org.apache.iceberg:iceberg-api:$iceberg_version"
implementation "org.apache.iceberg:iceberg-parquet:$iceberg_version"
implementation "org.apache.iceberg:iceberg-orc:$iceberg_version"
implementation "org.apache.iceberg:iceberg-data:$iceberg_version"
implementation library.java.hadoop_common
runtimeOnly "org.apache.iceberg:iceberg-gcp:$iceberg_version"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,14 +50,11 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.InternalRecordWrapper;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.exceptions.NoSuchTableException;
import org.apache.iceberg.expressions.Literal;
import org.apache.iceberg.transforms.Transform;
import org.apache.iceberg.transforms.Transforms;
import org.apache.iceberg.types.Types;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -106,12 +103,14 @@ class DestinationState {
@VisibleForTesting final Map<PartitionKey, Integer> writerCounts = Maps.newHashMap();
private final Map<String, PartitionField> partitionFieldMap = Maps.newHashMap();
private final List<Exception> exceptions = Lists.newArrayList();
private final InternalRecordWrapper wrapper; // wrapper that facilitates partitioning

DestinationState(IcebergDestination icebergDestination, Table table) {
this.icebergDestination = icebergDestination;
this.schema = table.schema();
this.spec = table.spec();
this.routingPartitionKey = new PartitionKey(spec, schema);
this.wrapper = new InternalRecordWrapper(schema.asStruct());
this.table = table;
for (PartitionField partitionField : spec.fields()) {
partitionFieldMap.put(partitionField.name(), partitionField);
Expand Down Expand Up @@ -156,7 +155,7 @@ class DestinationState {
* can't create a new writer, the {@link Record} is rejected and {@code false} is returned.
*/
boolean write(Record record) {
routingPartitionKey.partition(getPartitionableRecord(record));
routingPartitionKey.partition(wrapper.wrap(record));

if (!writers.asMap().containsKey(routingPartitionKey) && openWriters >= maxNumWriters) {
return false;
Expand Down Expand Up @@ -207,30 +206,6 @@ private RecordWriter createWriter(PartitionKey partitionKey) {
e);
}
}

/**
* Resolves an input {@link Record}'s partition values and returns another {@link Record} that
* can be applied to the destination's {@link PartitionSpec}.
*/
private Record getPartitionableRecord(Record record) {
if (spec.isUnpartitioned()) {
return record;
}
Record output = GenericRecord.create(schema);
for (PartitionField partitionField : spec.fields()) {
Transform<?, ?> transform = partitionField.transform();
Types.NestedField field = schema.findField(partitionField.sourceId());
String name = field.name();
Object value = record.getField(name);
@Nullable Literal<Object> literal = Literal.of(value.toString()).to(field.type());
if (literal == null || transform.isVoid() || transform.isIdentity()) {
output.setField(name, value);
} else {
output.setField(name, literal.value());
}
}
return output;
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.beam.sdk.io.iceberg;

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
Expand All @@ -27,9 +28,12 @@
import static org.junit.Assert.assertTrue;

import java.io.IOException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -451,10 +455,27 @@ public void testIdentityPartitioning() throws IOException {
.addFloatField("float")
.addDoubleField("double")
.addStringField("str")
.addLogicalTypeField("date", SqlTypes.DATE)
.addLogicalTypeField("time", SqlTypes.TIME)
.addLogicalTypeField("datetime", SqlTypes.DATETIME)
.addDateTimeField("datetime_tz")
.build();

String timestamp = "2025-01-21T13:18:20.053";
LocalDateTime localDateTime = LocalDateTime.parse(timestamp);
Row row =
Row.withSchema(primitiveTypeSchema).addValues(true, 1, 1L, 1.23f, 4.56, "str").build();
Row.withSchema(primitiveTypeSchema)
.addValues(
true,
1,
1L,
1.23f,
4.56,
"str",
localDateTime.toLocalDate(),
localDateTime.toLocalTime(),
localDateTime,
DateTime.parse(timestamp))
.build();
org.apache.iceberg.Schema icebergSchema =
IcebergUtils.beamSchemaToIcebergSchema(primitiveTypeSchema);
PartitionSpec spec =
Expand All @@ -465,6 +486,10 @@ public void testIdentityPartitioning() throws IOException {
.identity("float")
.identity("double")
.identity("str")
.identity("date")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for my knowledge, it's unclear to me how testIdentityPartitioning() was passing when identity should support Any source type. Is it worth adding an integration test to ensure just the Write works when partitioning by just date, as well as Any?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test used to not include date and the other time types. This PR adds adds them to the test (your comment is actually positioned on the lines that add them).

.identity("time")
.identity("datetime")
.identity("datetime_tz")
.build();
WindowedValue<IcebergDestination> dest =
getWindowedDestination("identity_partitioning", icebergSchema, spec);
Expand All @@ -479,8 +504,12 @@ public void testIdentityPartitioning() throws IOException {
assertEquals(1, dataFile.getRecordCount());
// build this string: bool=true/int=1/long=1/float=1.0/double=1.0/str=str
List<String> expectedPartitions = new ArrayList<>();
List<String> dateTypes = Arrays.asList("date", "time", "datetime", "datetime_tz");
for (Schema.Field field : primitiveTypeSchema.getFields()) {
Object val = row.getValue(field.getName());
Object val = checkStateNotNull(row.getValue(field.getName()));
if (dateTypes.contains(field.getName())) {
val = URLEncoder.encode(val.toString(), StandardCharsets.UTF_8.toString());
}
expectedPartitions.add(field.getName() + "=" + val);
}
String expectedPartitionPath = String.join("/", expectedPartitions);
Expand Down
Loading