Skip to content
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": 5
"modification": 1
}
3 changes: 2 additions & 1 deletion CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@

* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
* IcebergIO: support declaring a table's sort order on dynamic table creation via the new `sort_fields` config ([#38269](https://github.com/apache/beam/issues/38269)).
* IcebergIO: support writing with hash distribution mode, and with autosharding ([#38061](https://github.com/apache/beam/issues/38061))).

## New Features / Improvements

Expand Down Expand Up @@ -2434,4 +2435,4 @@ Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://iss

## Highlights

- For versions 2.19.0 and older release notes are available on [Apache Beam Blog](https://beam.apache.org/blog/).
- For versions 2.19.0 and older release notes are available on [Apache Beam Blog](https://beam.apache.org/blog/).
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* 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.beam.sdk.io.iceberg;

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import java.util.HashMap;
import java.util.Map;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.RowCoder;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.apache.iceberg.PartitionKey;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.exceptions.NoSuchTableException;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Instant;

/**
* Assigns destination metadata for each input record.
*
* <p>The output will have the format { {destination, partition}, data }
*/
class AssignDestinationsAndPartitions
extends PTransform<PCollection<Row>, PCollection<KV<Row, Row>>> {

private final DynamicDestinations dynamicDestinations;
private final IcebergCatalogConfig catalogConfig;
static final String DESTINATION = "destination";
static final String PARTITION = "partition";
static final org.apache.beam.sdk.schemas.Schema OUTPUT_SCHEMA =
org.apache.beam.sdk.schemas.Schema.builder()
.addStringField(DESTINATION)
.addStringField(PARTITION)
.build();

public AssignDestinationsAndPartitions(
DynamicDestinations dynamicDestinations, IcebergCatalogConfig catalogConfig) {
this.dynamicDestinations = dynamicDestinations;
this.catalogConfig = catalogConfig;
}

@Override
public PCollection<KV<Row, Row>> expand(PCollection<Row> input) {
return input
.apply(ParDo.of(new AssignDoFn(dynamicDestinations, catalogConfig)))
.setCoder(
KvCoder.of(
RowCoder.of(OUTPUT_SCHEMA), RowCoder.of(dynamicDestinations.getDataSchema())));
}

static class AssignDoFn extends DoFn<Row, KV<Row, Row>> {
private transient @MonotonicNonNull Map<String, PartitionKey> partitionKeys;
private transient @MonotonicNonNull Map<String, BeamRowWrapper> wrappers;
private final DynamicDestinations dynamicDestinations;
private final IcebergCatalogConfig catalogConfig;

AssignDoFn(DynamicDestinations dynamicDestinations, IcebergCatalogConfig catalogConfig) {
this.dynamicDestinations = dynamicDestinations;
this.catalogConfig = catalogConfig;
}

@Setup
public void setup() {
this.wrappers = new HashMap<>();
Comment thread
ahmedabu98 marked this conversation as resolved.
this.partitionKeys = new HashMap<>();
}

@ProcessElement
public void processElement(
@Element Row element,
BoundedWindow window,
PaneInfo paneInfo,
@Timestamp Instant timestamp,
OutputReceiver<KV<Row, Row>> out) {
String tableIdentifier =
dynamicDestinations.getTableStringIdentifier(
ValueInSingleWindow.of(element, timestamp, window, paneInfo));
Row data = dynamicDestinations.getData(element);

@Nullable PartitionKey partitionKey = checkStateNotNull(partitionKeys).get(tableIdentifier);
@Nullable BeamRowWrapper wrapper = checkStateNotNull(wrappers).get(tableIdentifier);
if (partitionKey == null || wrapper == null) {
PartitionSpec spec = PartitionSpec.unpartitioned();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Is this guaranteed to be unique across bundles ?

Copy link
Copy Markdown
Contributor Author

@ahmedabu98 ahmedabu98 Apr 30, 2026

Choose a reason for hiding this comment

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

Not always. If they provide partition fields then yes. Otherwise we fetch the spec from the table, which can change between one bundle and another.

Schema schema = IcebergUtils.beamSchemaToIcebergSchema(data.getSchema());
@Nullable
IcebergTableCreateConfig createConfig =
dynamicDestinations.instantiateDestination(tableIdentifier).getTableCreateConfig();
if (createConfig != null && createConfig.getPartitionFields() != null) {
spec =
PartitionUtils.toPartitionSpec(createConfig.getPartitionFields(), data.getSchema());
} else {
try {
// see if table already exists with a spec
// TODO(https://github.com/apache/beam/issues/38337): improve this by periodically
// refreshing the table to fetch updated specs
spec = catalogConfig.catalog().loadTable(TableIdentifier.parse(tableIdentifier)).spec();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

medium

Calling catalog().loadTable() inside processElement can be very expensive, especially in pipelines with many unique table identifiers or high bundle counts. While the results are cached within the DoFn instance's partitionKeys map, the first element for every unique table in every bundle will still trigger a catalog call. Consider using a static cache (similar to WritePartitionedRowsToFiles.LAST_REFRESHED_TABLE_CACHE) to share table metadata across bundles and workers.

} catch (NoSuchTableException ignored) {
// no partition to apply
}
}
partitionKey = new PartitionKey(spec, schema);
wrapper = new BeamRowWrapper(data.getSchema(), schema.asStruct());
checkStateNotNull(partitionKeys).put(tableIdentifier, partitionKey);
checkStateNotNull(wrappers).put(tableIdentifier, wrapper);
}
partitionKey.partition(wrapper.wrap(data));
String partitionPath = partitionKey.toPath();

Row destAndPartition =
Row.withSchema(OUTPUT_SCHEMA).addValues(tableIdentifier, partitionPath).build();
out.output(KV.of(destAndPartition, data));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,180 @@
/*
* 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.beam.sdk.io.iceberg;

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import java.lang.reflect.Array;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.util.concurrent.TimeUnit;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.logicaltypes.Date;
import org.apache.beam.sdk.schemas.logicaltypes.DateTime;
import org.apache.beam.sdk.schemas.logicaltypes.FixedPrecisionNumeric;
import org.apache.beam.sdk.schemas.logicaltypes.MicrosInstant;
import org.apache.beam.sdk.schemas.logicaltypes.Time;
import org.apache.beam.sdk.values.Row;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.UUIDUtil;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
* A wrapper that adapts a Beam {@link Row} to Iceberg's {@link StructLike} interface.
*
* <p>This class allows Beam rows to be processed by Iceberg internal components (like partition
* keys or writers) without requiring a full conversion into Iceberg's internal Record format. It
* handles the mapping between Beam's {@link Schema} and Iceberg's {@link Types.StructType},
* including complex type conversions for timestamps, logical types, and UUIDs.
*
* <p><b>Note:</b> This implementation is <b>read-only</b>. Calls to {@link #set(int, Object)} will
* throw an {@link UnsupportedOperationException}.
*/
public class BeamRowWrapper implements StructLike {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Please add documentation on why this class is needed and how to use it. A BeamRowWrapper sounds like a more general utility that might have to live outside Iceberg ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This class is specifically for Iceberg (helps Iceberg fetch data directly from a Beam Row, instead of having to copy the data over to an Iceberg Record)


private final FieldType[] types;
private final @Nullable PositionalGetter<?>[] getters;
private @Nullable Row row = null;

/** Constructs a new wrapper and pre-computes the mapping between Beam and Iceberg fields. */
public BeamRowWrapper(Schema schema, Types.StructType struct) {
int size = schema.getFieldCount();

types = (FieldType[]) Array.newInstance(FieldType.class, size);
getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size);

for (int i = 0; i < size; i++) {
types[i] = schema.getField(i).getType();
getters[i] = buildGetter(types[i], struct.fields().get(i).type());
}
}

/**
* Sets the current Beam {@link Row} to be wrapped. This method allows the wrapper to be reused
* across different rows to minimize object allocation.
*/
public BeamRowWrapper wrap(@Nullable Row row) {
this.row = row;
return this;
}

@Override
public int size() {
return types.length;
}

/**
* Retrieves a field value from the wrapped row, performing any necessary type conversion to match
* Iceberg's internal expectations (e.g., converting Timestamps to microseconds).
*/
@Override
public <T> @Nullable T get(int pos, Class<T> javaClass) {
if (row == null || row.getValue(pos) == null) {
return null;
} else if (getters[pos] != null) {
return javaClass.cast(getters[pos].get(checkStateNotNull(row), pos));
}

return javaClass.cast(checkStateNotNull(row).getValue(pos));
}

@Override
public <T> void set(int pos, T value) {
throw new UnsupportedOperationException(
"Could not set a field in the BeamRowWrapper because rowData is read-only");
}

private interface PositionalGetter<T> {
T get(Row data, int pos);
}

/**
* Factory method to create a getter that handles type-specific conversion logic.
*
* <p>Handles special cases:
*
* <ul>
* <li>UUID: Converts {@code byte[]} to Iceberg's UUID representation.
* <li>DateTime: Converts Beam {@code DateTime} or logical types to microsecond timestamps.
* <li>Nested Rows: Recursively wraps nested structures in a new {@code BeamRowWrapper}.
* </ul>
*/
private static @Nullable PositionalGetter<?> buildGetter(FieldType beamType, Type icebergType) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think this also came up in a previous PR. Can you clarify why we would need the wrapper class here ? Also is there a way to avoid having to iterate through the set of types in multiple places ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added some java doc

switch (beamType.getTypeName()) {
case BYTE:
return Row::getByte;
case INT16:
return Row::getInt16;
case STRING:
return Row::getString;
case BYTES:
return (row, pos) -> {
byte[] bytes = checkStateNotNull(row.getBytes(pos));
if (Type.TypeID.UUID == icebergType.typeId()) {
return UUIDUtil.convert(bytes);
} else {
return ByteBuffer.wrap(bytes);
}
};
case DECIMAL:
return Row::getDecimal;
case DATETIME:
return (row, pos) ->
TimeUnit.MILLISECONDS.toMicros(checkStateNotNull(row.getDateTime(pos)).getMillis());
case ROW:
Schema beamSchema = checkStateNotNull(beamType.getRowSchema());
Types.StructType structType = (Types.StructType) icebergType;

BeamRowWrapper nestedWrapper = new BeamRowWrapper(beamSchema, structType);
return (row, pos) -> nestedWrapper.wrap(row.getRow(pos));
case LOGICAL_TYPE:
if (beamType.isLogicalType(MicrosInstant.IDENTIFIER)) {
return (row, pos) -> {
Instant instant = checkStateNotNull(row.getLogicalTypeValue(pos, Instant.class));
return TimeUnit.SECONDS.toMicros(instant.getEpochSecond()) + instant.getNano() / 1000;
};
} else if (beamType.isLogicalType(DateTime.IDENTIFIER)) {
return (row, pos) ->
DateTimeUtil.microsFromTimestamp(
checkStateNotNull(row.getLogicalTypeValue(pos, LocalDateTime.class)));
} else if (beamType.isLogicalType(Date.IDENTIFIER)) {
return (row, pos) ->
DateTimeUtil.daysFromDate(
checkStateNotNull(row.getLogicalTypeValue(pos, LocalDate.class)));
} else if (beamType.isLogicalType(Time.IDENTIFIER)) {
return (row, pos) ->
DateTimeUtil.microsFromTime(
checkStateNotNull(row.getLogicalTypeValue(pos, LocalTime.class)));
} else if (beamType.isLogicalType(FixedPrecisionNumeric.IDENTIFIER)) {
return (row, pos) -> row.getLogicalTypeValue(pos, BigDecimal.class);
} else {
return null;
}
default:
return null;
}
}
}
Loading
Loading