diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 7ab7bcd9a9c6..b73af5e61a43 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 2 + "modification": 1 } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 45ecc7cf71c3..30b779cabaee 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java @@ -19,21 +19,26 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets.newHashSet; import com.google.auto.value.AutoValue; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Set; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.sdk.io.iceberg.cdc.IcebergCdcMetadataColumns; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -143,13 +148,13 @@ public org.apache.iceberg.Schema getRequiredSchema() { @Pure @Nullable - public Evaluator getEvaluator() { + public Evaluator getEvaluator(org.apache.iceberg.Schema requiredSchema) { @Nullable Expression filter = getFilter(); if (filter == null) { return null; } if (cachedEvaluator == null) { - cachedEvaluator = new Evaluator(getRequiredSchema().asStruct(), filter); + cachedEvaluator = new Evaluator(requiredSchema.asStruct(), filter); } return cachedEvaluator; } @@ -226,6 +231,9 @@ public Expression getFilter() { @Pure public abstract @Nullable List getDropFields(); + @Pure + public abstract List getMetadataColumns(); + @Pure public static Builder builder() { return new AutoValue_IcebergScanConfig.Builder() @@ -248,7 +256,8 @@ public static Builder builder() { .setPollInterval(null) .setStartingStrategy(null) .setTag(null) - .setBranch(null); + .setBranch(null) + .setMetadataColumns(ImmutableList.of()); } @AutoValue.Builder @@ -311,6 +320,8 @@ public Builder setTableIdentifier(String... names) { public abstract Builder setDropFields(@Nullable List fields); + public abstract Builder setMetadataColumns(List metadataColumns); + public abstract IcebergScanConfig build(); } @@ -364,6 +375,9 @@ void validate(Table table) { if (getStartingStrategy() != null) { invalidOptions.add("starting_strategy"); } + if (!getMetadataColumns().isEmpty()) { + invalidOptions.add("metadata_columns"); + } if (!invalidOptions.isEmpty()) { throw new IllegalArgumentException( error( @@ -371,6 +385,19 @@ void validate(Table table) { + "reading with Managed.ICEBERG_CDC: " + invalidOptions)); } + } else { + Set primaryKeyIds = new HashSet<>(table.schema().identifierFieldIds()); + checkState( + !primaryKeyIds.isEmpty(), + "Cannot read CDC records as the table schema does not specified any primary key fields."); + Set projectedFieldIds = TypeUtil.getProjectedIds(getProjectedSchema()); + primaryKeyIds.removeAll(projectedFieldIds); + checkArgument( + primaryKeyIds.isEmpty(), + "When reading CDC records, the projected schema must not drop primary key fields. " + + "The specified configuration drops the following PK fields: %s", + primaryKeyIds); + validateMetadataColumns(table); } if (getStartingStrategy() != null) { @@ -393,6 +420,47 @@ void validate(Table table) { } } + private void validateMetadataColumns(Table table) { + List metadataColumns = getMetadataColumns(); + if (metadataColumns.isEmpty()) { + return; + } + + Set uniqueMetadataColumns = new LinkedHashSet<>(metadataColumns); + checkArgument( + uniqueMetadataColumns.size() == metadataColumns.size(), + error("metadata_columns contains duplicate entries: %s"), + metadataColumns); + + List unsupportedMetadataColumns = new ArrayList<>(); + for (String metadataColumn : metadataColumns) { + if (!IcebergCdcMetadataColumns.isSupportedColumn(metadataColumn)) { + unsupportedMetadataColumns.add(metadataColumn); + } + } + checkArgument( + unsupportedMetadataColumns.isEmpty(), + error("unsupported metadata_columns: %s. Supported values are: %s"), + unsupportedMetadataColumns, + IcebergCdcMetadataColumns.SUPPORTED_COLUMNS); + + for (String metadataColumn : metadataColumns) { + checkArgument( + getProjectedSchema().findField(metadataColumn) == null, + error("metadata column '%s' conflicts with a projected data column"), + metadataColumn); + } + + boolean includesRowLineage = + metadataColumns.stream().anyMatch(IcebergCdcMetadataColumns::isRowMetadataColumn); + if (includesRowLineage) { + checkArgument( + TableUtil.formatVersion(table) >= 3, + error("row lineage metadata columns %s are only available for Iceberg format v3+ tables"), + metadataColumns); + } + } + private String error(String message) { return "Invalid source configuration: " + message; } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PartitionUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PartitionUtils.java index 805cc0672940..32a25439d850 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PartitionUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/PartitionUtils.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.iceberg.data.IdentityPartitionConverters.convertConstant; import java.util.List; import java.util.Map; @@ -25,11 +26,20 @@ import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Term; +import org.apache.iceberg.types.Types; import org.checkerframework.checker.nullness.qual.Nullable; class PartitionUtils { @@ -48,23 +58,28 @@ class PartitionUtils { Pattern, BiFunction> TRANSFORMATIONS = ImmutableMap.of( - HOUR, (builder, matcher) -> builder.hour(checkStateNotNull(matcher.group(1))), - DAY, (builder, matcher) -> builder.day(checkStateNotNull(matcher.group(1))), - MONTH, (builder, matcher) -> builder.month(checkStateNotNull(matcher.group(1))), - YEAR, (builder, matcher) -> builder.year(checkStateNotNull(matcher.group(1))), + HOUR, + (builder, matcher) -> builder.hour(checkStateNotNull(matcher.group(1))), + DAY, + (builder, matcher) -> builder.day(checkStateNotNull(matcher.group(1))), + MONTH, + (builder, matcher) -> builder.month(checkStateNotNull(matcher.group(1))), + YEAR, + (builder, matcher) -> builder.year(checkStateNotNull(matcher.group(1))), TRUNCATE, - (builder, matcher) -> - builder.truncate( - checkStateNotNull(matcher.group(1)), - Integer.parseInt(checkStateNotNull(matcher.group(2)))), + (builder, matcher) -> + builder.truncate( + checkStateNotNull(matcher.group(1)), + Integer.parseInt(checkStateNotNull(matcher.group(2)))), BUCKET, - (builder, matcher) -> - builder.bucket( - checkStateNotNull(matcher.group(1)), - Integer.parseInt(checkStateNotNull(matcher.group(2)))), - VOID, (builder, matcher) -> builder.alwaysNull(checkStateNotNull(matcher.group(1))), + (builder, matcher) -> + builder.bucket( + checkStateNotNull(matcher.group(1)), + Integer.parseInt(checkStateNotNull(matcher.group(2)))), + VOID, + (builder, matcher) -> builder.alwaysNull(checkStateNotNull(matcher.group(1))), IDENTITY, - (builder, matcher) -> builder.identity(checkStateNotNull(matcher.group(1)))); + (builder, matcher) -> builder.identity(checkStateNotNull(matcher.group(1)))); static PartitionSpec toPartitionSpec( @Nullable List fields, org.apache.beam.sdk.schemas.Schema beamSchema) { @@ -130,4 +145,61 @@ static Term toIcebergTerm(String field) { throw new IllegalArgumentException("Could not find a partition term for '" + field + "'."); } + + /** + * Copied over from Apache Iceberg's PartitionUtil. + * + *

Needed to accommodate CDC reads, where scans produce {@link ChangelogScanTask}s instead of + * {@link ContentScanTask}s. + */ + public static Map constantsMap( + PartitionSpec spec, ContentFile file, @Nullable Long fileSequenceNumber) { + Preconditions.checkState( + spec.specId() == file.specId(), + "File spec ID (%s) does not match PartitionSpec ID (%s)", + file.specId(), + spec.specId()); + StructLike partitionData = file.partition(); + + // use java.util.HashMap because partition data may contain null values + Map idToConstant = Maps.newHashMap(); + + // add first_row_id as _row_id + if (file.firstRowId() != null) { + idToConstant.put( + MetadataColumns.ROW_ID.fieldId(), + convertConstant(Types.LongType.get(), file.firstRowId())); + } + + // When reconstructing a DataFile, we lose the ability to attach its fileSequenceNumber, + // so we pipe it along the util methods to include it here. + fileSequenceNumber = + fileSequenceNumber != null ? fileSequenceNumber : file.fileSequenceNumber(); + idToConstant.put( + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(), + convertConstant(Types.LongType.get(), fileSequenceNumber)); + + // add _file + idToConstant.put( + MetadataColumns.FILE_PATH.fieldId(), + convertConstant(Types.StringType.get(), file.location())); + + // add _spec_id + idToConstant.put( + MetadataColumns.SPEC_ID.fieldId(), convertConstant(Types.IntegerType.get(), file.specId())); + + List partitionFields = spec.partitionType().fields(); + List fields = spec.fields(); + for (int pos = 0; pos < fields.size(); pos += 1) { + PartitionField field = fields.get(pos); + if (field.transform().isIdentity()) { + Object converted = + convertConstant(partitionFields.get(pos).type(), partitionData.get(pos, Object.class)); + idToConstant.put(field.sourceId(), converted); + } + } + + return idToConstant; + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 528b89c203bf..fea62356e431 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -75,9 +75,7 @@ public void process( } FileScanTask task = fileScanTasks.get((int) l); Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); - try (CloseableIterable fullIterable = - ReadUtils.createReader(task, table, scanConfig.getRequiredSchema())) { - CloseableIterable reader = ReadUtils.maybeApplyFilter(fullIterable, scanConfig); + try (CloseableIterable reader = ReadUtils.createReader(task, table, scanConfig)) { for (Record record : reader) { Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java index e7f50882f433..ea2dc7589c4e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java @@ -20,25 +20,22 @@ import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.function.BiFunction; import java.util.stream.Collectors; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.parquet.GenericParquetReaders; @@ -46,17 +43,12 @@ import org.apache.iceberg.encryption.EncryptedInputFile; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.parquet.ParquetReader; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; @@ -73,13 +65,34 @@ public class ReadUtils { "parquet.read.support.class", "parquet.crypto.factory.class"); - static ParquetReader createReader(FileScanTask task, Table table, Schema schema) { - String filePath = task.file().path().toString(); + public static CloseableIterable createReader( + ContentScanTask task, Table table, IcebergScanConfig scanConfig) { + return createReader( + table, + scanConfig, + scanConfig.getRequiredSchema(), + task.spec(), + task.file(), + null, + task.start(), + task.length(), + task.residual()); + } + + public static CloseableIterable createReader( + Table table, + IcebergScanConfig scanConfig, + Schema requiredSchema, + PartitionSpec spec, + ContentFile file, + @Nullable Long fileSequenceNumber, + long start, + long length, + Expression residual) { EncryptedInputFile encryptedInput = - EncryptedFiles.encryptedInput(table.io().newInputFile(filePath), task.file().keyMetadata()); + EncryptedFiles.encryptedInput(table.io().newInputFile(file.location()), file.keyMetadata()); InputFile inputFile = table.encryption().decrypt(encryptedInput); - Map idToConstants = - ReadUtils.constantsMap(task, IdentityPartitionConverters::convertConstant, table.schema()); + Map idToConstants = PartitionUtils.constantsMap(spec, file, fileSequenceNumber); ParquetReadOptions.Builder optionsBuilder; if (inputFile instanceof HadoopInputFile) { @@ -94,71 +107,30 @@ static ParquetReader createReader(FileScanTask task, Table table, Schema } optionsBuilder = optionsBuilder - .withRange(task.start(), task.start() + task.length()) + .withRange(start, start + length) .withMaxAllocationInBytes(MAX_FILE_BUFFER_SIZE); @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping mapping = nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); - return new ParquetReader<>( - inputFile, - schema, - optionsBuilder.build(), - // TODO(ahmedabu98): Implement a Parquet-to-Beam Row reader, bypassing conversion to Iceberg - // Record - fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema, idToConstants), - mapping, - task.residual(), - false, - true); + ParquetReader records = + new ParquetReader<>( + inputFile, + requiredSchema, + optionsBuilder.build(), + // TODO(ahmedabu98): Implement a Parquet-to-Beam Row reader, bypassing conversion to + // Iceberg Record + fileSchema -> + GenericParquetReaders.buildReader(requiredSchema, fileSchema, idToConstants), + mapping, + residual, + false, + true); + return maybeApplyFilter(records, scanConfig, requiredSchema); } - static ParquetReader createReader(InputFile inputFile, Schema schema) { - ParquetReadOptions.Builder optionsBuilder; - if (inputFile instanceof HadoopInputFile) { - // remove read properties already set that may conflict with this read - Configuration conf = new Configuration(((HadoopInputFile) inputFile).getConf()); - for (String property : READ_PROPERTIES_TO_REMOVE) { - conf.unset(property); - } - optionsBuilder = HadoopReadOptions.builder(conf); - } else { - optionsBuilder = ParquetReadOptions.builder(); - } - optionsBuilder = - optionsBuilder - .withRange(0, inputFile.getLength()) - .withMaxAllocationInBytes(MAX_FILE_BUFFER_SIZE); - - return new ParquetReader<>( - inputFile, - schema, - optionsBuilder.build(), - fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema), - MappingUtil.create(schema), - Expressions.alwaysTrue(), - false, - true); - } - - static Map constantsMap( - FileScanTask task, - BiFunction converter, - org.apache.iceberg.Schema schema) { - PartitionSpec spec = task.spec(); - Set idColumns = spec.identitySourceIds(); - org.apache.iceberg.Schema partitionSchema = TypeUtil.select(schema, idColumns); - boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty(); - - if (projectsIdentityPartitionColumns) { - return PartitionUtil.constantsMap(task, converter); - } else { - return Collections.emptyMap(); - } - } - - static @Nullable Long getFromSnapshotExclusive(Table table, IcebergScanConfig scanConfig) { + public static @Nullable Long getFromSnapshotInclusive(Table table, IcebergScanConfig scanConfig) { @Nullable StartingStrategy startingStrategy = scanConfig.getStartingStrategy(); boolean isStreaming = MoreObjects.firstNonNull(scanConfig.getStreaming(), false); if (startingStrategy == null) { @@ -179,6 +151,13 @@ static ParquetReader createReader(InputFile inputFile, Schema schema) { fromSnapshot = currentSnapshot.snapshotId(); } } + + return fromSnapshot; + } + + public static @Nullable Long getFromSnapshotExclusive(Table table, IcebergScanConfig scanConfig) { + @Nullable Long fromSnapshot = getFromSnapshotInclusive(table, scanConfig); + // incremental append scan can only be configured with an *exclusive* starting snapshot, // so we need to provide this snapshot's parent id. if (fromSnapshot != null) { @@ -189,7 +168,7 @@ static ParquetReader createReader(InputFile inputFile, Schema schema) { return fromSnapshot; } - static @Nullable Long getToSnapshot(Table table, IcebergScanConfig scanConfig) { + public static @Nullable Long getToSnapshot(Table table, IcebergScanConfig scanConfig) { // 1. fetch from to_snapshot @Nullable Long toSnapshot = scanConfig.getToSnapshot(); // 2. fetch from to_timestamp @@ -205,7 +184,7 @@ static ParquetReader createReader(InputFile inputFile, Schema schema) { * Returns a list of snapshots in the range (fromSnapshotId, toSnapshotId], ordered * chronologically. */ - static List snapshotsBetween( + public static List snapshotsBetween( Table table, String tableIdentifier, @Nullable Long fromSnapshotId, long toSnapshotId) { long from = MoreObjects.firstNonNull(fromSnapshotId, -1L); @SuppressWarnings("return") @@ -225,10 +204,14 @@ static List snapshotsBetween( public static CloseableIterable maybeApplyFilter( CloseableIterable iterable, IcebergScanConfig scanConfig) { - InternalRecordWrapper wrapper = - new InternalRecordWrapper(scanConfig.getRequiredSchema().asStruct()); + return maybeApplyFilter(iterable, scanConfig, scanConfig.getRequiredSchema()); + } + + public static CloseableIterable maybeApplyFilter( + CloseableIterable iterable, IcebergScanConfig scanConfig, Schema requiredSchema) { + InternalRecordWrapper wrapper = new InternalRecordWrapper(requiredSchema.asStruct()); Expression filter = scanConfig.getFilter(); - Evaluator evaluator = scanConfig.getEvaluator(); + Evaluator evaluator = scanConfig.getEvaluator(requiredSchema); if (filter != null && evaluator != null && filter.op() != Expression.Operation.TRUE) { return CloseableIterable.filter(iterable, record -> evaluator.eval(wrapper.wrap(record))); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java index b3485a7bcc4f..c9ad372a0751 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java @@ -36,7 +36,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.GenericDeleteFilter; -import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.orc.GenericOrcReader; @@ -121,8 +120,7 @@ public boolean advance() throws IOException { DataFile file = fileTask.file(); InputFile input = decryptor.getInputFile(fileTask); Map idToConstants = - ReadUtils.constantsMap( - fileTask, IdentityPartitionConverters::convertConstant, requiredSchema); + PartitionUtils.constantsMap(fileTask.spec(), fileTask.file(), null); CloseableIterable iterable; switch (file.format()) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDataFile.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDataFile.java index 9e75be0a1987..e1291601d149 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDataFile.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDataFile.java @@ -54,13 +54,13 @@ */ @DefaultSchema(AutoValueSchema.class) @AutoValue -abstract class SerializableDataFile { +public abstract class SerializableDataFile { public static Builder builder() { return new AutoValue_SerializableDataFile.Builder(); } @SchemaFieldNumber("0") - abstract String getPath(); + public abstract String getPath(); @SchemaFieldNumber("1") abstract String getFileFormat(); @@ -69,10 +69,10 @@ public static Builder builder() { abstract long getRecordCount(); @SchemaFieldNumber("3") - abstract long getFileSizeInBytes(); + public abstract long getFileSizeInBytes(); @SchemaFieldNumber("4") - abstract String getPartitionPath(); + public abstract String getPartitionPath(); @SchemaFieldNumber("5") abstract int getPartitionSpecId(); @@ -96,13 +96,22 @@ public static Builder builder() { abstract @Nullable Map getNanValueCounts(); @SchemaFieldNumber("12") - abstract @Nullable Map getLowerBounds(); + public abstract @Nullable Map getLowerBounds(); @SchemaFieldNumber("13") - abstract @Nullable Map getUpperBounds(); + public abstract @Nullable Map getUpperBounds(); + + @SchemaFieldNumber("14") + public abstract @Nullable Long getDataSequenceNumber(); + + @SchemaFieldNumber("15") + public abstract @Nullable Long getFileSequenceNumber(); + + @SchemaFieldNumber("16") + public abstract @Nullable Long getFirstRowId(); @AutoValue.Builder - abstract static class Builder { + public abstract static class Builder { abstract Builder setPath(String path); abstract Builder setFileFormat(String fileFormat); @@ -131,31 +140,49 @@ abstract static class Builder { abstract Builder setUpperBounds(@Nullable Map upperBounds); + abstract Builder setDataSequenceNumber(@Nullable Long number); + + abstract Builder setFileSequenceNumber(@Nullable Long number); + + abstract Builder setFirstRowId(@Nullable Long id); + abstract SerializableDataFile build(); } + public static SerializableDataFile from(DataFile f, String partitionPath) { + return from(f, partitionPath, true); + } + /** * Create a {@link SerializableDataFile} from a {@link DataFile} and its associated {@link * PartitionKey}. */ - static SerializableDataFile from(DataFile f, String partitionPath) { - - return SerializableDataFile.builder() - .setPath(f.location().toString()) - .setFileFormat(f.format().toString()) - .setRecordCount(f.recordCount()) - .setFileSizeInBytes(f.fileSizeInBytes()) - .setPartitionPath(partitionPath) - .setPartitionSpecId(f.specId()) - .setKeyMetadata(f.keyMetadata()) - .setSplitOffsets(f.splitOffsets()) - .setColumnSizes(f.columnSizes()) - .setValueCounts(f.valueCounts()) - .setNullValueCounts(f.nullValueCounts()) - .setNanValueCounts(f.nanValueCounts()) - .setLowerBounds(toByteArrayMap(f.lowerBounds())) - .setUpperBounds(toByteArrayMap(f.upperBounds())) - .build(); + public static SerializableDataFile from( + DataFile f, String partitionPath, boolean includeMetrics) { + SerializableDataFile.Builder builder = + SerializableDataFile.builder() + .setPath(f.location()) + .setFileFormat(f.format().toString()) + .setRecordCount(f.recordCount()) + .setFileSizeInBytes(f.fileSizeInBytes()) + .setPartitionPath(partitionPath) + .setPartitionSpecId(f.specId()) + .setKeyMetadata(f.keyMetadata()) + .setSplitOffsets(f.splitOffsets()) + .setColumnSizes(f.columnSizes()) + .setValueCounts(f.valueCounts()) + .setNullValueCounts(f.nullValueCounts()) + .setNanValueCounts(f.nanValueCounts()) + .setDataSequenceNumber(f.dataSequenceNumber()) + .setFileSequenceNumber(f.fileSequenceNumber()) + .setFirstRowId(f.firstRowId()); + if (includeMetrics) { + builder = + builder + .setLowerBounds(toByteArrayMap(f.lowerBounds())) + .setUpperBounds(toByteArrayMap(f.upperBounds())); + } + return builder.build(); } /** @@ -165,7 +192,7 @@ static SerializableDataFile from(DataFile f, String partitionPath) { * it from Beam-compatible types. */ @SuppressWarnings("nullness") - DataFile createDataFile(Map partitionSpecs) { + public DataFile createDataFile(Map partitionSpecs) { PartitionSpec partitionSpec = checkStateNotNull( partitionSpecs.get(getPartitionSpecId()), @@ -192,14 +219,14 @@ DataFile createDataFile(Map partitionSpecs) { .withFileSizeInBytes(getFileSizeInBytes()) .withMetrics(dataFileMetrics) .withSplitOffsets(getSplitOffsets()) + .withFirstRowId(getFirstRowId()) .build(); } // ByteBuddyUtils has trouble converting Map value type ByteBuffer // to byte[] and back to ByteBuffer, so we perform these conversions manually // TODO(https://github.com/apache/beam/issues/32701) - private static @Nullable Map toByteArrayMap( - @Nullable Map input) { + static @Nullable Map toByteArrayMap(@Nullable Map input) { if (input == null) { return null; } @@ -222,8 +249,7 @@ private static byte[] toByteArray(ByteBuffer buf) { return bytes; } - private static @Nullable Map toByteBufferMap( - @Nullable Map input) { + static @Nullable Map toByteBufferMap(@Nullable Map input) { if (input == null) { return null; } @@ -256,10 +282,13 @@ && getPartitionSpecId() == that.getPartitionSpecId() && Objects.equals(getNullValueCounts(), that.getNullValueCounts()) && Objects.equals(getNanValueCounts(), that.getNanValueCounts()) && mapEquals(getLowerBounds(), that.getLowerBounds()) - && mapEquals(getUpperBounds(), that.getUpperBounds()); + && mapEquals(getUpperBounds(), that.getUpperBounds()) + && Objects.equals(getDataSequenceNumber(), that.getDataSequenceNumber()) + && Objects.equals(getFileSequenceNumber(), that.getFileSequenceNumber()) + && Objects.equals(getFirstRowId(), that.getFirstRowId()); } - private static boolean mapEquals( + static boolean mapEquals( @Nullable Map map1, @Nullable Map map2) { if (map1 == null && map2 == null) { return true; @@ -297,13 +326,16 @@ public final int hashCode() { getColumnSizes(), getValueCounts(), getNullValueCounts(), - getNanValueCounts()); + getNanValueCounts(), + getDataSequenceNumber(), + getFileSequenceNumber(), + getFirstRowId()); hashCode = 31 * hashCode + computeMapByteHashCode(getLowerBounds()); hashCode = 31 * hashCode + computeMapByteHashCode(getUpperBounds()); return hashCode; } - private static int computeMapByteHashCode(@Nullable Map map) { + static int computeMapByteHashCode(@Nullable Map map) { if (map == null) { return 0; } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFile.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFile.java new file mode 100644 index 000000000000..ceb96d50f8aa --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFile.java @@ -0,0 +1,335 @@ +/* + * 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.io.iceberg.SerializableDataFile.computeMapByteHashCode; +import static org.apache.beam.sdk.io.iceberg.SerializableDataFile.mapEquals; +import static org.apache.beam.sdk.io.iceberg.SerializableDataFile.toByteArrayMap; +import static org.apache.beam.sdk.io.iceberg.SerializableDataFile.toByteBufferMap; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import com.google.auto.value.AutoValue; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldNumber; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SortOrder; +import org.checkerframework.checker.nullness.qual.Nullable; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SerializableDeleteFile { + public static SerializableDeleteFile.Builder builder() { + return new AutoValue_SerializableDeleteFile.Builder(); + } + + @SchemaFieldNumber("0") + public abstract FileContent getContentType(); + + @SchemaFieldNumber("1") + public abstract String getLocation(); + + @SchemaFieldNumber("2") + public abstract String getFileFormat(); + + @SchemaFieldNumber("3") + public abstract long getRecordCount(); + + @SchemaFieldNumber("4") + public abstract long getFileSizeInBytes(); + + @SchemaFieldNumber("5") + public abstract String getPartitionPath(); + + @SchemaFieldNumber("6") + public abstract int getPartitionSpecId(); + + @SchemaFieldNumber("7") + public abstract @Nullable Integer getSortOrderId(); + + @SchemaFieldNumber("8") + public abstract @Nullable List getEqualityFieldIds(); + + @SchemaFieldNumber("9") + public abstract @Nullable ByteBuffer getKeyMetadata(); + + @SchemaFieldNumber("10") + public abstract @Nullable List getSplitOffsets(); + + @SchemaFieldNumber("11") + public abstract @Nullable Map getColumnSizes(); + + @SchemaFieldNumber("12") + public abstract @Nullable Map getValueCounts(); + + @SchemaFieldNumber("13") + public abstract @Nullable Map getNullValueCounts(); + + @SchemaFieldNumber("14") + public abstract @Nullable Map getNanValueCounts(); + + @SchemaFieldNumber("15") + public abstract @Nullable Map getLowerBounds(); + + @SchemaFieldNumber("16") + public abstract @Nullable Map getUpperBounds(); + + @SchemaFieldNumber("17") + public abstract @Nullable Long getContentOffset(); + + @SchemaFieldNumber("18") + public abstract @Nullable Long getContentSizeInBytes(); + + @SchemaFieldNumber("19") + public abstract @Nullable String getReferencedDataFile(); + + @SchemaFieldNumber("20") + public abstract @Nullable Long getDataSequenceNumber(); + + @SchemaFieldNumber("21") + public abstract @Nullable Long getFileSequenceNumber(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setContentType(FileContent content); + + abstract Builder setLocation(String path); + + abstract Builder setFileFormat(String fileFormat); + + abstract Builder setRecordCount(long recordCount); + + abstract Builder setFileSizeInBytes(long fileSizeInBytes); + + abstract Builder setPartitionPath(String partitionPath); + + abstract Builder setPartitionSpecId(int partitionSpec); + + abstract Builder setSortOrderId(@Nullable Integer sortOrderId); + + abstract Builder setEqualityFieldIds(List equalityFieldIds); + + abstract Builder setKeyMetadata(ByteBuffer keyMetadata); + + abstract Builder setSplitOffsets(List splitOffsets); + + abstract Builder setColumnSizes(Map columnSizes); + + abstract Builder setValueCounts(Map valueCounts); + + abstract Builder setNullValueCounts(Map nullValueCounts); + + abstract Builder setNanValueCounts(Map nanValueCounts); + + abstract Builder setLowerBounds(@Nullable Map lowerBounds); + + abstract Builder setUpperBounds(@Nullable Map upperBounds); + + abstract Builder setContentOffset(@Nullable Long offset); + + abstract Builder setContentSizeInBytes(@Nullable Long sizeInBytes); + + abstract Builder setReferencedDataFile(@Nullable String dataFile); + + abstract Builder setDataSequenceNumber(@Nullable Long number); + + abstract Builder setFileSequenceNumber(@Nullable Long number); + + abstract SerializableDeleteFile build(); + } + + public static SerializableDeleteFile from( + DeleteFile deleteFile, String partitionPath, boolean includeMetrics) { + + SerializableDeleteFile.Builder builder = + SerializableDeleteFile.builder() + .setLocation(deleteFile.location()) + .setFileFormat(deleteFile.format().name()) + .setFileSizeInBytes(deleteFile.fileSizeInBytes()) + .setPartitionPath(partitionPath) + .setPartitionSpecId(deleteFile.specId()) + .setRecordCount(deleteFile.recordCount()) + .setColumnSizes(deleteFile.columnSizes()) + .setValueCounts(deleteFile.valueCounts()) + .setNullValueCounts(deleteFile.nullValueCounts()) + .setNanValueCounts(deleteFile.nanValueCounts()) + .setSplitOffsets(deleteFile.splitOffsets()) + .setKeyMetadata(deleteFile.keyMetadata()) + .setEqualityFieldIds(deleteFile.equalityFieldIds()) + .setSortOrderId(deleteFile.sortOrderId()) + .setContentOffset(deleteFile.contentOffset()) + .setContentSizeInBytes(deleteFile.contentSizeInBytes()) + .setReferencedDataFile(deleteFile.referencedDataFile()) + .setContentType(deleteFile.content()) + .setDataSequenceNumber(deleteFile.dataSequenceNumber()) + .setFileSequenceNumber(deleteFile.fileSequenceNumber()); + + if (includeMetrics) { + builder = + builder + .setLowerBounds(toByteArrayMap(deleteFile.lowerBounds())) + .setUpperBounds(toByteArrayMap(deleteFile.upperBounds())); + } + + return builder.build(); + } + + @SuppressWarnings("nullness") + public DeleteFile createDeleteFile( + Map partitionSpecs, @Nullable Map sortOrders) { + PartitionSpec partitionSpec = + checkStateNotNull( + partitionSpecs.get(getPartitionSpecId()), + "This DeleteFile was originally created with spec id '%s', " + + "but table only has spec ids: %s.", + getPartitionSpecId(), + partitionSpecs.keySet()); + + Metrics metrics = + new Metrics( + getRecordCount(), + getColumnSizes(), + getValueCounts(), + getNullValueCounts(), + getNanValueCounts(), + toByteBufferMap(getLowerBounds()), + toByteBufferMap(getUpperBounds())); + + FileMetadata.Builder deleteFileBuilder = + FileMetadata.deleteFileBuilder(partitionSpec) + .withPath(getLocation()) + .withFormat(getFileFormat()) + .withFileSizeInBytes(getFileSizeInBytes()) + .withRecordCount(getRecordCount()) + .withMetrics(metrics) + .withSplitOffsets(getSplitOffsets()) + .withEncryptionKeyMetadata(getKeyMetadata()) + .withPartitionPath(getPartitionPath()); + + switch (getContentType()) { + case POSITION_DELETES: + deleteFileBuilder = deleteFileBuilder.ofPositionDeletes(); + break; + case EQUALITY_DELETES: + List fieldIds = getEqualityFieldIds(); + int[] equalityFieldIds = new int[fieldIds != null ? fieldIds.size() : 0]; + if (fieldIds != null) { + for (int i = 0; i < fieldIds.size(); i++) { + equalityFieldIds[i] = fieldIds.get(i); + } + } + SortOrder sortOrder = SortOrder.unsorted(); + if (sortOrders != null) { + sortOrder = + checkStateNotNull( + sortOrders.get(getSortOrderId()), + "This DeleteFile was originally created with sort order id '%s', " + + "but table only has sort order ids: %s.", + getSortOrderId(), + sortOrders.keySet()); + } + deleteFileBuilder = + deleteFileBuilder.ofEqualityDeletes(equalityFieldIds).withSortOrder(sortOrder); + break; + default: + throw new IllegalStateException( + "Unexpected content type for DeleteFile: " + getContentType()); + } + + // needed for puffin files + if (getFileFormat().equalsIgnoreCase(FileFormat.PUFFIN.name())) { + deleteFileBuilder = + deleteFileBuilder + .withContentOffset(checkStateNotNull(getContentOffset())) + .withContentSizeInBytes(checkStateNotNull(getContentSizeInBytes())) + .withReferencedDataFile(checkStateNotNull(getReferencedDataFile())); + } + return deleteFileBuilder.build(); + } + + @Override + public final boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SerializableDeleteFile)) { + return false; + } + SerializableDeleteFile that = (SerializableDeleteFile) o; + return getContentType().equals(that.getContentType()) + && getLocation().equals(that.getLocation()) + && getFileFormat().equals(that.getFileFormat()) + && getRecordCount() == that.getRecordCount() + && getFileSizeInBytes() == that.getFileSizeInBytes() + && getPartitionPath().equals(that.getPartitionPath()) + && getPartitionSpecId() == that.getPartitionSpecId() + && Objects.equals(getSortOrderId(), that.getSortOrderId()) + && Objects.equals(getEqualityFieldIds(), that.getEqualityFieldIds()) + && Objects.equals(getKeyMetadata(), that.getKeyMetadata()) + && Objects.equals(getSplitOffsets(), that.getSplitOffsets()) + && Objects.equals(getColumnSizes(), that.getColumnSizes()) + && Objects.equals(getValueCounts(), that.getValueCounts()) + && Objects.equals(getNullValueCounts(), that.getNullValueCounts()) + && Objects.equals(getNanValueCounts(), that.getNanValueCounts()) + && mapEquals(getLowerBounds(), that.getLowerBounds()) + && mapEquals(getUpperBounds(), that.getUpperBounds()) + && Objects.equals(getContentOffset(), that.getContentOffset()) + && Objects.equals(getContentSizeInBytes(), that.getContentSizeInBytes()) + && Objects.equals(getReferencedDataFile(), that.getReferencedDataFile()) + && Objects.equals(getDataSequenceNumber(), that.getDataSequenceNumber()) + && Objects.equals(getFileSequenceNumber(), that.getFileSequenceNumber()); + } + + @Override + public final int hashCode() { + int hashCode = + Objects.hash( + getContentType(), + getLocation(), + getFileFormat(), + getRecordCount(), + getFileSizeInBytes(), + getPartitionPath(), + getPartitionSpecId(), + getSortOrderId(), + getEqualityFieldIds(), + getKeyMetadata(), + getSplitOffsets(), + getColumnSizes(), + getValueCounts(), + getNullValueCounts(), + getNanValueCounts(), + getContentOffset(), + getContentSizeInBytes(), + getReferencedDataFile(), + getDataSequenceNumber(), + getFileSequenceNumber()); + hashCode = 31 * hashCode + computeMapByteHashCode(getLowerBounds()); + hashCode = 31 * hashCode + computeMapByteHashCode(getUpperBounds()); + return hashCode; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcOutputUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcOutputUtils.java new file mode 100644 index 000000000000..147e2adda1a7 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcOutputUtils.java @@ -0,0 +1,178 @@ +/* + * 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.cdc; + +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.ValueKind; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.types.Types; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Helpers for CDC schemas and output row construction. + * + *

CDC metadata is handled in two phases. Row metadata, such as {@code _row_id} and {@code + * _last_updated_sequence_number}, is added to intermediate read schemas so Iceberg readers can + * populate those values. Commit metadata, such as {@code _commit_snapshot_id} and {@code + * _commit_snapshot_sequence_number}, is carried separately in CDC descriptors. The {@code + * _change_type} metadata column comes from the resolved Beam output {@link ValueKind}. + * + *

The public output shape is assembled only when final Beam {@link Row}s are emitted. This keeps + * the read path table-shaped while still exposing all requested metadata as top-level output + * fields. + */ +final class CdcOutputUtils { + /** + * Returns the public CDC output schema: projected data fields followed by requested metadata + * columns in user-configured order. + */ + static Schema outputSchema(IcebergScanConfig scanConfig, Schema dataSchema) { + if (scanConfig.getMetadataColumns().isEmpty()) { + return dataSchema; + } + + Schema.Builder builder = Schema.builder().addFields(dataSchema.getFields()); + for (String metadataColumn : scanConfig.getMetadataColumns()) { + builder.addField(IcebergCdcMetadataColumns.beamField(metadataColumn)); + } + return builder.build(); + } + + /** + * Returns an Iceberg read schema that includes row metadata columns. + * + *

Commit metadata columns are not added here because Iceberg readers cannot populate them; + * those values are taken from {@link ChangelogDescriptor} or {@link CdcRowDescriptor} when output + * rows are built. + */ + static org.apache.iceberg.Schema readSchemaWithRowMetadata( + List metadataColumns, org.apache.iceberg.Schema dataSchema) { + List fields = new ArrayList<>(dataSchema.columns()); + for (String metadataColumn : metadataColumns) { + Types.NestedField rowMetadataField = + IcebergCdcMetadataColumns.icebergRowMetadataField(metadataColumn); + if (rowMetadataField != null && dataSchema.findField(rowMetadataField.fieldId()) == null) { + fields.add(rowMetadataField); + } + } + return new org.apache.iceberg.Schema(fields, dataSchema.identifierFieldIds()); + } + + /** + * Beam-schema equivalent of {@link #readSchemaWithRowMetadata(List, org.apache.iceberg.Schema)}. + */ + static Schema readBeamSchemaWithRowMetadata(List metadataColumns, Schema dataSchema) { + if (metadataColumns.stream().noneMatch(IcebergCdcMetadataColumns::isRowMetadataColumn)) { + return dataSchema; + } + + Schema.Builder builder = Schema.builder().addFields(dataSchema.getFields()); + for (String metadataColumn : metadataColumns) { + if (IcebergCdcMetadataColumns.isRowMetadataColumn(metadataColumn) + && !dataSchema.hasField(metadataColumn)) { + builder.addField(IcebergCdcMetadataColumns.beamField(metadataColumn)); + } + } + return builder.build(); + } + + /** + * Builds the final public Beam row. + * + *

{@code dataAndRowMetadata} may already include row metadata read from Iceberg. This method + * copies only data fields first, then appends every requested metadata column at the top level. + * That preserves configured column order and avoids exposing row metadata twice. + */ + static Row outputRow( + List metadataColumns, + Schema outputSchema, + long commitSnapshotId, + long snapshotSequentNumber, + ValueKind valueKind, + Row dataAndRowMetadata) { + if (metadataColumns.isEmpty() + || metadataColumns.stream().allMatch(IcebergCdcMetadataColumns::isRowMetadataColumn)) { + return dataAndRowMetadata; + } + + List<@Nullable Object> values = new ArrayList<>(outputSchema.getFieldCount()); + for (Schema.Field field : dataAndRowMetadata.getSchema().getFields()) { + if (!metadataColumns.contains(field.getName())) { + values.add(dataAndRowMetadata.getValue(field.getName())); + } + } + + for (String metadataColumn : metadataColumns) { + values.add( + metadataValue( + metadataColumn, + commitSnapshotId, + snapshotSequentNumber, + valueKind, + dataAndRowMetadata)); + } + return Row.withSchema(outputSchema).addValues(values).build(); + } + + static Schema readBeamSchemaWithRowMetadata( + List metadataColumns, org.apache.iceberg.Schema dataSchema) { + return IcebergUtils.icebergSchemaToBeamSchema( + readSchemaWithRowMetadata(metadataColumns, dataSchema)); + } + + private static @Nullable Object metadataValue( + String metadataColumn, + long commitSnapshotId, + long commitSnapshotSequenceNumber, + ValueKind valueKind, + Row dataAndRowMetadata) { + if (IcebergCdcMetadataColumns.CHANGE_TYPE.equals(metadataColumn)) { + return changelogOperation(valueKind).name(); + } + if (IcebergCdcMetadataColumns.COMMIT_SNAPSHOT_ID.equals(metadataColumn)) { + return commitSnapshotId; + } + if (IcebergCdcMetadataColumns.COMMIT_SNAPSHOT_SEQUENCE_NUMBER.equals(metadataColumn)) { + return commitSnapshotSequenceNumber; + } + if (dataAndRowMetadata.getSchema().hasField(metadataColumn)) { + return dataAndRowMetadata.getValue(metadataColumn); + } + return null; + } + + private static ChangelogOperation changelogOperation(ValueKind valueKind) { + switch (valueKind) { + case INSERT: + return ChangelogOperation.INSERT; + case DELETE: + return ChangelogOperation.DELETE; + case UPDATE_BEFORE: + return ChangelogOperation.UPDATE_BEFORE; + case UPDATE_AFTER: + return ChangelogOperation.UPDATE_AFTER; + default: + throw new IllegalArgumentException("Unsupported CDC ValueKind: " + valueKind); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtils.java new file mode 100644 index 000000000000..fd0fe5399f7a --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtils.java @@ -0,0 +1,698 @@ +/* + * 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.cdc; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.ReadUtils; +import org.apache.beam.sdk.io.iceberg.SerializableDeleteFile; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.parquet.ParquetMetricsRowGroupFilter; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.io.DelegatingSeekableInputStream; +import org.apache.parquet.schema.MessageType; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Read-side helpers specific to the CDC source. Keeps {@link ReadUtils} focused on the + * general-purpose append-only read path; everything that takes a {@link SerializableChangelogTask}, + * references {@link DeleteReader}, or implements the delete-pushdown row-group skipping lives here. + */ +public final class CdcReadUtils { + private static final Logger LOG = LoggerFactory.getLogger(CdcReadUtils.class); + + /** + * Maximum size of an equality delete set to push down as a Parquet residual {@code IN} + * expression. Matches {@link ParquetMetricsRowGroupFilter#IN_PREDICATE_LIMIT}. + */ + private static final int IN_PREDICATE_LIMIT = 200; + + public static CloseableIterable createReader( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + Schema outputSchema) { + return createReader(task, table, scanConfig, outputSchema, Expressions.alwaysTrue()); + } + + /** + * Same as {@link #createReader(SerializableChangelogTask, Table, IcebergScanConfig, Schema)} but + * ANDs {@code extraResidual} into the task's residual expression. The combined expression is + * passed to Iceberg's Parquet reader, which uses it as a row-group-level filter (skips row groups + * whose column statistics cannot match). The caller is still responsible for applying the + * residual at the row level. + * + *

This is used to push extra predicates (e.g. an equality-delete {@code IN} expression) down + * to the reader for cheap row-group skipping. + */ + public static CloseableIterable createReader( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + Schema outputSchema, + Expression extraResidual) { + return createReader( + task, table, scanConfig, outputSchema, extraResidual, task.getStart(), task.getLength()); + } + + /** + * Same as {@link #createReader(SerializableChangelogTask, Table, IcebergScanConfig, Schema, + * Expression)} but reads the byte range {@code [start, start + length)} of the DataFile. + * Iceberg's Parquet reader selects the row groups whose starting offset falls within this range, + * allowing us to prune row-groups by byte-range. + * + *

Callers are responsible for ensuring the requested range stays within the task's assigned + * range, to avoid reading a section that is meant for another worker. + */ + public static CloseableIterable createReader( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + Schema outputSchema, + Expression extraResidual, + long start, + long length) { + Expression baseResidual = task.getExpression(table.schema()); + Expression combined = + extraResidual.op() == Expression.Operation.TRUE + ? baseResidual + : Expressions.and(baseResidual, extraResidual); + return ReadUtils.createReader( + table, + scanConfig, + outputSchema, + checkStateNotNull(table.specs().get(task.getSpecId())), + task.getDataFile().createDataFile(table.specs()), + task.getDataFile().getFileSequenceNumber(), + start, + length, + combined); + } + + /** Returns a filter that skips records marked for deletion. */ + public static DeleteFilter genericDeleteFilter( + Table table, Schema outputSchema, String dataFilePath, List deletes) { + return new GenericDeleteFilter( + table.io(), + dataFilePath, + table.schema(), + outputSchema, + deletes.stream() + .map(sdf -> sdf.createDeleteFile(table.specs(), table.sortOrders())) + .collect(Collectors.toList())); + } + + /** Returns a delete reader that reuses delete structures already loaded by CDC planning. */ + public static DeleteReader genericDeleteReader( + Table table, + Schema outputSchema, + String dataFilePath, + List deletes, + DeleteReader.PreloadedDeletes preloadedDeletes) { + return new GenericDeleteReader( + table.io(), + dataFilePath, + table.schema(), + outputSchema, + deletes.stream() + .map(sdf -> sdf.createDeleteFile(table.specs(), table.sortOrders())) + .collect(Collectors.toList()), + preloadedDeletes); + } + + /** + * Opens the records that a CDC reader should process for a single {@link + * SerializableChangelogTask}, applying the appropriate delete-filter / delete-reader chain for + * the task's type: + * + *

    + *
  • {@code ADDED_ROWS}: Collect and return the records that became live in this commit: + *
      + *
    • 1. Iterate over records in the added DataFile + *
    • 2. Filter out records matched by any added deletes + *
    + *
  • {@code DELETED_ROWS}: Return records in the DataFile that are marked for deletion by new + * DeleteFiles, making sure to first ignore records that have already been marked by + * previous DeleteFiles: + *
      + *
    • 1. Iterate over records in the referenced DataFile + *
    • 2. Filter out records matched from existing deletes. + *
    • 3. Filter out records NOT matched from added deletes + *
    + *
  • {@code DELETED_FILE} — every record in the DataFile that wasn't already deleted by {@code + * existingDeletes}. + *
      + *
    • 1. Iterate over records in the referenced DataFile + *
    • 2. Filter out records matched from existing deletes. + *
    + *
+ * + *

Projection pushdown should not be used when reading bi-directional tasks because we need to + * compare all record columns to accurately identify updates. Otherwise, user-configured + * projection may drop a column that contains real updates. If this happens, the downstream + * resolver will mistakenly determine the (delete, insert) pair to be a duplicate. + * + *

If CDC metadata columns are requested, this method only adds row-sourced metadata columns + * ({@code _row_id}, {@code _last_updated_sequence_number}) to the Iceberg read schema. Changelog + * context columns are added later by {@link CdcOutputUtils#outputRow}. + */ + public static CloseableIterable changelogRecordsForTask( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + boolean useProjectedSchema) { + String dataFilePath = task.getDataFile().getPath(); + Schema outputSchema = + CdcOutputUtils.readSchemaWithRowMetadata( + scanConfig.getMetadataColumns(), + useProjectedSchema ? scanConfig.getRequiredSchema() : table.schema()); + switch (task.getType()) { + case ADDED_ROWS: + DeleteFilter addedDeletesFilter = + genericDeleteFilter(table, outputSchema, dataFilePath, task.getAddedDeletes()); + return addedDeletesFilter.filter( + createReader(task, table, scanConfig, addedDeletesFilter.requiredSchema())); + case DELETED_FILE: + DeleteFilter existingDeletesFilter = + genericDeleteFilter(table, outputSchema, dataFilePath, task.getExistingDeletes()); + return existingDeletesFilter.filter( + createReader(task, table, scanConfig, existingDeletesFilter.requiredSchema())); + case DELETED_ROWS: + return deletedRowsForTask(task, table, scanConfig, outputSchema); + default: + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getType()); + } + } + + /** + * Builds the reader chain for a {@code DELETED_ROWS} task with row-group pushdown when possible. + * This helps the reader skip entire row groups. For unskipped row groups, the reader should still + * apply per-record position + equality checks at the row level. + * + *

We use two pushdown strategies, depending on the type of {@link DeleteFile} in the task + * (Position Delete vs. Equality Delete). The two strategies can be combined if both {@link + * DeleteFile} types are present. + * + *

    + *
  1. Byte-range pushdown for Position Deletes: pre-load the {@link + * PositionDeleteIndex}, read the Parquet footer, and compute a single contiguous byte range + * covering the row groups that contain at least one deleted position. + *
  2. IN-expression pushdown for Equality Deletes: build an Iceberg {@code IN} + * expression and pass it as a Parquet residual so the metrics row-group filter can skip + * non-matching row groups. + *
+ * + *

If Position and Equality deletes are both present, both strategies are used to get one + * contiguous range. We read only that range, skipping leading and trailing row groups that + * contain no deletions. + * + *

Note: Equality pushdown is only used when all delete files share a single equality field. + * Multi-column equality requires an exploded OR expression that Parquet's metrics filter handles + * poorly. + */ + private static CloseableIterable deletedRowsForTask( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + Schema outputSchema) { + String dataFilePath = task.getDataFile().getPath(); + List addedDeletes = task.getAddedDeletes(); + + // Split into position vs equality. + List posFiles = new ArrayList<>(); + List eqFiles = new ArrayList<>(); + for (SerializableDeleteFile sd : addedDeletes) { + DeleteFile df = sd.createDeleteFile(table.specs(), table.sortOrders()); + if (df.content() == FileContent.POSITION_DELETES) { + posFiles.add(df); + } else if (df.content() == FileContent.EQUALITY_DELETES) { + eqFiles.add(df); + } + } + + // Strategy 1: byte-range pushdown around row groups with position deletes (+ eq + // matches). + DeleteReader.PreloadedDeletes preloadedDeletes = DeleteReader.PreloadedDeletes.empty(); + if (!posFiles.isEmpty()) { + @Nullable + PositionPushdownResult pushdown = + tryPositionByteRangePushdown( + task, table, scanConfig, outputSchema, posFiles, eqFiles, addedDeletes); + if (pushdown != null) { + if (pushdown.deletedRecords != null) { + return pushdown.deletedRecords; + } + preloadedDeletes = pushdown.preloadedDeletes; + } + // fall through to the default chain on failure + } + + // Strategy 2: equality IN-expression pushdown applied as a reader residual. + // Only safe when no position deletes are present. when both exist, the + // byte-range path above already incorporates the eq filter + Expression eqResidual = Expressions.alwaysTrue(); + if (posFiles.isEmpty() && !eqFiles.isEmpty()) { + EqualityPushdownResult eqPushdown = buildEqualityDeletePushdown(table, eqFiles); + eqResidual = eqPushdown.applicable ? eqPushdown.residual : Expressions.alwaysTrue(); + preloadedDeletes = eqPushdown.preloadedDeletes(null); + } + + DeleteFilter existingDeletesFilter = + genericDeleteFilter(table, outputSchema, dataFilePath, task.getExistingDeletes()); + DeleteReader addedDeletesReader = + genericDeleteReader(table, outputSchema, dataFilePath, addedDeletes, preloadedDeletes); + Schema requiredSchema = + TypeUtil.join(existingDeletesFilter.requiredSchema(), addedDeletesReader.requiredSchema()); + + CloseableIterable records = + createReader(task, table, scanConfig, requiredSchema, eqResidual); + CloseableIterable liveRecords = existingDeletesFilter.filter(records); + return addedDeletesReader.read(liveRecords); + } + + /** + * Path-A byte-range position-delete pushdown. Returns {@code null} if pushdown isn't applicable + * or any step fails, signaling to the caller to fall back. Returns an empty iterable if every row + * group is pruned. + */ + private static @Nullable PositionPushdownResult tryPositionByteRangePushdown( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + Schema outputSchema, + List posFiles, + List eqFiles, + List addedDeletes) { + String dataFilePath = task.getDataFile().getPath(); + + // 1. pre-load the position index for this data file. + PositionDeleteIndex posIndex; + try { + DeleteLoader loader = new BaseDeleteLoader(df -> table.io().newInputFile(df.location())); + posIndex = loader.loadPositionDeletes(posFiles, dataFilePath); + } catch (RuntimeException e) { + LOG.info( + "Failed to pre-load position deletes for {}; falling back to default reader chain.", + dataFilePath, + e); + return null; + } + if (posIndex.isEmpty()) { + // the pos-delete files don't actually target this data file (rare but possible + // after metadata operations). Fall back so the eq pushdown does not run here either. + return PositionPushdownResult.fallback( + DeleteReader.PreloadedDeletes.of(posIndex, Collections.emptyMap())); + } + + // 2. optional equality filter (used to extend the byte range to include row groups + // whose stats match the equality IN values). + @Nullable ParquetMetricsRowGroupFilter eqFilter = null; + EqualityPushdownResult eqPushdown = EqualityPushdownResult.notApplicable(); + if (!eqFiles.isEmpty()) { + eqPushdown = buildEqualityDeletePushdown(table, eqFiles); + if (!eqPushdown.applicable) { + // eq deletes are present but we can't safely identify which row groups they target. + // A narrowed position-only range could drop eq-deleted rows, so fall back to the + // default full-range reader. DeleteReader will still apply residual per record. + return PositionPushdownResult.fallback(eqPushdown.preloadedDeletes(posIndex)); + } + eqFilter = new ParquetMetricsRowGroupFilter(table.schema(), eqPushdown.residual); + } + DeleteReader.PreloadedDeletes preloadedDeletes = eqPushdown.preloadedDeletes(posIndex); + + // 3. read the footer and compute the task byte range covering every row group that + // contains a position delete or matches the eq filter. + long taskStart = task.getStart(); + long taskEnd = taskStart + task.getLength(); + long minStart = Long.MAX_VALUE; + long maxEnd = Long.MIN_VALUE; + long[] sortedDeletePositions = sortedDeletePositions(posIndex); + + try { + InputFile inputFile = table.io().newInputFile(dataFilePath); + try (ParquetFileReader reader = ParquetFileReader.open(asParquetInputFile(inputFile))) { + ParquetMetadata footer = reader.getFooter(); + MessageType parquetSchema = footer.getFileMetaData().getSchema(); + + // track cumulative row count ourselves. not all Parquet writers will include + // it in BlockMetaData.getRowIndexOffset + long cumulativeRows = 0; + for (BlockMetaData rowGroup : footer.getBlocks()) { + long rgStartPos = cumulativeRows; + long rgEndPos = cumulativeRows + rowGroup.getRowCount(); + cumulativeRows = rgEndPos; + + long rgByteStart = rowGroup.getStartingPos(); + long rgByteEnd = rgByteStart + rowGroup.getCompressedSize(); + + // skip row groups outside this task's range. + if (rgByteEnd <= taskStart || rgByteStart >= taskEnd) { + continue; + } + + // if row group has a position and/or an equality delete, include it in the global range + boolean rowGroupHasPosDelete = anyInRange(sortedDeletePositions, rgStartPos, rgEndPos); + boolean rowGroupMatchesEq = + eqFilter != null && eqFilter.shouldRead(parquetSchema, rowGroup); + + if (rowGroupHasPosDelete || rowGroupMatchesEq) { + minStart = Math.min(minStart, rgByteStart); + maxEnd = Math.max(maxEnd, rgByteEnd); + } + } + } + } catch (IOException | RuntimeException e) { + LOG.info( + "Failed to read Parquet footer for {}; falling back to default reader chain.", + dataFilePath, + e); + return PositionPushdownResult.fallback(preloadedDeletes); + } + + long readStart = Math.max(minStart, taskStart); + long readEnd = Math.min(maxEnd, taskEnd); + if (readStart >= readEnd) { + // deletes don't target the portion of the DataFile covered by this read task. + return PositionPushdownResult.of(CloseableIterable.empty(), preloadedDeletes); + } + + // 4. Open the reader with the narrowed byte range. This range represents the union + // of "has position delete" + "matches eq stats" + DeleteFilter existingDeletesFilter = + genericDeleteFilter(table, outputSchema, dataFilePath, task.getExistingDeletes()); + DeleteReader addedDeletesReader = + genericDeleteReader(table, outputSchema, dataFilePath, addedDeletes, preloadedDeletes); + Schema requiredSchema = + TypeUtil.join(existingDeletesFilter.requiredSchema(), addedDeletesReader.requiredSchema()); + CloseableIterable records = + createReader( + task, + table, + scanConfig, + requiredSchema, + Expressions.alwaysTrue(), + readStart, + readEnd - readStart); + CloseableIterable liveRecords = existingDeletesFilter.filter(records); + return PositionPushdownResult.of(addedDeletesReader.read(liveRecords), preloadedDeletes); + } + + /** Materializes a sorted long[] of the positions in {@code posIndex} for binary-search lookup. */ + private static long[] sortedDeletePositions(PositionDeleteIndex posIndex) { + long cardinality = posIndex.cardinality(); + if (cardinality > Integer.MAX_VALUE) { + throw new IllegalStateException( + "Position delete index cardinality exceeds Integer.MAX_VALUE: " + cardinality); + } + long[] arr = new long[(int) cardinality]; + int[] idx = {0}; + posIndex.forEach(p -> arr[idx[0]++] = p); + // forEach is ordered for the bitmap-backed implementation, but the interface doesn't + // promise it, so sort defensively. Cheap relative to the I/O it gates. + Arrays.sort(arr); + return arr; + } + + /** Returns true iff {@code sortedDeletes} contains any value in {@code [start, end)}. */ + private static boolean anyInRange(long[] sortedDeletes, long startInclusive, long endExclusive) { + if (sortedDeletes.length == 0) { + return false; + } + int i = Arrays.binarySearch(sortedDeletes, startInclusive); + if (i < 0) { + i = -i - 1; // insertion point + } + return i < sortedDeletes.length && sortedDeletes[i] < endExclusive; + } + + /** + * Returns an {@code IN} expression suitable as a Parquet residual for the given equality-delete + * files, or {@link Expressions#alwaysTrue()} if pushdown is not applicable. See {@link + * #deletedRowsForTask} for the applicability rules. + */ + private static EqualityPushdownResult buildEqualityDeletePushdown( + Table table, List eqFiles) { + // All eq delete files in this task must share a single equality field id. + Set sharedIds = null; + for (DeleteFile df : eqFiles) { + Set ids = new HashSet<>(df.equalityFieldIds()); + if (sharedIds == null) { + sharedIds = ids; + } else if (!sharedIds.equals(ids)) { + return EqualityPushdownResult.notApplicable(); + } + } + if (sharedIds == null || sharedIds.size() != 1) { + return EqualityPushdownResult.notApplicable(); + } + + int fieldId = Iterables.getOnlyElement(sharedIds); + Types.NestedField field = table.schema().findField(fieldId); + if (field == null) { + return EqualityPushdownResult.notApplicable(); + } + Schema deleteSchema = TypeUtil.select(table.schema(), sharedIds); + + DeleteLoader loader = new BaseDeleteLoader(df -> table.io().newInputFile(df.location())); + StructLikeSet set; + try { + set = loader.loadEqualityDeletes(eqFiles, deleteSchema); + } catch (RuntimeException e) { + LOG.info( + "Failed to pre-load equality deletes for pushdown; falling back to per-record check.", e); + return EqualityPushdownResult.notApplicable(); + } + + Map, StructLikeSet> preloadedSets = new HashMap<>(); + preloadedSets.put(sharedIds, set); + + if (set.size() > IN_PREDICATE_LIMIT) { + return EqualityPushdownResult.notApplicable(preloadedSets); + } + Class javaClass = field.type().typeId().javaClass(); + List values = new ArrayList<>(set.size()); + for (StructLike s : set) { + @Nullable Object v = s.get(0, javaClass); + if (v == null) { + // Nulls don't match an IN-expression. pushing down would drop those deletions. + return EqualityPushdownResult.notApplicable(preloadedSets); + } + values.add(v); + } + if (values.isEmpty()) { + return EqualityPushdownResult.notApplicable(preloadedSets); + } + return EqualityPushdownResult.applicable(Expressions.in(field.name(), values), preloadedSets); + } + + private static final class PositionPushdownResult { + private final @Nullable CloseableIterable deletedRecords; + private final DeleteReader.PreloadedDeletes preloadedDeletes; + + private static PositionPushdownResult of( + CloseableIterable deletedRecords, DeleteReader.PreloadedDeletes preloadedDeletes) { + return new PositionPushdownResult(deletedRecords, preloadedDeletes); + } + + private static PositionPushdownResult fallback(DeleteReader.PreloadedDeletes preloadedDeletes) { + return new PositionPushdownResult(null, preloadedDeletes); + } + + private PositionPushdownResult( + @Nullable CloseableIterable records, + DeleteReader.PreloadedDeletes preloadedDeletes) { + this.deletedRecords = records; + this.preloadedDeletes = preloadedDeletes; + } + } + + private static final class EqualityPushdownResult { + private static final EqualityPushdownResult NOT_APPLICABLE = + new EqualityPushdownResult(Expressions.alwaysTrue(), Collections.emptyMap(), false); + + private final Expression residual; + private final Map, StructLikeSet> preloadedSets; + private final boolean applicable; + + private static EqualityPushdownResult applicable( + Expression residual, Map, StructLikeSet> preloadedSets) { + return new EqualityPushdownResult(residual, preloadedSets, true); + } + + private static EqualityPushdownResult notApplicable() { + return NOT_APPLICABLE; + } + + private static EqualityPushdownResult notApplicable( + Map, StructLikeSet> preloadedSets) { + if (preloadedSets.isEmpty()) { + return NOT_APPLICABLE; + } + return new EqualityPushdownResult(Expressions.alwaysTrue(), preloadedSets, false); + } + + private EqualityPushdownResult( + Expression residual, Map, StructLikeSet> preloadedSets, boolean applicable) { + this.residual = residual; + this.preloadedSets = preloadedSets; + this.applicable = applicable; + } + + private DeleteReader.PreloadedDeletes preloadedDeletes( + @Nullable PositionDeleteIndex positionDeleteIndex) { + return DeleteReader.PreloadedDeletes.of(positionDeleteIndex, preloadedSets); + } + } + + public static class GenericDeleteFilter extends DeleteFilter { + private final FileIO io; + private final InternalRecordWrapper asStructLike; + + @SuppressWarnings("method.invocation") + public GenericDeleteFilter( + FileIO io, + String dataFilePath, + Schema tableSchema, + Schema requiredSchema, + List deleteFiles) { + super(dataFilePath, deleteFiles, tableSchema, requiredSchema); + this.io = io; + this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); + } + + @Override + protected StructLike asStructLike(Record record) { + return asStructLike.wrap(record); + } + + @Override + protected InputFile getInputFile(String location) { + return io.newInputFile(location); + } + } + + public static class GenericDeleteReader extends DeleteReader { + private final FileIO io; + private final InternalRecordWrapper asStructLike; + + @SuppressWarnings("method.invocation") + public GenericDeleteReader( + FileIO io, + String dataFilePath, + Schema tableSchema, + Schema requiredSchema, + List deleteFiles, + DeleteReader.PreloadedDeletes preloadedDeletes) { + super(dataFilePath, deleteFiles, tableSchema, requiredSchema, true, preloadedDeletes); + this.io = io; + this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); + } + + @Override + protected StructLike asStructLike(Record record) { + return asStructLike.wrap(record); + } + + @Override + protected InputFile getInputFile(String location) { + return io.newInputFile(location); + } + } + + /** + * Adapter from Iceberg's {@link InputFile} to Parquet's {@link org.apache.parquet.io.InputFile}, + * for callers that need to open a Parquet file directly (e.g. to read the footer for row-group + * pruning decisions). Iceberg has an equivalent internal {@code ParquetIO} but it's + * package-private. + */ + public static org.apache.parquet.io.InputFile asParquetInputFile(InputFile icebergFile) { + return new IcebergParquetInputFile(icebergFile); + } + + private static final class IcebergParquetInputFile implements org.apache.parquet.io.InputFile { + private final InputFile delegate; + + IcebergParquetInputFile(InputFile delegate) { + this.delegate = delegate; + } + + @Override + public long getLength() { + return delegate.getLength(); + } + + @Override + public org.apache.parquet.io.SeekableInputStream newStream() { + return new IcebergParquetSeekableStream(delegate.newStream()); + } + } + + private static final class IcebergParquetSeekableStream extends DelegatingSeekableInputStream { + private final SeekableInputStream delegate; + + IcebergParquetSeekableStream(SeekableInputStream delegate) { + super(delegate); + this.delegate = delegate; + } + + @Override + public long getPos() throws java.io.IOException { + return delegate.getPos(); + } + + @Override + public void seek(long newPos) throws java.io.IOException { + delegate.seek(newPos); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReader.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReader.java new file mode 100644 index 000000000000..e1b9a9c98583 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReader.java @@ -0,0 +1,309 @@ +/* + * 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.cdc; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimaps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.iceberg.Accessor; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructProjection; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Reads a {@link org.apache.iceberg.DataFile} and returns records marked deleted by the given + * {@link DeleteFile}s. + * + *

This is mostly a copy of {@link org.apache.iceberg.data.DeleteFilter}, but flipping the logic + * to output deleted records instead of filtering them out. + */ +public abstract class DeleteReader { + private static final Logger LOG = LoggerFactory.getLogger(DeleteReader.class); + + private final String filePath; + private final List posDeletes; + private final List eqDeletes; + private final PreloadedDeletes preloadedDeletes; + private final Schema requiredSchema; + private final Accessor posAccessor; + private volatile @Nullable DeleteLoader deleteLoader = null; + private @Nullable PositionDeleteIndex deleteRowPositions = null; + private @Nullable List> isInDeleteSets = null; + + protected DeleteReader( + String filePath, + List deletes, + Schema tableSchema, + Schema expectedSchema, + boolean needRowPosCol, + PreloadedDeletes preloadedDeletes) { + this.filePath = filePath; + this.preloadedDeletes = preloadedDeletes; + + ImmutableList.Builder posDeleteBuilder = ImmutableList.builder(); + ImmutableList.Builder eqDeleteBuilder = ImmutableList.builder(); + for (DeleteFile delete : deletes) { + switch (delete.content()) { + case POSITION_DELETES: + LOG.debug("Adding position delete file {} to reader", delete.location()); + posDeleteBuilder.add(delete); + break; + case EQUALITY_DELETES: + LOG.debug("Adding equality delete file {} to reader", delete.location()); + eqDeleteBuilder.add(delete); + break; + default: + throw new UnsupportedOperationException( + "Unknown delete file content: " + delete.content()); + } + } + + this.posDeletes = posDeleteBuilder.build(); + this.eqDeletes = eqDeleteBuilder.build(); + this.requiredSchema = + fileProjection(tableSchema, expectedSchema, posDeletes, eqDeletes, needRowPosCol); + this.posAccessor = requiredSchema.accessorForField(MetadataColumns.ROW_POSITION.fieldId()); + } + + public Schema requiredSchema() { + return requiredSchema; + } + + protected abstract StructLike asStructLike(T record); + + protected abstract InputFile getInputFile(String location); + + protected InputFile loadInputFile(DeleteFile deleteFile) { + return getInputFile(deleteFile.location()); + } + + protected long pos(T record) { + return (Long) posAccessor.get(asStructLike(record)); + } + + protected DeleteLoader newDeleteLoader() { + return new BaseDeleteLoader(this::loadInputFile); + } + + private DeleteLoader deleteLoader() { + if (deleteLoader == null) { + synchronized (this) { + if (deleteLoader == null) { + this.deleteLoader = newDeleteLoader(); + } + } + } + + return deleteLoader; + } + + /** + * Returns records that are deleted by either the position deletes or the equality + * deletes attached to this reader — i.e. the union of the two delete predicates. + * + *

Each delete-type predicate is built independently and defaults to "false" (no contribution + * to the union) when its side has no delete files. Both predicates are then OR-combined and + * applied in a single pass over {@code records}. This guarantees that: + * + *

    + *
  • A task with only position deletes emits all records whose position is in the index. + *
  • A task with only equality deletes emits all records matching any equality delete value. + *
  • A task with both emits the union of the two (without duplication). + *
+ */ + public CloseableIterable read(CloseableIterable records) { + Predicate isPosDeleted = + posDeletes.isEmpty() ? t -> false : positionDeletePredicate(deletedRowPositions()); + Predicate isEqDeleted = applyEqDeletes().stream().reduce(Predicate::or).orElse(t -> false); + return CloseableIterable.filter(records, isPosDeleted.or(isEqDeleted)); + } + + private Predicate positionDeletePredicate(PositionDeleteIndex positionIndex) { + return record -> positionIndex.isDeleted(pos(record)); + } + + private List> applyEqDeletes() { + if (isInDeleteSets != null) { + return isInDeleteSets; + } + + isInDeleteSets = Lists.newArrayList(); + if (eqDeletes.isEmpty()) { + return isInDeleteSets; + } + + Multimap, DeleteFile> filesByDeleteIds = + Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList); + for (DeleteFile delete : eqDeletes) { + filesByDeleteIds.put(Sets.newHashSet(delete.equalityFieldIds()), delete); + } + + for (Map.Entry, Collection> entry : + filesByDeleteIds.asMap().entrySet()) { + Set ids = entry.getKey(); + Iterable deletes = entry.getValue(); + + Schema deleteSchema = TypeUtil.select(requiredSchema, ids); + + // a projection to select and reorder fields of the file schema to match the delete rows + StructProjection projectRow = StructProjection.create(requiredSchema, deleteSchema); + + StructLikeSet deleteSet = preloadedDeletes.equalityDeleteSet(ids); + if (deleteSet == null) { + deleteSet = deleteLoader().loadEqualityDeletes(deletes, deleteSchema); + } + StructLikeSet deleteSetForPredicate = deleteSet; + Predicate isInDeleteSet = + record -> deleteSetForPredicate.contains(projectRow.wrap(asStructLike(record))); + checkStateNotNull(isInDeleteSets).add(isInDeleteSet); + } + + return checkStateNotNull(isInDeleteSets); + } + + public PositionDeleteIndex deletedRowPositions() { + if (deleteRowPositions == null) { + deleteRowPositions = preloadedDeletes.positionDeleteIndex(); + if (deleteRowPositions == null && !posDeletes.isEmpty()) { + deleteRowPositions = deleteLoader().loadPositionDeletes(posDeletes, filePath); + } + } + + return checkStateNotNull(deleteRowPositions); + } + + /** Delete data already loaded by a planning/pushdown path for one task read. */ + public static final class PreloadedDeletes { + private static final PreloadedDeletes EMPTY = + new PreloadedDeletes(null, Collections.emptyMap()); + + private final @Nullable PositionDeleteIndex positionDeleteIndex; + private final Map, StructLikeSet> equalityDeleteSets; + + public static PreloadedDeletes empty() { + return EMPTY; + } + + public static PreloadedDeletes of( + @Nullable PositionDeleteIndex positionDeleteIndex, + Map, StructLikeSet> equalityDeleteSets) { + if (positionDeleteIndex == null && equalityDeleteSets.isEmpty()) { + return EMPTY; + } + return new PreloadedDeletes(positionDeleteIndex, equalityDeleteSets); + } + + private PreloadedDeletes( + @Nullable PositionDeleteIndex positionDeleteIndex, + Map, StructLikeSet> equalityDeleteSets) { + this.positionDeleteIndex = positionDeleteIndex; + Map, StructLikeSet> copied = new HashMap<>(); + for (Map.Entry, StructLikeSet> entry : equalityDeleteSets.entrySet()) { + copied.put(Collections.unmodifiableSet(Sets.newHashSet(entry.getKey())), entry.getValue()); + } + this.equalityDeleteSets = Collections.unmodifiableMap(copied); + } + + public @Nullable PositionDeleteIndex positionDeleteIndex() { + return positionDeleteIndex; + } + + public @Nullable StructLikeSet equalityDeleteSet(Set equalityFieldIds) { + return equalityDeleteSets.get(equalityFieldIds); + } + } + + private static Schema fileProjection( + Schema tableSchema, + Schema requestedSchema, + List posDeletes, + List eqDeletes, + boolean needRowPosCol) { + if (posDeletes.isEmpty() && eqDeletes.isEmpty()) { + return requestedSchema; + } + + Set requiredIds = Sets.newLinkedHashSet(); + if (needRowPosCol && !posDeletes.isEmpty()) { + requiredIds.add(MetadataColumns.ROW_POSITION.fieldId()); + } + + for (DeleteFile eqDelete : eqDeletes) { + requiredIds.addAll(eqDelete.equalityFieldIds()); + } + + Set missingIds = + Sets.newLinkedHashSet( + Sets.difference(requiredIds, TypeUtil.getProjectedIds(requestedSchema))); + + if (missingIds.isEmpty()) { + return requestedSchema; + } + + // TODO: support adding nested columns. this will currently fail when finding nested columns to + // add + List columns = Lists.newArrayList(requestedSchema.columns()); + for (int fieldId : missingIds) { + if (fieldId == MetadataColumns.ROW_POSITION.fieldId() + || fieldId == MetadataColumns.IS_DELETED.fieldId()) { + continue; // add _pos and _deleted at the end + } + + Types.NestedField field = tableSchema.asStruct().field(fieldId); + Preconditions.checkArgument(field != null, "Cannot find required field for ID %s", fieldId); + + columns.add(field); + } + + if (missingIds.contains(MetadataColumns.ROW_POSITION.fieldId())) { + columns.add(MetadataColumns.ROW_POSITION); + } + + if (missingIds.contains(MetadataColumns.IS_DELETED.fieldId())) { + columns.add(MetadataColumns.IS_DELETED); + } + + return new Schema(columns); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IcebergCdcMetadataColumns.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IcebergCdcMetadataColumns.java new file mode 100644 index 000000000000..407934fb188d --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IcebergCdcMetadataColumns.java @@ -0,0 +1,93 @@ +/* + * 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.cdc; + +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.types.Types; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Supported top-level metadata columns for Beam Iceberg CDC reads. + * + *

The supported columns come from two sources: + * + *

    + *
  • Iceberg row metadata: {@code _row_id} and {@code _last_updated_sequence_number}. These are + * requested from the physical Iceberg reader and are only available for row-lineage tables + * (v3+). + *
  • Changelog context metadata: {@code _change_type}, {@code _commit_snapshot_id}, and {@code + * _commit_snapshot_sequence_number}. These are known from the changelog snapshot/task context + * and are appended when Beam output rows are built. + *
+ */ +@Internal +public final class IcebergCdcMetadataColumns { + public static final String CHANGE_TYPE = MetadataColumns.CHANGE_TYPE.name(); + public static final String COMMIT_SNAPSHOT_SEQUENCE_NUMBER = "_commit_snapshot_sequence_number"; + public static final String COMMIT_SNAPSHOT_ID = MetadataColumns.COMMIT_SNAPSHOT_ID.name(); + public static final String ROW_ID = MetadataColumns.ROW_ID.name(); + public static final String LAST_UPDATED_SEQUENCE_NUMBER = + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name(); + + public static final ImmutableList SUPPORTED_COLUMNS = + ImmutableList.of( + CHANGE_TYPE, + COMMIT_SNAPSHOT_ID, + COMMIT_SNAPSHOT_SEQUENCE_NUMBER, + ROW_ID, + LAST_UPDATED_SEQUENCE_NUMBER); + + private static final ImmutableSet ROW_METADATA_COLUMNS = + ImmutableSet.of(ROW_ID, LAST_UPDATED_SEQUENCE_NUMBER); + + public static boolean isSupportedColumn(String name) { + return SUPPORTED_COLUMNS.contains(name); + } + + public static boolean isRowMetadataColumn(String name) { + return ROW_METADATA_COLUMNS.contains(name); + } + + public static Schema.Field beamField(String name) { + if (CHANGE_TYPE.equals(name)) { + return Schema.Field.of(name, Schema.FieldType.STRING); + } + if (COMMIT_SNAPSHOT_ID.equals(name) || COMMIT_SNAPSHOT_SEQUENCE_NUMBER.equals(name)) { + return Schema.Field.of(name, Schema.FieldType.INT64); + } + if (ROW_ID.equals(name) || LAST_UPDATED_SEQUENCE_NUMBER.equals(name)) { + return Schema.Field.nullable(name, Schema.FieldType.INT64); + } + throw new IllegalArgumentException("Unsupported CDC metadata column: " + name); + } + + /** Returns the Iceberg reader field for row-sourced metadata, or null for commit metadata. */ + public static Types.@Nullable NestedField icebergRowMetadataField(String name) { + if (ROW_ID.equals(name)) { + return MetadataColumns.ROW_ID; + } + if (LAST_UPDATED_SEQUENCE_NUMBER.equals(name)) { + return MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER; + } + return null; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTask.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTask.java new file mode 100644 index 000000000000..9b6955d9e4a5 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTask.java @@ -0,0 +1,282 @@ +/* + * 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.cdc; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.SerializableDataFile; +import org.apache.beam.sdk.io.iceberg.SerializableDeleteFile; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldNumber; +import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; +import org.apache.iceberg.AddedRowsScanTask; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DeletedDataFileScanTask; +import org.apache.iceberg.DeletedRowsScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SerializableChangelogTask { + public enum Type { + ADDED_ROWS, + DELETED_ROWS, + DELETED_FILE + } + + public static SchemaCoder coder() { + try { + return SchemaRegistry.createDefault().getSchemaCoder(SerializableChangelogTask.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + public static SerializableChangelogTask.Builder builder() { + return new AutoValue_SerializableChangelogTask.Builder() + .setExistingDeletes(Collections.emptyList()) + .setAddedDeletes(Collections.emptyList()); + } + + @SchemaFieldNumber("0") + public abstract Type getType(); + + @SchemaFieldNumber("1") + public abstract SerializableDataFile getDataFile(); + + @SchemaFieldNumber("2") + public abstract List getExistingDeletes(); + + @SchemaFieldNumber("3") + public abstract List getAddedDeletes(); + + @SchemaFieldNumber("4") + public abstract int getSpecId(); + + @SchemaFieldNumber("5") + public abstract ChangelogOperation getOperation(); + + @SchemaFieldNumber("6") + public abstract int getOrdinal(); + + @SchemaFieldNumber("7") + public abstract long getCommitSnapshotId(); + + @SchemaFieldNumber("8") + public abstract long getStart(); + + @SchemaFieldNumber("9") + public abstract long getLength(); + + @SchemaFieldNumber("10") + public abstract String getJsonExpression(); + + @SchemaIgnore + public Expression getExpression(Schema schema) { + return ExpressionParser.fromJson(getJsonExpression(), schema); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + public abstract static class Builder { + abstract Builder setType(Type type); + + abstract Builder setDataFile(SerializableDataFile dataFile); + + @SchemaIgnore + public Builder setDataFile(DataFile df, String partitionPath, boolean includeMetrics) { + return setDataFile(SerializableDataFile.from(df, partitionPath, includeMetrics)); + } + + abstract Builder setExistingDeletes(List existingDeletes); + + abstract Builder setAddedDeletes(List addedDeletes); + + abstract Builder setSpecId(int specId); + + abstract Builder setOperation(ChangelogOperation operation); + + abstract Builder setOrdinal(int ordinal); + + abstract Builder setCommitSnapshotId(long commitSnapshotId); + + abstract Builder setStart(long start); + + abstract Builder setLength(long length); + + abstract Builder setJsonExpression(String expression); + + abstract SerializableChangelogTask build(); + } + + public static SerializableChangelogTask from( + ChangelogScanTask task, Map specs) { + return from(task, specs, false); + } + + public static SerializableChangelogTask from( + ChangelogScanTask task, Map specs, boolean includeMetrics) { + checkState( + task instanceof ContentScanTask, "Expected ChangelogScanTask to also be a ContentScanTask"); + ContentScanTask contentScanTask = (ContentScanTask) task; + PartitionSpec spec = contentScanTask.spec(); + SerializableChangelogTask.Builder builder = + SerializableChangelogTask.builder() + .setOperation(task.operation()) + .setOrdinal(task.changeOrdinal()) + .setCommitSnapshotId(task.commitSnapshotId()) + .setDataFile( + contentScanTask.file(), + spec.partitionToPath(contentScanTask.partition()), + includeMetrics) + .setSpecId(spec.specId()) + .setStart(contentScanTask.start()) + .setLength(contentScanTask.length()) + .setJsonExpression(ExpressionParser.toJson(contentScanTask.residual())); + + if (task instanceof AddedRowsScanTask) { + AddedRowsScanTask addedRowsTask = (AddedRowsScanTask) task; + builder = + builder + .setType(Type.ADDED_ROWS) + .setAddedDeletes( + toSerializableDeletes(addedRowsTask.deletes(), specs, includeMetrics)); + } else if (task instanceof DeletedRowsScanTask) { + DeletedRowsScanTask deletedRowsTask = (DeletedRowsScanTask) task; + builder = + builder + .setType(Type.DELETED_ROWS) + .setAddedDeletes( + toSerializableDeletes(deletedRowsTask.addedDeletes(), specs, includeMetrics)) + .setExistingDeletes( + toSerializableDeletes(deletedRowsTask.existingDeletes(), specs, includeMetrics)); + } else if (task instanceof DeletedDataFileScanTask) { + DeletedDataFileScanTask deletedFileTask = (DeletedDataFileScanTask) task; + builder = + builder + .setType(Type.DELETED_FILE) + .setExistingDeletes( + toSerializableDeletes(deletedFileTask.existingDeletes(), specs, includeMetrics)); + } else { + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + return builder.build(); + } + + static Type getType(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return Type.ADDED_ROWS; + } else if (task instanceof DeletedRowsScanTask) { + return Type.DELETED_ROWS; + } else if (task instanceof DeletedDataFileScanTask) { + return Type.DELETED_FILE; + } else { + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + } + + static long getTotalLength(List tasks) { + return tasks.stream().mapToLong(SerializableChangelogTask::getLength).sum(); + } + + static long getLength(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return ((AddedRowsScanTask) task).length(); + } else if (task instanceof DeletedRowsScanTask) { + return ((DeletedRowsScanTask) task).length(); + } else if (task instanceof DeletedDataFileScanTask) { + return ((DeletedDataFileScanTask) task).length(); + } + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + + static StructLike getPartition(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return ((AddedRowsScanTask) task).partition(); + } else if (task instanceof DeletedRowsScanTask) { + return ((DeletedRowsScanTask) task).partition(); + } else if (task instanceof DeletedDataFileScanTask) { + return ((DeletedDataFileScanTask) task).partition(); + } + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + + static PartitionSpec getSpec(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return ((AddedRowsScanTask) task).spec(); + } else if (task instanceof DeletedRowsScanTask) { + return ((DeletedRowsScanTask) task).spec(); + } else if (task instanceof DeletedDataFileScanTask) { + return ((DeletedDataFileScanTask) task).spec(); + } + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + + static DataFile getDataFile(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return ((AddedRowsScanTask) task).file(); + } else if (task instanceof DeletedRowsScanTask) { + return ((DeletedRowsScanTask) task).file(); + } else if (task instanceof DeletedDataFileScanTask) { + return ((DeletedDataFileScanTask) task).file(); + } + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + + static List getAddedDeleteFiles(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return ((AddedRowsScanTask) task).deletes(); + } else if (task instanceof DeletedRowsScanTask) { + return ((DeletedRowsScanTask) task).addedDeletes(); + } else if (task instanceof DeletedDataFileScanTask) { + return Collections.emptyList(); + } + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + + private static List toSerializableDeletes( + List dfs, Map specs, boolean includeMetrics) { + return dfs.stream() + .map( + df -> + SerializableDeleteFile.from( + df, + checkStateNotNull(specs.get(df.specId())).partitionToPath(df.partition()), + includeMetrics)) + .collect(Collectors.toList()); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/package-info.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/package-info.java new file mode 100644 index 000000000000..8285d91689be --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Iceberg CDC connectors. */ +package org.apache.beam.sdk.io.iceberg.cdc; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java index 9b08e1ff86e1..5849cbd00774 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Snapshot; @@ -52,6 +53,9 @@ public class IcebergCdcReadSchemaTransformProviderTest { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private static final org.apache.iceberg.Schema CDC_SCHEMA = + new org.apache.iceberg.Schema(TestFixtures.SCHEMA.columns(), ImmutableSet.of(1)); + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); @Rule public TestPipeline testPipeline = TestPipeline.create(); @@ -83,8 +87,8 @@ public void testSimpleScan() throws Exception { String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); TableIdentifier tableId = TableIdentifier.parse(identifier); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, CDC_SCHEMA); + final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema()); List> expectedRecords = warehouse.commitData(simpleTable); @@ -122,8 +126,8 @@ public void testStreamingReadUsingManagedTransform() throws Exception { String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); TableIdentifier tableId = TableIdentifier.parse(identifier); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, CDC_SCHEMA); + final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema()); List> expectedRecords = warehouse.commitData(simpleTable).subList(3, 9); List snapshots = Lists.newArrayList(simpleTable.snapshots()); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java index f79991cee571..d7c97efa19f3 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java @@ -54,6 +54,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -119,6 +120,14 @@ public static Iterable data() { @Parameter(0) public boolean useIncrementalScan; + private org.apache.iceberg.Schema schemaForMode( + org.apache.iceberg.Schema schema, Integer... identifierFieldIds) { + if (!useIncrementalScan) { + return schema; + } + return new org.apache.iceberg.Schema(schema.columns(), ImmutableSet.copyOf(identifierFieldIds)); + } + static class PrintRow extends PTransform, PCollection> { @Override @@ -143,7 +152,7 @@ public void process(@Element Row row, OutputReceiver output) { public void testFailWhenBothStartingSnapshotAndTimestampAreSet() { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); + warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()) .from(tableId) @@ -161,7 +170,7 @@ public void testFailWhenBothStartingSnapshotAndTimestampAreSet() { public void testFailWhenBothEndingSnapshotAndTimestampAreSet() { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); + warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()) .withCdc() @@ -179,7 +188,7 @@ public void testFailWhenBothEndingSnapshotAndTimestampAreSet() { public void testFailWhenStartingPointAndStartingStrategyAreSet() { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); + warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()) .withCdc() @@ -197,7 +206,7 @@ public void testFailWhenStartingPointAndStartingStrategyAreSet() { public void testFailWhenPollIntervalIsSetOnBatchRead() { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); + warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()) .withCdc() @@ -210,6 +219,32 @@ public void testFailWhenPollIntervalIsSetOnBatchRead() { read.expand(PBegin.in(testPipeline)); } + @Test + public void testCdcFailsWhenTableHasNoIdentifierFields() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId).withCdc(); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("Cannot read CDC records"); + thrown.expectMessage("primary key fields"); + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testCdcFailsWhenProjectionDropsIdentifierFields() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()).from(tableId).withCdc().dropping(singletonList("id")); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("projected schema must not drop primary key fields"); + read.expand(PBegin.in(testPipeline)); + } + @Test public void testFailWhenDropAndKeepAreSet() { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); @@ -307,8 +342,8 @@ public void testProjectedSchemaWithNestedFields() { public void testSimpleScan() throws Exception { TableIdentifier tableId = TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - final Schema schema = icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); + final Schema schema = icebergSchemaToBeamSchema(simpleTable.schema()); List> expectedRecords = warehouse.commitData(simpleTable); @@ -339,8 +374,8 @@ public void testSimpleScan() throws Exception { public void testScanSelectedFields() throws Exception { TableIdentifier tableId = TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - final Schema schema = icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); + final Schema schema = icebergSchemaToBeamSchema(simpleTable.schema()); List> expectedRecords = warehouse.commitData(simpleTable); @@ -368,6 +403,11 @@ public void testScanSelectedFields() throws Exception { return null; }); + if (useIncrementalScan) { + testPipeline.run(); + return; + } + // test drop fields read = read.keeping(null).dropping(singletonList("id")); PCollection outputDrop = @@ -387,7 +427,7 @@ public void testScanSelectedFields() throws Exception { public void testScanWithFilter() throws Exception { TableIdentifier tableId = TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); List> expectedRecords = warehouse.commitData(simpleTable); @@ -440,6 +480,7 @@ public void testReadSchemaWithRandomlyOrderedIds() throws IOException { required(1, "a", Types.IntegerType.get()), required(2, "b", StructType.of(nestedSchema.columns())), required(5, "c", StringType.get())); + schema = schemaForMode(schema, 1); // hadoop catalog will re-order by breadth-first ordering Table simpleTable = warehouse.createTable(tableId, schema); @@ -484,7 +525,8 @@ public void testReadSchemaWithRandomlyOrderedIds() throws IOException { public void testIdentityColumnScan() throws Exception { TableIdentifier tableId = TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + org.apache.iceberg.Schema baseSchema = schemaForMode(TestFixtures.SCHEMA, 1); + Table simpleTable = warehouse.createTable(tableId, baseSchema); String identityColumnName = "identity"; String identityColumnValue = "some-value"; @@ -499,11 +541,7 @@ public void testIdentityColumnScan() throws Exception { .newFastAppend() .appendFile( warehouse.writeRecords( - "file1s1.parquet", - TestFixtures.SCHEMA, - spec, - partitionKey, - TestFixtures.FILE1SNAPSHOT1)) + "file1s1.parquet", baseSchema, spec, partitionKey, TestFixtures.FILE1SNAPSHOT1)) .commit(); final Schema schema = icebergSchemaToBeamSchema(simpleTable.schema()); @@ -609,9 +647,10 @@ public void testNameMappingScan() throws Exception { TableIdentifier tableId = TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + org.apache.iceberg.Schema tableSchema = schemaForMode(TestFixtures.NESTED_SCHEMA, 1); Table simpleTable = warehouse - .buildTable(tableId, TestFixtures.NESTED_SCHEMA) + .buildTable(tableId, tableSchema) .withProperties(tableProperties) .withPartitionSpec(PartitionSpec.unpartitioned()) .create(); @@ -625,7 +664,7 @@ public void testNameMappingScan() throws Exception { .withMetrics(metrics) .build(); - final Schema beamSchema = icebergSchemaToBeamSchema(TestFixtures.NESTED_SCHEMA); + final Schema beamSchema = icebergSchemaToBeamSchema(simpleTable.schema()); simpleTable.newFastAppend().appendFile(dataFile).commit(); @@ -637,7 +676,7 @@ public void testNameMappingScan() throws Exception { final Row[] expectedRows = recordData.stream() - .map(data -> icebergGenericRecord(TestFixtures.NESTED_SCHEMA.asStruct(), data)) + .map(data -> icebergGenericRecord(simpleTable.schema().asStruct(), data)) .map(record -> IcebergUtils.icebergRecordToBeamRow(beamSchema, record)) .toArray(Row[]::new); @@ -695,8 +734,8 @@ public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean throws IOException { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - Schema schema = icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); + Schema schema = icebergSchemaToBeamSchema(simpleTable.schema()); List> expectedRecords = warehouse.commitData(simpleTable); if ((strategy == StartingStrategy.LATEST) || (streaming && strategy == null)) { @@ -731,8 +770,8 @@ public void runReadWithBoundary(boolean useSnapshotBoundary, boolean streaming) throws IOException { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - Schema schema = icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Table simpleTable = warehouse.createTable(tableId, schemaForMode(TestFixtures.SCHEMA, 1)); + Schema schema = icebergSchemaToBeamSchema(simpleTable.schema()); // only read data committed in the second and third snapshots List> expectedRecords = warehouse.commitData(simpleTable).subList(3, 9); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java index a3217503564c..1319efa7229a 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java @@ -46,9 +46,14 @@ import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -77,6 +82,13 @@ public class IcebergSchemaTransformTranslationTest { .build(); private static final Map CONFIG_PROPERTIES = ImmutableMap.builder().put("key", "value").put("key2", "value2").build(); + private static final org.apache.iceberg.Schema CDC_TRANSLATION_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.required(3, "event_micros", Types.LongType.get())), + ImmutableSet.of(1)); private static final Row WRITE_CONFIG_ROW = Row.withSchema(WRITE_PROVIDER.configurationSchema()) .withFieldValue("table", "test_table_identifier") @@ -104,6 +116,8 @@ public class IcebergSchemaTransformTranslationTest { .withFieldValue("to_timestamp", 456L) .withFieldValue("poll_interval_seconds", 123) .withFieldValue("streaming", true) + .withFieldValue("keep", ImmutableList.of("id", "event_micros")) + .withFieldValue("filter", "\"data\" = 'keep'") .build(); @Test @@ -269,7 +283,15 @@ public void testCdcReadTransformProtoTranslation() // First build a pipeline Pipeline p = Pipeline.create(); String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); - warehouse.createTable(TableIdentifier.parse(identifier), TestFixtures.SCHEMA); + Table table = warehouse.createTable(TableIdentifier.parse(identifier), CDC_TRANSLATION_SCHEMA); + table + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "cdc-translation.parquet", + table.schema(), + Collections.singletonList(record(1L, "keep", 123L)))) + .commit(); Map properties = new HashMap<>(CATALOG_PROPERTIES); properties.put("warehouse", warehouse.location); @@ -278,6 +300,9 @@ public void testCdcReadTransformProtoTranslation() Row.fromRow(READ_CDC_CONFIG_ROW) .withFieldValue("table", identifier) .withFieldValue("catalog_properties", properties) + .withFieldValue("from_snapshot", table.currentSnapshot().snapshotId()) + .withFieldValue("to_snapshot", table.currentSnapshot().snapshotId()) + .withFieldValue("to_timestamp", null) .build(); IcebergCdcReadSchemaTransform readTransform = @@ -320,4 +345,10 @@ public void testCdcReadTransformProtoTranslation() assertEquals(transformConfigRow, readTransformFromSpec.getConfigurationRow()); } + + private static Record record(long id, String data, long eventMicros) { + return TestFixtures.createRecord( + CDC_TRANSLATION_SCHEMA, + ImmutableMap.of("id", id, "data", data, "event_micros", eventMicros)); + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/PartitionUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/PartitionUtilsTest.java index d80ec4f95310..740ede55811b 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/PartitionUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/PartitionUtilsTest.java @@ -23,17 +23,25 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Objects; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.transforms.Days; import org.apache.iceberg.transforms.Hours; import org.apache.iceberg.transforms.Months; import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.types.Types; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Test; @@ -166,6 +174,66 @@ public void testAll() { assertEquals(expectedSpec, spec); } + @Test + public void testConstantsMapIncludesCdcMetadataAndIdentityConstants() throws Exception { + org.apache.iceberg.Schema icebergSchema = + new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "category", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(icebergSchema).identity("category").build(); + DataFile file = + DataFiles.builder(spec) + .withFormat(FileFormat.PARQUET) + .withPath("file:///tmp/table/category=A/data.parquet") + .withPartitionPath("category=A") + .withFileSizeInBytes(100L) + .withRecordCount(2L) + .withFirstRowId(99L) + .build(); + setFileSequenceNumber(file, 42L); + + Map constants = PartitionUtils.constantsMap(spec, file, null); + + assertEquals(99L, constants.get(MetadataColumns.ROW_ID.fieldId())); + assertEquals(42L, constants.get(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId())); + assertEquals(file.location(), constants.get(MetadataColumns.FILE_PATH.fieldId())); + assertEquals(file.specId(), constants.get(MetadataColumns.SPEC_ID.fieldId())); + assertEquals("A", constants.get(2)); + } + + @Test + public void testConstantsMapUsesExplicitSequenceNumberWhenFileSequenceIsUnavailable() { + org.apache.iceberg.Schema icebergSchema = + new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "category", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(icebergSchema).identity("category").build(); + DataFile file = + DataFiles.builder(spec) + .withFormat(FileFormat.PARQUET) + .withPath("file:///tmp/table/category=B/data.parquet") + .withPartitionPath("category=B") + .withFileSizeInBytes(100L) + .withRecordCount(2L) + .build(); + + Map constants = PartitionUtils.constantsMap(spec, file, 123L); + + assertEquals(123L, constants.get(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId())); + assertEquals("B", constants.get(2)); + } + + private static void setFileSequenceNumber(DataFile dataFile, long fileSequenceNumber) + throws Exception { + Method method = dataFile.getClass().getMethod("setFileSequenceNumber", Long.class); + method.setAccessible(true); + try { + method.invoke(dataFile, fileSequenceNumber); + } catch (InvocationTargetException e) { + throw (Exception) e.getCause(); + } + } + static class TestCase { private final String field; private @Nullable String name; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java index 73a0fd19e893..df9c44b7b4f3 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java @@ -30,6 +30,7 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -40,7 +41,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.parquet.ParquetReader; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.ClassRule; import org.junit.Rule; @@ -75,14 +75,25 @@ public void testCreateReader() throws IOException { .commit(); } + IcebergScanConfig scanConfig = + IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema())) + .build(); + int numFiles = 0; try (CloseableIterable iterable = simpleTable.newScan().planTasks()) { for (CombinedScanTask combinedScanTask : iterable) { for (FileScanTask fileScanTask : combinedScanTask.tasks()) { String fileName = Iterables.getLast(Splitter.on("/").split(fileScanTask.file().path())); List recordsRead = new ArrayList<>(); - try (ParquetReader reader = - ReadUtils.createReader(fileScanTask, simpleTable, simpleTable.schema())) { + try (CloseableIterable reader = + ReadUtils.createReader(fileScanTask, simpleTable, scanConfig)) { reader.forEach(recordsRead::add); } @@ -94,6 +105,36 @@ public void testCreateReader() throws IOException { assertEquals(data.size(), numFiles); } + @Test + public void testMaybeApplyFilterUsesRequiredSchemaWithFilterOnlyField() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + IcebergScanConfig scanConfig = + IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema())) + .setKeepFields(ImmutableList.of("data")) + .setFilterString("id = 2") + .build(); + + assertEquals(ImmutableList.of("data"), fieldNames(scanConfig.getProjectedSchema())); + assertEquals(ImmutableList.of("id", "data"), fieldNames(scanConfig.getRequiredSchema())); + + CloseableIterable filtered = + ReadUtils.maybeApplyFilter( + CloseableIterable.withNoopClose(TestFixtures.FILE1SNAPSHOT1), + scanConfig, + scanConfig.getRequiredSchema()); + + assertEquals(ImmutableList.of("falafel"), dataOf(filtered)); + } + @Test public void testSnapshotsBetween() throws IOException { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); @@ -239,4 +280,16 @@ static TestCase of( return new TestCase(scanConfig, expectedSnapshotId, description); } } + + private static List fieldNames(org.apache.iceberg.Schema schema) { + return schema.columns().stream() + .map(org.apache.iceberg.types.Types.NestedField::name) + .collect(Collectors.toList()); + } + + private static List dataOf(CloseableIterable records) { + return ImmutableList.copyOf(records).stream() + .map(record -> (String) record.getField("data")) + .collect(Collectors.toList()); + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDataFileTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDataFileTest.java index d4e7793718d8..5126822c06f6 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDataFileTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDataFileTest.java @@ -61,6 +61,9 @@ public class SerializableDataFileTest { .add("nanValueCounts") .add("lowerBounds") .add("upperBounds") + .add("dataSequenceNumber") + .add("fileSequenceNumber") + .add("firstRowId") .build(); @Test diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFileTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFileTest.java new file mode 100644 index 000000000000..29ef30c97efb --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFileTest.java @@ -0,0 +1,222 @@ +/* + * 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 java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; +import org.junit.Test; + +/** Tests for {@link SerializableDeleteFile}. */ +public class SerializableDeleteFileTest { + private static final org.apache.iceberg.Schema SCHEMA = + new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "category", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("category").build(); + + @Test + public void testPositionDeleteRoundTripPreservesMetadataUsedByCdcReads() throws Exception { + Map columnSizes = new HashMap<>(); + columnSizes.put(1, 11L); + Map valueCounts = new HashMap<>(); + valueCounts.put(1, 3L); + Map nullValueCounts = new HashMap<>(); + nullValueCounts.put(1, 0L); + Map nanValueCounts = new HashMap<>(); + nanValueCounts.put(1, 0L); + Map lowerBounds = new HashMap<>(); + lowerBounds.put(1, ByteBuffer.wrap(new byte[] {0x01})); + Map upperBounds = new HashMap<>(); + upperBounds.put(1, ByteBuffer.wrap(new byte[] {0x05})); + Metrics metrics = + new Metrics( + 3L, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("gs://bucket/deletes/category=A/pos.parquet") + .withFormat(FileFormat.PARQUET) + .withPartitionPath("category=A") + .withFileSizeInBytes(256L) + .withMetrics(metrics) + .withSplitOffsets(Arrays.asList(4L, 128L)) + .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[] {0x0A, 0x0B})) + .build(); + setSequenceNumbers(deleteFile, 44L, 45L); + + SerializableDeleteFile serialized = SerializableDeleteFile.from(deleteFile, "category=A", true); + DeleteFile reconstructed = + serialized.createDeleteFile( + singletonMap(SPEC.specId(), SPEC), singletonMap(0, SortOrder.unsorted())); + + assertEquals(deleteFile.content(), reconstructed.content()); + assertEquals(deleteFile.location(), reconstructed.location()); + assertEquals(deleteFile.format(), reconstructed.format()); + assertEquals(deleteFile.recordCount(), reconstructed.recordCount()); + assertEquals(deleteFile.fileSizeInBytes(), reconstructed.fileSizeInBytes()); + assertEquals(deleteFile.partition(), reconstructed.partition()); + assertEquals(deleteFile.specId(), reconstructed.specId()); + assertEquals(deleteFile.keyMetadata(), reconstructed.keyMetadata()); + assertEquals(deleteFile.splitOffsets(), reconstructed.splitOffsets()); + assertEquals(deleteFile.columnSizes(), reconstructed.columnSizes()); + assertEquals(deleteFile.valueCounts(), reconstructed.valueCounts()); + assertEquals(deleteFile.nullValueCounts(), reconstructed.nullValueCounts()); + assertEquals(deleteFile.nanValueCounts(), reconstructed.nanValueCounts()); + assertEquals(deleteFile.lowerBounds(), reconstructed.lowerBounds()); + assertEquals(deleteFile.upperBounds(), reconstructed.upperBounds()); + assertEquals(Long.valueOf(44L), serialized.getDataSequenceNumber()); + assertEquals(Long.valueOf(45L), serialized.getFileSequenceNumber()); + assertNull(reconstructed.dataSequenceNumber()); + assertNull(reconstructed.fileSequenceNumber()); + } + + @Test + public void testEqualityDeleteRoundTripPreservesFieldIdsAndSortOrder() { + SortOrder sortOrder = SortOrder.builderFor(SCHEMA).asc("id").withOrderId(7).build(); + DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1, 2) + .withSortOrder(sortOrder) + .withPath("gs://bucket/deletes/category=A/eq.parquet") + .withFormat(FileFormat.PARQUET) + .withPartitionPath("category=A") + .withFileSizeInBytes(256L) + .withRecordCount(2L) + .build(); + + SerializableDeleteFile serialized = SerializableDeleteFile.from(deleteFile, "category=A", true); + DeleteFile reconstructed = + serialized.createDeleteFile(singletonMap(SPEC.specId(), SPEC), singletonMap(7, sortOrder)); + + assertEquals(FileContent.EQUALITY_DELETES, reconstructed.content()); + assertEquals(Arrays.asList(1, 2), reconstructed.equalityFieldIds()); + assertEquals(Integer.valueOf(7), reconstructed.sortOrderId()); + } + + @Test + public void testPuffinDeleteRoundTripPreservesDeletionVectorMetadata() { + DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("gs://bucket/deletes/category=A/dv.puffin") + .withFormat(FileFormat.PUFFIN) + .withPartitionPath("category=A") + .withFileSizeInBytes(512L) + .withRecordCount(1L) + .withContentOffset(64L) + .withContentSizeInBytes(128L) + .withReferencedDataFile("gs://bucket/data/category=A/data.parquet") + .build(); + + SerializableDeleteFile serialized = SerializableDeleteFile.from(deleteFile, "category=A", true); + DeleteFile reconstructed = + serialized.createDeleteFile( + singletonMap(SPEC.specId(), SPEC), singletonMap(0, SortOrder.unsorted())); + + assertEquals(FileFormat.PUFFIN, reconstructed.format()); + assertEquals(Long.valueOf(64L), reconstructed.contentOffset()); + assertEquals(Long.valueOf(128L), reconstructed.contentSizeInBytes()); + assertEquals("gs://bucket/data/category=A/data.parquet", reconstructed.referencedDataFile()); + } + + @Test + public void testCreateDeleteFileFailsClearlyForMissingPartitionSpec() { + DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("gs://bucket/deletes/category=A/pos.parquet") + .withFormat(FileFormat.PARQUET) + .withPartitionPath("category=A") + .withFileSizeInBytes(256L) + .withRecordCount(2L) + .build(); + SerializableDeleteFile serialized = SerializableDeleteFile.from(deleteFile, "category=A", true); + + IllegalStateException thrown = + assertThrows( + IllegalStateException.class, () -> serialized.createDeleteFile(emptyMap(), null)); + + assertTrue(thrown.getMessage().contains("created with spec id '" + SPEC.specId() + "'")); + } + + @Test + public void testCreateEqualityDeleteFileFailsClearlyForMissingSortOrder() { + SortOrder sortOrder = SortOrder.builderFor(SCHEMA).asc("id").withOrderId(7).build(); + DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1) + .withSortOrder(sortOrder) + .withPath("gs://bucket/deletes/category=A/eq.parquet") + .withFormat(FileFormat.PARQUET) + .withPartitionPath("category=A") + .withFileSizeInBytes(256L) + .withRecordCount(2L) + .build(); + SerializableDeleteFile serialized = SerializableDeleteFile.from(deleteFile, "category=A", true); + + IllegalStateException thrown = + assertThrows( + IllegalStateException.class, + () -> serialized.createDeleteFile(singletonMap(SPEC.specId(), SPEC), emptyMap())); + + assertTrue(thrown.getMessage().contains("sort order id '7'")); + } + + private static void setSequenceNumbers( + DeleteFile deleteFile, long dataSequenceNumber, long fileSequenceNumber) throws Exception { + invoke(deleteFile, "setDataSequenceNumber", dataSequenceNumber); + invoke(deleteFile, "setFileSequenceNumber", fileSequenceNumber); + } + + private static void invoke(DeleteFile deleteFile, String methodName, Long value) + throws Exception { + Method method = deleteFile.getClass().getMethod(methodName, Long.class); + method.setAccessible(true); + try { + method.invoke(deleteFile, value); + } catch (InvocationTargetException e) { + throw (Exception) e.getCause(); + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java index dcb2d804d2e6..2e711219349c 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java @@ -64,7 +64,7 @@ public class TestDataWarehouse extends ExternalResource { protected final Configuration hadoopConf; - protected String location; + public String location; protected Catalog catalog; protected boolean someTableHasBeenCreated = false; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java index 74408d67ed86..f0c7ae925df7 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java @@ -318,7 +318,8 @@ public Row apply(Long num) { }; protected static final org.apache.iceberg.Schema ICEBERG_SCHEMA = - beamSchemaToIcebergSchema(BEAM_SCHEMA); + new org.apache.iceberg.Schema( + beamSchemaToIcebergSchema(BEAM_SCHEMA).columns(), Collections.singleton(1)); protected static final SimpleFunction RECORD_FUNC = new SimpleFunction() { @Override @@ -450,7 +451,7 @@ public void testReadWithColumnPruning_keep() throws Exception { List expectedRows = populateTable(table); - List fieldsToKeep = Arrays.asList("row", "str", "modulo_5", "nullable_long"); + List fieldsToKeep = Arrays.asList("row", "modulo_5", "nullable_long"); RowFilter rowFilter = new RowFilter(BEAM_SCHEMA).keep(fieldsToKeep); Map config = new HashMap<>(managedIcebergConfig(tableId())); @@ -543,7 +544,7 @@ public void testStreamingReadWithColumnPruning_drop() throws Exception { List expectedRows = populateTable(table); - List fieldsToDrop = Arrays.asList("row", "str", "modulo_5", "nullable_long"); + List fieldsToDrop = Arrays.asList("row", "modulo_5", "nullable_long"); RowFilter rowFilter = new RowFilter(BEAM_SCHEMA).drop(fieldsToDrop); Map config = new HashMap<>(managedIcebergConfig(tableId())); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtilsTest.java new file mode 100644 index 000000000000..546386073ff6 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtilsTest.java @@ -0,0 +1,381 @@ +/* + * 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.cdc; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.SerializableDeleteFile; +import org.apache.beam.sdk.io.iceberg.TestDataWarehouse; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link CdcReadUtils}. */ +@RunWith(JUnit4.class) +public class CdcReadUtilsTest { + private static final org.apache.iceberg.Schema CDC_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())), + ImmutableSet.of(1)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + @Rule public TestName testName = new TestName(); + + @Test + public void addedRowsFiltersPositionAndEqualityDeletesWithUnprojectedEqualityColumn() + throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-added.parquet", + table.schema(), + ImmutableList.of( + record(0L, "keep-0"), + record(1L, "drop-by-pos"), + record(2L, "drop-by-data"), + record(3L, "keep-3"))); + + DeleteFile positionDelete = + writePositionDelete(table, dataFile, "cdc-added-pos-delete.parquet", 1L); + DeleteFile equalityDelete = + writeEqualityDelete(table, dataFile, "cdc-added-eq-delete.parquet", "drop-by-data"); + + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.ADDED_ROWS, + dataFile, + ImmutableList.of(positionDelete, equalityDelete), + ImmutableList.of(), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), true); + + assertEquals(ImmutableList.of(0L, 3L), idsOf(records)); + } + + @Test + public void addedRowsUsesFullTableSchemaWhenProjectionDisabled() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-added-full-schema.parquet", + table.schema(), + ImmutableList.of(record(0L, "non-projected-value"))); + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.ADDED_ROWS, + dataFile, + ImmutableList.of(), + ImmutableList.of(), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), false); + + // with projection disabled, all columns should be returned + assertEquals(ImmutableList.of("non-projected-value"), dataValuesOf(records)); + } + + @Test + public void deletedRowsExcludesRowsAlreadyHiddenByExistingDeletes() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-deleted-rows.parquet", + table.schema(), + ImmutableList.of( + record(0L, "already-hidden-by-position"), + record(1L, "new-position-delete"), + record(2L, "already-hidden-by-equality"), + record(3L, "new-equality-delete"), + record(4L, "still-live"))); + + DeleteFile existingPositionDelete = + writePositionDelete(table, dataFile, "cdc-existing-pos-delete.parquet", 0L); + DeleteFile existingEqualityDelete = + writeEqualityDelete( + table, dataFile, "cdc-existing-eq-delete.parquet", "already-hidden-by-equality"); + DeleteFile addedPositionDelete = + writePositionDelete(table, dataFile, "cdc-added-pos-delete.parquet", 1L); + DeleteFile addedEqualityDelete = + writeEqualityDelete(table, dataFile, "cdc-added-eq-delete.parquet", "new-equality-delete"); + + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.DELETED_ROWS, + dataFile, + ImmutableList.of(addedPositionDelete, addedEqualityDelete), + ImmutableList.of(existingPositionDelete, existingEqualityDelete), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), true); + + assertEquals(ImmutableList.of(1L, 3L), idsOf(records)); + } + + @Test + public void deletedRowsUsesFullTableSchemaWhenProjectionDisabled() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-deleted-rows-full-schema.parquet", + table.schema(), + ImmutableList.of(record(0L, "position-deleted-value"), record(1L, "still-live-value"))); + DeleteFile positionDelete = + writePositionDelete(table, dataFile, "cdc-deleted-rows-full-schema-pos-delete.parquet", 0L); + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.DELETED_ROWS, + dataFile, + ImmutableList.of(positionDelete), + ImmutableList.of(), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), false); + + // with projection disabled, all columns should be returned + assertEquals(ImmutableList.of("position-deleted-value"), dataValuesOf(records)); + } + + @Test + public void deletedFileEmitsOnlyRowsNotAlreadyHiddenByExistingDeletes() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-deleted-file.parquet", + table.schema(), + ImmutableList.of( + record(0L, "already-hidden-by-position"), + record(1L, "still-live-one"), + record(2L, "already-hidden-by-equality"), + record(3L, "still-live-two"))); + + DeleteFile existingPositionDelete = + writePositionDelete(table, dataFile, "cdc-deleted-file-pos-delete.parquet", 0L); + DeleteFile existingEqualityDelete = + writeEqualityDelete( + table, dataFile, "cdc-deleted-file-eq-delete.parquet", "already-hidden-by-equality"); + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.DELETED_FILE, + dataFile, + ImmutableList.of(), + ImmutableList.of(existingPositionDelete, existingEqualityDelete), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), true); + + assertEquals(ImmutableList.of(1L, 3L), idsOf(records)); + } + + @Test + public void deletedFileUsesFullTableSchemaWhenProjectionDisabled() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-deleted-file-full-schema.parquet", + table.schema(), + ImmutableList.of(record(0L, "deleted-file-value"))); + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.DELETED_FILE, + dataFile, + ImmutableList.of(), + ImmutableList.of(), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), false); + + // with projection disabled, all columns should be returned + assertEquals(ImmutableList.of("deleted-file-value"), dataValuesOf(records)); + } + + @Test + public void deletedRowsHandlesNullEqualityDeletesWithoutPushdown() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + DataFile dataFile = + warehouse.writeRecords( + "cdc-null-equality-delete.parquet", + table.schema(), + ImmutableList.of(record(0L, "live"), record(1L, null), record(2L, "also-live"))); + DeleteFile nullEqualityDelete = + writeEqualityDelete(table, dataFile, "cdc-null-eq-delete.parquet", null); + SerializableChangelogTask task = + task( + SerializableChangelogTask.Type.DELETED_ROWS, + dataFile, + ImmutableList.of(nullEqualityDelete), + ImmutableList.of(), + table); + + CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig(table, tableId), true); + + assertEquals(ImmutableList.of(1L), idsOf(records)); + } + + private static Record record(long id, String data) { + GenericRecord record = GenericRecord.create(CDC_SCHEMA); + record.setField("id", id); + record.setField("data", data); + return record; + } + + private static DeleteFile writePositionDelete( + Table table, DataFile dataFile, String filename, long... positions) throws IOException { + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(table.schema(), table.spec()); + PositionDeleteWriter writer = + appenderFactory.newPosDeleteWriter( + EncryptedFiles.plainAsEncryptedOutput( + table.io().newOutputFile(dataFile.location() + "." + filename)), + FileFormat.PARQUET, + null); + try (writer) { + for (long position : positions) { + writer.write(PositionDelete.create().set(dataFile.location(), position)); + } + } + return writer.toDeleteFile(); + } + + private static DeleteFile writeEqualityDelete( + Table table, DataFile dataFile, String filename, @Nullable String data) throws IOException { + org.apache.iceberg.Schema deleteSchema = table.schema().select("data"); + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(table.schema(), table.spec(), new int[] {2}, deleteSchema, null); + EqualityDeleteWriter writer = + appenderFactory.newEqDeleteWriter( + EncryptedFiles.plainAsEncryptedOutput( + table.io().newOutputFile(dataFile.location() + "." + filename)), + FileFormat.PARQUET, + null); + try (writer) { + GenericRecord deleteRecord = GenericRecord.create(deleteSchema); + deleteRecord.setField("data", data); + writer.write(deleteRecord); + } + return writer.toDeleteFile(); + } + + private IcebergScanConfig scanConfig(Table table, TableIdentifier tableId) { + return IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) + .setKeepFields(ImmutableList.of("id")) + .build(); + } + + private static SerializableChangelogTask task( + SerializableChangelogTask.Type type, + DataFile dataFile, + List addedDeletes, + List existingDeletes, + Table table) { + return SerializableChangelogTask.builder() + .setType(type) + .setDataFile(dataFile, table.spec().partitionToPath(dataFile.partition()), true) + .setAddedDeletes(serializableDeletes(addedDeletes, table)) + .setExistingDeletes(serializableDeletes(existingDeletes, table)) + .setSpecId(table.spec().specId()) + .setOperation( + type == SerializableChangelogTask.Type.ADDED_ROWS + ? ChangelogOperation.INSERT + : ChangelogOperation.DELETE) + .setOrdinal(0) + .setCommitSnapshotId(1L) + .setStart(0L) + .setLength(dataFile.fileSizeInBytes()) + .setJsonExpression(ExpressionParser.toJson(Expressions.alwaysTrue())) + .build(); + } + + private static List serializableDeletes( + List deletes, Table table) { + return deletes.stream() + .map( + delete -> + SerializableDeleteFile.from( + delete, table.spec().partitionToPath(delete.partition()), true)) + .collect(Collectors.toList()); + } + + private static List idsOf(CloseableIterable records) { + return ImmutableList.copyOf(records).stream() + .map(record -> (Long) record.getField("id")) + .collect(Collectors.toList()); + } + + private static List dataValuesOf(CloseableIterable records) { + return ImmutableList.copyOf(records).stream() + .map(record -> (String) record.getField("data")) + .collect(Collectors.toList()); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReaderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReaderTest.java new file mode 100644 index 000000000000..bf3aaf414579 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReaderTest.java @@ -0,0 +1,419 @@ +/* + * 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.cdc; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Verifies that {@link DeleteReader#read} returns the union of records matched by position + * and equality deletes. + * + *

The tests stub the {@link DeleteLoader} so we exercise the predicate-composition logic + * directly without writing real delete files. End-to-end is covered by other tests. + */ +@RunWith(JUnit4.class) +public class DeleteReaderTest { + private static final Schema TABLE_SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + private static final DeleteFile POS_FILE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/test/pos.parquet") + .withFileSizeInBytes(100) + .withRecordCount(3) + .build(); + + private static final DeleteFile EQ_FILE_ID = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes(1) + .withPath("/test/eq.parquet") + .withFileSizeInBytes(100) + .withRecordCount(2) + .build(); + + private static final DeleteFile EQ_FILE_NAME = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes(2) + .withPath("/test/eq-name.parquet") + .withFileSizeInBytes(100) + .withRecordCount(2) + .build(); + + /** {@link DeleteReader} that returns a stubbed {@link DeleteLoader} for tests. */ + private static class StubDeleteReader extends DeleteReader { + private final DeleteLoader stub; + + StubDeleteReader(List deletes, DeleteLoader stub) { + super( + "/test/data.parquet", + deletes, + TABLE_SCHEMA, + TABLE_SCHEMA, + true, + PreloadedDeletes.empty()); + this.stub = stub; + } + + StubDeleteReader( + List deletes, + DeleteLoader stub, + DeleteReader.PreloadedDeletes preloadedDeletes) { + super("/test/data.parquet", deletes, TABLE_SCHEMA, TABLE_SCHEMA, true, preloadedDeletes); + this.stub = stub; + } + + StubDeleteReader( + List deletes, + DeleteLoader stub, + Schema requestedSchema, + boolean needRowPosCol) { + super( + "/test/data.parquet", + deletes, + TABLE_SCHEMA, + requestedSchema, + needRowPosCol, + PreloadedDeletes.empty()); + this.stub = stub; + } + + @Override + protected StructLike asStructLike(Record record) { + return record; + } + + @Override + protected InputFile getInputFile(String location) { + throw new UnsupportedOperationException("not used with a stubbed DeleteLoader"); + } + + @Override + protected DeleteLoader newDeleteLoader() { + return stub; + } + } + + /** {@link DeleteLoader} that returns pre-built indexes. */ + private static class StubLoader implements DeleteLoader { + private final PositionDeleteIndex posIndex; + private final Map, StructLikeSet> eqSets; + private int posLoadCount = 0; + private int eqLoadCount = 0; + + StubLoader(PositionDeleteIndex posIndex, StructLikeSet eqSet) { + this(posIndex, Collections.singletonMap(Collections.singleton(1), eqSet)); + } + + StubLoader(PositionDeleteIndex posIndex, Map, StructLikeSet> eqSets) { + this.posIndex = posIndex; + this.eqSets = eqSets; + } + + @Override + public PositionDeleteIndex loadPositionDeletes(Iterable files, CharSequence path) { + posLoadCount++; + return posIndex; + } + + @Override + public StructLikeSet loadEqualityDeletes(Iterable files, Schema schema) { + eqLoadCount++; + return eqSets.getOrDefault( + Sets.newHashSet(TypeUtil.getProjectedIds(new Schema(schema.asStruct().fields()))), + StructLikeSet.create(schema.asStruct())); + } + } + + /** A minimal HashSet-backed {@link PositionDeleteIndex} for tests. */ + private static PositionDeleteIndex posIndexOf(long... positions) { + Set backing = new HashSet<>(); + for (long p : positions) { + backing.add(p); + } + return new PositionDeleteIndex() { + @Override + public void delete(long pos) { + backing.add(pos); + } + + @Override + public void delete(long from, long to) { + for (long p = from; p < to; p++) { + backing.add(p); + } + } + + @Override + public boolean isDeleted(long pos) { + return backing.contains(pos); + } + + @Override + public boolean isEmpty() { + return backing.isEmpty(); + } + + @Override + public long cardinality() { + return backing.size(); + } + }; + } + + private static StructLikeSet eqSetOfIds(int... ids) { + Schema idSchema = TABLE_SCHEMA.select("id"); + StructLikeSet set = StructLikeSet.create(idSchema.asStruct()); + for (int id : ids) { + GenericRecord r = GenericRecord.create(idSchema); + r.setField("id", id); + set.add(r); + } + return set; + } + + private static StructLikeSet eqSetOfNames(String... names) { + Schema nameSchema = TABLE_SCHEMA.select("name"); + StructLikeSet set = StructLikeSet.create(nameSchema.asStruct()); + for (String name : names) { + GenericRecord r = GenericRecord.create(nameSchema); + r.setField("name", name); + set.add(r); + } + return set; + } + + /** Builds N records (id=0..N-1, name="v0".."vN-1") matching {@code readSchema}. */ + private static List records(Schema readSchema, int n) { + boolean hasPos = readSchema.findField("_pos") != null; + List recs = new ArrayList<>(n); + for (long i = 0; i < n; i++) { + GenericRecord r = GenericRecord.create(readSchema); + r.setField("id", (int) i); + r.setField("name", "v" + i); + if (hasPos) { + r.setField("_pos", i); + } + recs.add(r); + } + return recs; + } + + /** Sorted list of "id" values from the output, for stable assertions. */ + private static List idsOf(CloseableIterable records) { + return ImmutableList.copyOf(records).stream() + .map(r -> (Integer) r.getField("id")) + .sorted() + .collect(Collectors.toList()); + } + + /** With no delete files at all, {@code read()} emits nothing. */ + @Test + public void noDeletesEmitsNothing() { + DeleteLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); + DeleteReader reader = new StubDeleteReader(Collections.emptyList(), loader); + List input = records(reader.requiredSchema(), 5); + + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + assertEquals(Collections.emptyList(), idsOf(output)); + } + + /** Pos-only emits only the pos-deleted records. */ + @Test + public void posOnlyEmitsPosDeletedRecords() { + DeleteLoader loader = new StubLoader(posIndexOf(1L, 3L), eqSetOfIds()); + DeleteReader reader = new StubDeleteReader(ImmutableList.of(POS_FILE), loader); + List input = records(reader.requiredSchema(), 5); + + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + assertEquals(ImmutableList.of(1, 3), idsOf(output)); + } + + /** Only equality deletes, emits records matching the eq set. */ + @Test + public void eqOnlyEmitsEqDeletedRecords() { + DeleteLoader loader = new StubLoader(posIndexOf(), eqSetOfIds(2, 4)); + DeleteReader reader = new StubDeleteReader(ImmutableList.of(EQ_FILE_ID), loader); + List input = records(reader.requiredSchema(), 5); + + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + assertEquals(ImmutableList.of(2, 4), idsOf(output)); + } + + /** Pos-deletes plus equality deletes, emit the union without duplication. */ + @Test + public void posAndEqEmitUnion() { + DeleteLoader loader = new StubLoader(posIndexOf(0L, 4L), eqSetOfIds(2, 4)); + DeleteReader reader = + new StubDeleteReader(ImmutableList.of(POS_FILE, EQ_FILE_ID), loader); + List input = records(reader.requiredSchema(), 6); + + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + // id 4 is in both sides; it must appear exactly once. + assertEquals(ImmutableList.of(0, 2, 4), idsOf(output)); + } + + /** Preloaded position deletes are reused instead of loading the same delete files again. */ + @Test + public void preloadedPositionDeletesAvoidSecondLoad() { + StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); + PositionDeleteIndex preloadedPosIndex = posIndexOf(1L, 3L); + DeleteReader reader = + new StubDeleteReader( + ImmutableList.of(POS_FILE), + loader, + DeleteReader.PreloadedDeletes.of(preloadedPosIndex, Collections.emptyMap())); + List input = records(reader.requiredSchema(), 5); + + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + assertEquals(ImmutableList.of(1, 3), idsOf(output)); + assertEquals(0, loader.posLoadCount); + } + + /** Preloaded equality delete sets are reused instead of loading the same delete files again. */ + @Test + public void preloadedEqualityDeletesAvoidSecondLoad() { + StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); + Map, StructLikeSet> preloadedEqSets = new HashMap<>(); + preloadedEqSets.put(Collections.singleton(1), eqSetOfIds(2, 4)); + DeleteReader reader = + new StubDeleteReader( + ImmutableList.of(EQ_FILE_ID), + loader, + DeleteReader.PreloadedDeletes.of(null, preloadedEqSets)); + List input = records(reader.requiredSchema(), 5); + + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + assertEquals(ImmutableList.of(2, 4), idsOf(output)); + assertEquals(0, loader.eqLoadCount); + } + + @Test + public void requiredSchemaAddsUnprojectedEqualityDeleteField() { + Schema requestedSchema = TABLE_SCHEMA.select("id"); + DeleteLoader loader = + new StubLoader( + posIndexOf(), Collections.singletonMap(Collections.singleton(2), eqSetOfNames("v2"))); + DeleteReader reader = + new StubDeleteReader(ImmutableList.of(EQ_FILE_NAME), loader, requestedSchema, true); + + assertEquals( + ImmutableList.of("id", "name"), + reader.requiredSchema().columns().stream() + .map(Types.NestedField::name) + .collect(Collectors.toList())); + + List input = records(reader.requiredSchema(), 4); + CloseableIterable output = reader.read(CloseableIterable.withNoopClose(input)); + + assertEquals(ImmutableList.of(2), idsOf(output)); + } + + @Test + public void rowPositionColumnIsOnlyAddedWhenRequiredForPositionDeletes() { + DeleteLoader loader = new StubLoader(posIndexOf(1L), eqSetOfIds()); + + DeleteReader posReaderNeedsPos = + new StubDeleteReader(ImmutableList.of(POS_FILE), loader, TABLE_SCHEMA, true); + DeleteReader posReaderDoesNotNeedPos = + new StubDeleteReader(ImmutableList.of(POS_FILE), loader, TABLE_SCHEMA, false); + DeleteReader eqReader = + new StubDeleteReader(ImmutableList.of(EQ_FILE_ID), loader, TABLE_SCHEMA, true); + + assertNotNull(posReaderNeedsPos.requiredSchema().findField("_pos")); + assertNull(posReaderDoesNotNeedPos.requiredSchema().findField("_pos")); + assertNull(eqReader.requiredSchema().findField("_pos")); + } + + @Test + public void multipleEqualityDeleteGroupsAreOrCombined() { + Map, StructLikeSet> eqSets = new HashMap<>(); + eqSets.put(Collections.singleton(1), eqSetOfIds(1)); + eqSets.put(Collections.singleton(2), eqSetOfNames("v3")); + StubLoader loader = new StubLoader(posIndexOf(), eqSets); + DeleteReader reader = + new StubDeleteReader(ImmutableList.of(EQ_FILE_ID, EQ_FILE_NAME), loader); + + CloseableIterable output = + reader.read(CloseableIterable.withNoopClose(records(reader.requiredSchema(), 5))); + + assertEquals(ImmutableList.of(1, 3), idsOf(output)); + assertEquals(2, loader.eqLoadCount); + } + + @Test + public void preloadedEqualityDeleteKeysAreDefensivelyCopied() { + StructLikeSet idDeletes = eqSetOfIds(2); + Set mutableKey = new HashSet<>(Collections.singleton(1)); + Map, StructLikeSet> preloadedEqSets = new HashMap<>(); + preloadedEqSets.put(mutableKey, idDeletes); + + DeleteReader.PreloadedDeletes preloadedDeletes = + DeleteReader.PreloadedDeletes.of(null, preloadedEqSets); + mutableKey.add(2); + + assertEquals(idDeletes, preloadedDeletes.equalityDeleteSet(Collections.singleton(1))); + + StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); + DeleteReader reader = + new StubDeleteReader(ImmutableList.of(EQ_FILE_ID), loader, preloadedDeletes); + CloseableIterable output = + reader.read(CloseableIterable.withNoopClose(records(reader.requiredSchema(), 4))); + + assertEquals(ImmutableList.of(2), idsOf(output)); + assertEquals(0, loader.eqLoadCount); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTaskTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTaskTest.java new file mode 100644 index 000000000000..aa77d37af7af --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTaskTest.java @@ -0,0 +1,256 @@ +/* + * 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.cdc; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; + +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.iceberg.SerializableDataFile; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.iceberg.AddedRowsScanTask; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DeletedDataFileScanTask; +import org.apache.iceberg.DeletedRowsScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.expressions.Expressions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link SerializableChangelogTask}. */ +@RunWith(JUnit4.class) +public class SerializableChangelogTaskTest { + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final DataFile DATA_FILE = + DataFiles.builder(SPEC) + .withFormat(FileFormat.PARQUET) + .withPath("gs://bucket/data/file.parquet") + .withFileSizeInBytes(512L) + .withMetrics(new Metrics(3L, null, null, null, null, null, null)) + .build(); + private static final DeleteFile ADDED_DELETE = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("gs://bucket/delete/added.parquet") + .withFileSizeInBytes(32L) + .withRecordCount(1L) + .build(); + private static final DeleteFile EXISTING_DELETE = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("gs://bucket/delete/existing.parquet") + .withFileSizeInBytes(64L) + .withRecordCount(2L) + .build(); + + @Test + public void coderRoundTripPreservesTaskBasics() throws Exception { + SerializableChangelogTask task = + SerializableChangelogTask.builder() + .setType(SerializableChangelogTask.Type.ADDED_ROWS) + .setDataFile(SerializableDataFile.from(DATA_FILE, "", false)) + .setSpecId(SPEC.specId()) + .setOperation(ChangelogOperation.INSERT) + .setOrdinal(7) + .setCommitSnapshotId(123L) + .setStart(5L) + .setLength(99L) + .setJsonExpression(ExpressionParser.toJson(Expressions.alwaysTrue())) + .build(); + + SerializableChangelogTask decoded = CoderUtils.clone(SerializableChangelogTask.coder(), task); + + assertEquals(SerializableChangelogTask.Type.ADDED_ROWS, decoded.getType()); + assertEquals(ChangelogOperation.INSERT, decoded.getOperation()); + assertEquals(7, decoded.getOrdinal()); + assertEquals(123L, decoded.getCommitSnapshotId()); + assertEquals(5L, decoded.getStart()); + assertEquals(99L, decoded.getLength()); + assertEquals(DATA_FILE.location(), decoded.getDataFile().getPath()); + assertEquals(DATA_FILE.fileSizeInBytes(), decoded.getDataFile().getFileSizeInBytes()); + assertEquals(Collections.emptyList(), decoded.getExistingDeletes()); + assertEquals(Collections.emptyList(), decoded.getAddedDeletes()); + assertEquals(Expressions.alwaysTrue().toString(), decoded.getExpression(null).toString()); + } + + @Test + public void helperMethodsReadSupportedTaskTypes() { + FakeAddedRowsTask added = new FakeAddedRowsTask(ImmutableList.of(ADDED_DELETE)); + FakeDeletedRowsTask deletedRows = + new FakeDeletedRowsTask(ImmutableList.of(ADDED_DELETE), ImmutableList.of(EXISTING_DELETE)); + FakeDeletedDataFileTask deletedFile = + new FakeDeletedDataFileTask(ImmutableList.of(EXISTING_DELETE)); + + assertEquals( + SerializableChangelogTask.Type.ADDED_ROWS, SerializableChangelogTask.getType(added)); + assertEquals( + SerializableChangelogTask.Type.DELETED_ROWS, + SerializableChangelogTask.getType(deletedRows)); + assertEquals( + SerializableChangelogTask.Type.DELETED_FILE, + SerializableChangelogTask.getType(deletedFile)); + + assertEquals(22L, SerializableChangelogTask.getLength(added)); + assertEquals( + 44L, SerializableChangelogTask.getTotalLength(ImmutableList.of(added, deletedRows))); + assertSame(DATA_FILE, SerializableChangelogTask.getDataFile(deletedRows)); + assertSame(SPEC, SerializableChangelogTask.getSpec(deletedFile)); + assertSame(DATA_FILE.partition(), SerializableChangelogTask.getPartition(added)); + assertThat(SerializableChangelogTask.getAddedDeleteFiles(added), contains(ADDED_DELETE)); + assertThat(SerializableChangelogTask.getAddedDeleteFiles(deletedRows), contains(ADDED_DELETE)); + assertEquals( + Collections.emptyList(), SerializableChangelogTask.getAddedDeleteFiles(deletedFile)); + } + + @Test + public void unsupportedTaskTypeFailsClearly() { + ChangelogScanTask unsupported = + new ChangelogScanTask() { + @Override + public ChangelogOperation operation() { + return ChangelogOperation.INSERT; + } + + @Override + public int changeOrdinal() { + return 0; + } + + @Override + public long commitSnapshotId() { + return 0L; + } + }; + + IllegalStateException thrown = + assertThrows( + IllegalStateException.class, () -> SerializableChangelogTask.getLength(unsupported)); + + assertThat( + thrown.getMessage(), + containsString("Unknown ChangelogScanTask type: " + unsupported.getClass())); + } + + private abstract static class FakeContentTask + implements ChangelogScanTask, ContentScanTask { + @Override + public DataFile file() { + return DATA_FILE; + } + + @Override + public PartitionSpec spec() { + return SPEC; + } + + @Override + public StructLike partition() { + return DATA_FILE.partition(); + } + + @Override + public long start() { + return 11L; + } + + @Override + public long length() { + return 22L; + } + + @Override + public Expression residual() { + return Expressions.alwaysTrue(); + } + + @Override + public int changeOrdinal() { + return 2; + } + + @Override + public long commitSnapshotId() { + return 101L; + } + } + + private static class FakeAddedRowsTask extends FakeContentTask implements AddedRowsScanTask { + private final List deletes; + + FakeAddedRowsTask(List deletes) { + this.deletes = deletes; + } + + @Override + public List deletes() { + return deletes; + } + } + + private static class FakeDeletedRowsTask extends FakeContentTask implements DeletedRowsScanTask { + private final List addedDeletes; + private final List existingDeletes; + + FakeDeletedRowsTask(List addedDeletes, List existingDeletes) { + this.addedDeletes = addedDeletes; + this.existingDeletes = existingDeletes; + } + + @Override + public List addedDeletes() { + return addedDeletes; + } + + @Override + public List existingDeletes() { + return existingDeletes; + } + } + + private static class FakeDeletedDataFileTask extends FakeContentTask + implements DeletedDataFileScanTask { + private final List existingDeletes; + + FakeDeletedDataFileTask(List existingDeletes) { + this.existingDeletes = existingDeletes; + } + + @Override + public List existingDeletes() { + return existingDeletes; + } + } +}