From fbf407f0a4b9beba01afff4b9b3d99aceb47358d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 24 May 2026 13:08:47 -0400 Subject: [PATCH 1/9] refactor: add a local resolution path, optimizations for reading deletes --- sdks/java/io/iceberg/build.gradle | 1 + ...IcebergCdcReadSchemaTransformProvider.java | 23 +- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 33 +- .../sdk/io/iceberg/IcebergScanConfig.java | 115 +- .../beam/sdk/io/iceberg/IcebergUtils.java | 232 ++-- .../beam/sdk/io/iceberg/PartitionUtils.java | 67 ++ .../beam/sdk/io/iceberg/ReadFromTasks.java | 4 +- .../apache/beam/sdk/io/iceberg/ReadUtils.java | 137 +-- .../beam/sdk/io/iceberg/ScanTaskReader.java | 4 +- .../sdk/io/iceberg/SerializableDataFile.java | 100 +- .../io/iceberg/SerializableDeleteFile.java | 333 ++++++ .../beam/sdk/io/iceberg/SnapshotInfo.java | 2 +- .../beam/sdk/io/iceberg/TableCache.java | 9 +- .../io/iceberg/cdc/ApplyWatermarkColumn.java | 96 ++ .../beam/sdk/io/iceberg/cdc/CdcReadUtils.java | 694 +++++++++++ .../beam/sdk/io/iceberg/cdc/CdcResolver.java | 158 +++ .../io/iceberg/cdc/ChangelogDescriptor.java | 65 + .../sdk/io/iceberg/cdc/ChangelogScanner.java | 1047 +++++++++++++++++ .../beam/sdk/io/iceberg/cdc/DeleteReader.java | 309 +++++ .../cdc/IncrementalChangelogSource.java | 197 ++++ .../sdk/io/iceberg/cdc/LocalResolveDoFn.java | 237 ++++ .../beam/sdk/io/iceberg/cdc/OverlapRange.java | 92 ++ .../io/iceberg/cdc/ReadFromChangelogs.java | 437 +++++++ .../sdk/io/iceberg/cdc/ResolveChanges.java | 146 +++ .../cdc/SerializableChangelogTask.java | 282 +++++ .../sdk/io/iceberg/cdc/SnapshotWindowFn.java | 87 ++ .../io/iceberg/cdc/WatchForSnapshotsSdf.java | 286 +++++ .../beam/sdk/io/iceberg/cdc/package-info.java | 20 + .../beam/sdk/io/iceberg/ReadUtilsTest.java | 16 +- .../io/iceberg/SerializableDataFileTest.java | 3 + .../beam/sdk/io/iceberg/TestFixtures.java | 6 +- .../sdk/io/iceberg/cdc/DeleteReaderTest.java | 302 +++++ 32 files changed, 5291 insertions(+), 249 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFile.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumn.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtils.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReader.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRange.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTask.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFn.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/package-info.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReaderTest.java diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 8142c5f5b90b..f874df14ee45 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -50,6 +50,7 @@ dependencies { implementation library.java.slf4j_api implementation library.java.joda_time implementation "org.apache.parquet:parquet-column:$parquet_version" + implementation "org.apache.parquet:parquet-common:$parquet_version" implementation "org.apache.parquet:parquet-hadoop:$parquet_version" implementation "org.apache.parquet:parquet-common:$parquet_version" implementation project(":sdks:java:io:parquet") diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java index 31ff57a668bb..c450beefeab3 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java @@ -118,12 +118,17 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .streaming(configuration.getStreaming()) .keeping(configuration.getKeep()) .dropping(configuration.getDrop()) - .withFilter(configuration.getFilter()); + .withFilter(configuration.getFilter()) + .withWatermarkColumn(configuration.getWatermarkColumn()); @Nullable Integer pollIntervalSeconds = configuration.getPollIntervalSeconds(); if (pollIntervalSeconds != null) { readRows = readRows.withPollInterval(Duration.standardSeconds(pollIntervalSeconds)); } + @Nullable Long maxDelay = configuration.getMaxSnapshotDiscoveryDelay(); + if (maxDelay != null) { + readRows = readRows.withMaxSnapshotDiscoveryDelay(Duration.standardSeconds(maxDelay)); + } PCollection output = input.getPipeline().apply(readRows); @@ -194,6 +199,18 @@ static Builder builder() { "A subset of column names to exclude from reading. If null or empty, all columns will be read.") abstract @Nullable List getDrop(); + @SchemaFieldDescription( + "Column used to derive the source's output watermark. " + + "Must be an existing, required, top-level column of type 'long' or 'timestamp'. " + + "If not set, the watermark advances according to snapshot commit timestamp.") + abstract @Nullable String getWatermarkColumn(); + + @SchemaFieldDescription( + "Maximum expected snapshot discovery delay in seconds. While idle, the source may advance " + + "the watermark to now() minus this delay; snapshots discovered later with older commit " + + "timestamps may be treated as late by downstream windowing. Default: 600 seconds.") + abstract @Nullable Long getMaxSnapshotDiscoveryDelay(); + @AutoValue.Builder abstract static class Builder { abstract Builder setTable(String table); @@ -224,6 +241,10 @@ abstract static class Builder { abstract Builder setFilter(String filter); + abstract Builder setWatermarkColumn(String watermarkColumn); + + abstract Builder setMaxSnapshotDiscoveryDelay(Long seconds); + abstract Configuration build(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java index a5a3beef8f51..6e35ab412b15 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.iceberg.cdc.IncrementalChangelogSource; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PBegin; @@ -569,6 +570,10 @@ public enum StartingStrategy { abstract @Nullable String getFilter(); + abstract @Nullable String getWatermarkColumn(); + + abstract @Nullable Duration getMaxSnapshotDiscoveryDelay(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -599,6 +604,10 @@ abstract static class Builder { abstract Builder setFilter(@Nullable String filter); + abstract Builder setWatermarkColumn(@Nullable String watermarkColumn); + + abstract Builder setMaxSnapshotDiscoveryDelay(@Nullable Duration delay); + abstract ReadRows build(); } @@ -650,12 +659,27 @@ public ReadRows withFilter(@Nullable String filter) { return toBuilder().setFilter(filter).build(); } + public ReadRows withWatermarkColumn(@Nullable String watermarkColumn) { + return toBuilder().setWatermarkColumn(watermarkColumn).build(); + } + + public ReadRows withMaxSnapshotDiscoveryDelay(Duration delay) { + return toBuilder().setMaxSnapshotDiscoveryDelay(delay).build(); + } + @Override public PCollection expand(PBegin input) { TableIdentifier tableId = checkStateNotNull(getTableIdentifier(), "Must set a table to read from."); - - Table table = getCatalogConfig().catalog().loadTable(tableId); + Table table; + try { + table = getCatalogConfig().catalog().loadTable(tableId); + } catch (Exception e) { + throw new RuntimeException( + "Could not fetch table at expansion time. Doing so is needed to " + + "determine the output Row schema.", + e); + } IcebergScanConfig scanConfig = IcebergScanConfig.builder() @@ -674,12 +698,15 @@ public PCollection expand(PBegin input) { .setKeepFields(getKeep()) .setDropFields(getDrop()) .setFilterString(getFilter()) + .setWatermarkColumn(getWatermarkColumn()) + .setMaxSnapshotDiscoveryDelay(getMaxSnapshotDiscoveryDelay()) .build(); scanConfig.validate(table); PTransform> source = getUseCdc() - ? new IncrementalScanSource(scanConfig) + ? new IncrementalChangelogSource(scanConfig) + // ? new IncrementalScanSource(scanConfig) : Read.from(new ScanSource(scanConfig)); return input.apply(source); 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..8d98197caf54 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 @@ -17,27 +17,37 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.io.iceberg.IcebergUtils.icebergSchemaToBeamSchema; +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; 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 static org.apache.iceberg.types.Type.TypeID.LONG; +import static org.apache.iceberg.types.Type.TypeID.TIMESTAMP; import com.google.auto.value.AutoValue; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; +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.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.MetadataColumns; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types.NestedField; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @@ -50,6 +60,8 @@ public abstract class IcebergScanConfig implements Serializable { private transient org.apache.iceberg.@MonotonicNonNull Schema cachedRequiredSchema; private transient @MonotonicNonNull Evaluator cachedEvaluator; private transient @MonotonicNonNull Expression cachedFilter; + private transient org.apache.iceberg.@MonotonicNonNull Schema cachedRecordIdSchema; + private transient @MonotonicNonNull Schema cachedRowIdBeamSchema; public enum ScanType { TABLE, @@ -89,9 +101,9 @@ static org.apache.iceberg.Schema resolveSchema( @Nullable List keep, @Nullable List drop, @Nullable Set fieldsInFilter) { - ImmutableList.Builder selectedFieldsBuilder = ImmutableList.builder(); + Set selectedFields = new LinkedHashSet<>(); if (keep != null && !keep.isEmpty()) { - selectedFieldsBuilder.addAll(keep); + selectedFields.addAll(keep); } else if (drop != null && !drop.isEmpty()) { List paths = new ArrayList<>(TypeUtil.indexNameById(schema.asStruct()).values()); Collections.sort(paths); @@ -100,7 +112,7 @@ static org.apache.iceberg.Schema resolveSchema( boolean isParent = i + 1 < paths.size() && paths.get(i + 1).startsWith(path + "."); boolean isDrop = drop.stream().anyMatch(d -> path.equals(d) || path.startsWith(d + ".")); if (!isParent && !isDrop) { - selectedFieldsBuilder.add(path); + selectedFields.add(path); } } } else { @@ -111,9 +123,8 @@ static org.apache.iceberg.Schema resolveSchema( if (fieldsInFilter != null && !fieldsInFilter.isEmpty()) { fieldsInFilter.stream() .map(f -> schema.caseInsensitiveFindField(f).name()) - .forEach(selectedFieldsBuilder::add); + .forEach(selectedFields::add); } - ImmutableList selectedFields = selectedFieldsBuilder.build(); return selectedFields.isEmpty() ? schema : schema.select(selectedFields); } @@ -141,15 +152,34 @@ public org.apache.iceberg.Schema getRequiredSchema() { return cachedRequiredSchema; } + public org.apache.iceberg.Schema recordIdSchema() { + if (cachedRecordIdSchema == null) { + org.apache.iceberg.Schema fullSchema = getTable().schema(); + cachedRecordIdSchema = TypeUtil.select(fullSchema, fullSchema.identifierFieldIds()); + } + return cachedRecordIdSchema; + } + + public Schema rowIdBeamSchema() { + if (cachedRowIdBeamSchema == null) { + cachedRowIdBeamSchema = icebergSchemaToBeamSchema(recordIdSchema()); + } + return cachedRowIdBeamSchema; + } + + public Comparator recordIdComparator() { + return Comparators.forType(recordIdSchema().asStruct()); + } + @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 +256,12 @@ public Expression getFilter() { @Pure public abstract @Nullable List getDropFields(); + @Pure + public abstract @Nullable String getWatermarkColumn(); + + @Pure + public abstract @Nullable Duration getMaxSnapshotDiscoveryDelay(); + @Pure public static Builder builder() { return new AutoValue_IcebergScanConfig.Builder() @@ -248,7 +284,8 @@ public static Builder builder() { .setPollInterval(null) .setStartingStrategy(null) .setTag(null) - .setBranch(null); + .setBranch(null) + .setWatermarkColumn(null); } @AutoValue.Builder @@ -311,6 +348,10 @@ public Builder setTableIdentifier(String... names) { public abstract Builder setDropFields(@Nullable List fields); + public abstract Builder setWatermarkColumn(@Nullable String watermarkColumn); + + public abstract Builder setMaxSnapshotDiscoveryDelay(@Nullable Duration delay); + public abstract IcebergScanConfig build(); } @@ -328,16 +369,19 @@ void validate(Table table) { String param; if (keep != null) { param = "keep"; - fieldsSpecified = newHashSet(checkNotNull(keep)); + fieldsSpecified = newHashSet(checkArgumentNotNull(keep)); } else { // drop != null param = "drop"; - fieldsSpecified = newHashSet(checkNotNull(drop)); + fieldsSpecified = newHashSet(checkArgumentNotNull(drop)); } fieldsSpecified.removeIf(name -> table.schema().findField(name) != null); checkArgument( - fieldsSpecified.isEmpty(), - error(String.format("'%s' specifies unknown field(s): %s", param, fieldsSpecified))); + fieldsSpecified.isEmpty() + || fieldsSpecified.stream().allMatch(MetadataColumns::isMetadataColumn), + error("'%s' specifies unknown field(s): %s"), + param, + fieldsSpecified); } // TODO(#34168, ahmedabu98): fill these gaps for the existing batch source @@ -371,6 +415,18 @@ 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 projectedPrimaryKeyIds = getProjectedSchema().identifierFieldIds(); + primaryKeyIds.removeAll(projectedPrimaryKeyIds); + 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); } if (getStartingStrategy() != null) { @@ -385,12 +441,43 @@ void validate(Table table) { checkArgument( getToTimestamp() == null || getToSnapshot() == null, error("only one of 'to_timestamp' or 'to_snapshot' can be set")); + @Nullable Long fromSnapshotId = ReadUtils.getFromSnapshotInclusive(table, this); + @Nullable Long toSnapshotId = ReadUtils.getToSnapshot(table, this); + if (fromSnapshotId != null) { + checkArgumentNotNull( + table.snapshot(fromSnapshotId), + error("configured starting snapshot does not exist: '%s'"), + fromSnapshotId); + } + if (toSnapshotId != null) { + checkArgumentNotNull( + table.snapshot(toSnapshotId), + error("configured end snapshot does not exist: '%s'"), + toSnapshotId); + } if (getPollInterval() != null) { checkArgument( Boolean.TRUE.equals(getStreaming()), error("'poll_interval_seconds' can only be set when streaming is true")); } + + @Nullable String watermarkColumn = getWatermarkColumn(); + if (watermarkColumn != null) { + checkArgument(getUseCdc(), error("'watermark_column' is only supported in CDC mode")); + NestedField field = table.schema().findField(watermarkColumn); + checkArgument( + field != null, error("'watermark_column' refers to unknown column: %s"), watermarkColumn); + checkArgument( + field.isRequired(), + error("'watermark_column' refers to a nullable column: %s"), + watermarkColumn); + checkArgument( + field.type().typeId() == TIMESTAMP || field.type().typeId() == LONG, + error("'watermark_column' must be a timestamp-typed column, but '%s' has type %s"), + watermarkColumn, + field.type().typeId()); + } } private String error(String message) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index d0d24532ff39..2b7a8f956bae 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -46,6 +46,7 @@ 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.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.StructLike; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Type; @@ -469,120 +470,141 @@ private static Object getIcebergTimestampValue(Object beamValue, boolean shouldA } } + /** Converts a {@link StructLike} to a Beam {@link Row}. */ + public static Row structToRow(Schema schema, StructLike struct) { + checkState( + schema.getFieldCount() == struct.size(), + "Struct of size %s does not match expected schema size %s", + struct.size(), + schema.getFieldCount()); + Row.Builder rowBuilder = Row.withSchema(schema); + for (int i = 0; i < schema.getFieldCount(); i++) { + Schema.Field field = schema.getField(i); + @Nullable Object icebergValue = struct.get(i, Object.class); + addIcebergValue(rowBuilder, field, icebergValue); + } + return rowBuilder.build(); + } + /** Converts an Iceberg {@link Record} to a Beam {@link Row}. */ public static Row icebergRecordToBeamRow(Schema schema, Record record) { Row.Builder rowBuilder = Row.withSchema(schema); for (Schema.Field field : schema.getFields()) { - boolean isNullable = field.getType().getNullable(); @Nullable Object icebergValue = record.getField(field.getName()); - if (icebergValue == null) { - if (isNullable) { - rowBuilder.addValue(null); - continue; - } - throw new RuntimeException( - String.format("Received null value for required field '%s'.", field.getName())); + addIcebergValue(rowBuilder, field, icebergValue); + } + return rowBuilder.build(); + } + + private static void addIcebergValue( + Row.Builder rowBuilder, Schema.Field field, @Nullable Object icebergValue) { + boolean isNullable = field.getType().getNullable(); + if (icebergValue == null) { + if (isNullable) { + rowBuilder.addValue(null); + return; } - switch (field.getType().getTypeName()) { - case BYTE: - case INT16: - case INT32: - case INT64: - case DECIMAL: // Iceberg and Beam both use BigDecimal - case FLOAT: // Iceberg and Beam both use float - case DOUBLE: // Iceberg and Beam both use double - case STRING: // Iceberg and Beam both use String - case BOOLEAN: // Iceberg and Beam both use boolean - rowBuilder.addValue(icebergValue); - break; - case ARRAY: - checkState( - icebergValue instanceof List, - "Expected List type for field '%s' but received %s", - field.getName(), - icebergValue.getClass()); - List<@NonNull ?> beamList = (List<@NonNull ?>) icebergValue; - Schema.FieldType collectionType = - checkStateNotNull(field.getType().getCollectionElementType()); - // recurse on struct types - if (collectionType.getTypeName().isCompositeType()) { - Schema innerSchema = checkStateNotNull(collectionType.getRowSchema()); - beamList = - beamList.stream() - .map(v -> icebergRecordToBeamRow(innerSchema, (Record) v)) - .collect(Collectors.toList()); - } - rowBuilder.addValue(beamList); - break; - case ITERABLE: - checkState( - icebergValue instanceof Iterable, - "Expected Iterable type for field '%s' but received %s", - field.getName(), - icebergValue.getClass()); - Iterable<@NonNull ?> beamIterable = (Iterable<@NonNull ?>) icebergValue; - Schema.FieldType iterableCollectionType = - checkStateNotNull(field.getType().getCollectionElementType()); - // recurse on struct types - if (iterableCollectionType.getTypeName().isCompositeType()) { - Schema innerSchema = checkStateNotNull(iterableCollectionType.getRowSchema()); - ImmutableList.Builder builder = ImmutableList.builder(); - for (Record v : (Iterable<@NonNull Record>) icebergValue) { - builder.add(icebergRecordToBeamRow(innerSchema, v)); - } - beamIterable = builder.build(); + throw new RuntimeException( + String.format("Received null value for required field '%s'.", field.getName())); + } + switch (field.getType().getTypeName()) { + case BYTE: + case INT16: + case INT32: + case INT64: + case DECIMAL: // Iceberg and Beam both use BigDecimal + case FLOAT: // Iceberg and Beam both use float + case DOUBLE: // Iceberg and Beam both use double + case STRING: // Iceberg and Beam both use String + case BOOLEAN: // Iceberg and Beam both use boolean + rowBuilder.addValue(icebergValue); + break; + case ARRAY: + checkState( + icebergValue instanceof List, + "Expected List type for field '%s' but received %s", + field.getName(), + icebergValue.getClass()); + List<@NonNull ?> beamList = (List<@NonNull ?>) icebergValue; + Schema.FieldType collectionType = + checkStateNotNull(field.getType().getCollectionElementType()); + // recurse on struct types + if (collectionType.getTypeName().isCompositeType()) { + Schema innerSchema = checkStateNotNull(collectionType.getRowSchema()); + beamList = + beamList.stream() + .map(v -> icebergRecordToBeamRow(innerSchema, (Record) v)) + .collect(Collectors.toList()); + } + rowBuilder.addValue(beamList); + break; + case ITERABLE: + checkState( + icebergValue instanceof Iterable, + "Expected Iterable type for field '%s' but received %s", + field.getName(), + icebergValue.getClass()); + Iterable<@NonNull ?> beamIterable = (Iterable<@NonNull ?>) icebergValue; + Schema.FieldType iterableCollectionType = + checkStateNotNull(field.getType().getCollectionElementType()); + // recurse on struct types + if (iterableCollectionType.getTypeName().isCompositeType()) { + Schema innerSchema = checkStateNotNull(iterableCollectionType.getRowSchema()); + ImmutableList.Builder builder = ImmutableList.builder(); + for (Record v : (Iterable<@NonNull Record>) icebergValue) { + builder.add(icebergRecordToBeamRow(innerSchema, v)); } - rowBuilder.addValue(beamIterable); - break; - case MAP: - checkState( - icebergValue instanceof Map, - "Expected Map type for field '%s' but received %s", - field.getName(), - icebergValue.getClass()); - Map beamMap = (Map) icebergValue; - Schema.FieldType valueType = checkStateNotNull(field.getType().getMapValueType()); - // recurse on struct types - if (valueType.getTypeName().isCompositeType()) { - Schema innerSchema = checkStateNotNull(valueType.getRowSchema()); - ImmutableMap.Builder newMap = ImmutableMap.builder(); - for (Map.Entry entry : ((Map) icebergValue).entrySet()) { - Record rec = ((Record) entry.getValue()); - newMap.put( - checkStateNotNull(entry.getKey()), - icebergRecordToBeamRow(innerSchema, checkStateNotNull(rec))); - } - beamMap = newMap.build(); + beamIterable = builder.build(); + } + rowBuilder.addValue(beamIterable); + break; + case MAP: + checkState( + icebergValue instanceof Map, + "Expected Map type for field '%s' but received %s", + field.getName(), + icebergValue.getClass()); + Map beamMap = (Map) icebergValue; + Schema.FieldType valueType = checkStateNotNull(field.getType().getMapValueType()); + // recurse on struct types + if (valueType.getTypeName().isCompositeType()) { + Schema innerSchema = checkStateNotNull(valueType.getRowSchema()); + ImmutableMap.Builder newMap = ImmutableMap.builder(); + for (Map.Entry entry : ((Map) icebergValue).entrySet()) { + Record rec = ((Record) entry.getValue()); + newMap.put( + checkStateNotNull(entry.getKey()), + icebergRecordToBeamRow(innerSchema, checkStateNotNull(rec))); } - rowBuilder.addValue(beamMap); - break; - case DATETIME: - // Iceberg uses a long for micros. - // Beam DATETIME uses joda's DateTime, which only supports millis, - // so we do lose some precision here - rowBuilder.addValue(getBeamDateTimeValue(icebergValue)); - break; - case BYTES: - // Iceberg uses ByteBuffer; Beam uses byte[] - rowBuilder.addValue(((ByteBuffer) icebergValue).array()); - break; - case ROW: - Record nestedRecord = (Record) icebergValue; - Schema nestedSchema = - checkArgumentNotNull( - field.getType().getRowSchema(), - "Corrupted schema: Row type did not have associated nested schema."); - rowBuilder.addValue(icebergRecordToBeamRow(nestedSchema, nestedRecord)); - break; - case LOGICAL_TYPE: - rowBuilder.addValue(getLogicalTypeValue(icebergValue, field.getType())); - break; - default: - throw new UnsupportedOperationException( - "Unsupported Beam type: " + field.getType().getTypeName()); - } + beamMap = newMap.build(); + } + rowBuilder.addValue(beamMap); + break; + case DATETIME: + // Iceberg uses a long for micros. + // Beam DATETIME uses joda's DateTime, which only supports millis, + // so we do lose some precision here + rowBuilder.addValue(getBeamDateTimeValue(icebergValue)); + break; + case BYTES: + // Iceberg uses ByteBuffer; Beam uses byte[] + rowBuilder.addValue(((ByteBuffer) icebergValue).array()); + break; + case ROW: + Record nestedRecord = (Record) icebergValue; + Schema nestedSchema = + checkArgumentNotNull( + field.getType().getRowSchema(), + "Corrupted schema: Row type did not have associated nested schema."); + rowBuilder.addValue(icebergRecordToBeamRow(nestedSchema, nestedRecord)); + break; + case LOGICAL_TYPE: + rowBuilder.addValue(getLogicalTypeValue(icebergValue, field.getType())); + break; + default: + throw new UnsupportedOperationException( + "Unsupported Beam type: " + field.getType().getTypeName()); } - return rowBuilder.build(); } private static DateTime getBeamDateTimeValue(Object icebergValue) { 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..91031fc69e9e 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 { @@ -130,4 +140,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..e2aeeff6d7fd --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFile.java @@ -0,0 +1,333 @@ +/* + * 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.ArrayList; +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(int 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: + int[] equalityFieldIds = + Objects.requireNonNullElse(getEqualityFieldIds(), new ArrayList()).stream() + .mapToInt(Integer::intValue) + .toArray(); + 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/SnapshotInfo.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java index bab5405cd4a5..b51e5be9a5a4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java @@ -78,7 +78,7 @@ public Row toRow() { private static @MonotonicNonNull SchemaCoder coder; private static @MonotonicNonNull Schema schema; - static SchemaCoder getCoder() { + public static SchemaCoder getCoder() { if (coder == null) { initSchemaAndCoder(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java index cb00d90f7fb3..e37157c6ddfe 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java @@ -33,7 +33,8 @@ import org.apache.iceberg.catalog.TableIdentifier; /** Utility to fetch and cache Iceberg {@link Table}s. */ -class TableCache { +// TODO(ahmedabu98): add periodic refresh +public class TableCache { private static final Map CATALOG_CACHE = new ConcurrentHashMap<>(); private static final LoadingCache INTERNAL_CACHE = CacheBuilder.newBuilder() @@ -55,7 +56,7 @@ public ListenableFuture reload(String unusedIdentifier, Table table) { } });; - static Table get(String identifier) { + public static Table get(String identifier) { try { return INTERNAL_CACHE.get(identifier); } catch (ExecutionException e) { @@ -65,12 +66,12 @@ static Table get(String identifier) { } /** Forces a table refresh and returns. */ - static Table getRefreshed(String identifier) { + public static Table getRefreshed(String identifier) { INTERNAL_CACHE.refresh(identifier); return get(identifier); } - static void setup(IcebergScanConfig scanConfig) { + public static void setup(IcebergScanConfig scanConfig) { String tableIdentifier = scanConfig.getTableIdentifier(); IcebergCatalogConfig catalogConfig = scanConfig.getCatalogConfig(); if (CATALOG_CACHE.containsKey(tableIdentifier)) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumn.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumn.java new file mode 100644 index 000000000000..9c1e627f64fd --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumn.java @@ -0,0 +1,96 @@ +/* + * 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 java.util.concurrent.TimeUnit.MICROSECONDS; + +import java.time.LocalDateTime; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.util.DateTimeUtil; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * Re-stamps each output row using the configured {@code watermarkColumn}'s value, so the source's + * output watermark advances per record rather than per snapshot. + * + *

If the configured column's value on a record is null or missing, this DoFn is a pass-through, + * preserving the snapshot commit timestamp. + * + *

The {@link #getAllowedTimestampSkew()} return is intentionally generous — the user's watermark + * column may produce values well before the snapshot commit time (event-time data can lag + * wall-clock by hours or days). Restricting the skew here would force the source to drop legitimate + * output. + */ +class ApplyWatermarkColumn extends DoFn { + private final String watermarkColumn; + + ApplyWatermarkColumn(String watermarkColumn) { + this.watermarkColumn = watermarkColumn; + } + + @ProcessElement + public void process(@Element Row row, OutputReceiver out) { + @Nullable + Instant instant = + getInstant(row.getValue(watermarkColumn), row.getSchema().getField(watermarkColumn)); + if (instant != null) { + out.outputWithTimestamp(row, instant); + } else { + out.output(row); + } + } + + private @Nullable Instant getInstant(@Nullable Object value, Schema.Field field) { + if (value == null) { + return null; + } + switch (field.getType().getTypeName()) { + case INT64: + return Instant.ofEpochMilli(MICROSECONDS.toMillis((Long) value)); + case DATETIME: + return (Instant) value; + case LOGICAL_TYPE: + String logicalType = + Preconditions.checkStateNotNull(field.getType().getLogicalType()).getIdentifier(); + if (logicalType.equals(SqlTypes.DATETIME.getIdentifier())) { + return Instant.ofEpochMilli( + MICROSECONDS.toMillis(DateTimeUtil.microsFromTimestamp((LocalDateTime) value))); + } else if (logicalType.equals(SqlTypes.TIMESTAMP.getIdentifier()) + || logicalType.equals(org.apache.beam.sdk.schemas.logicaltypes.Timestamp.IDENTIFIER)) { + return Instant.ofEpochMilli( + MICROSECONDS.toMillis(DateTimeUtil.microsFromInstant((java.time.Instant) value))); + } else { + throw new UnsupportedOperationException("Unexpected logical type: " + logicalType); + } + default: + throw new UnsupportedOperationException("Unexpected Beam type: " + field.getType()); + } + } + + @Override + public Duration getAllowedTimestampSkew() { + // Generous skew to cover backfill of historical data and late-arriving CDC patterns. + return Duration.standardDays(365); + } +} 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..b8ec49dfcc1c --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtils.java @@ -0,0 +1,694 @@ +/* + * 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. + * + *

This class still delegates to {@link ReadUtils} for the low-level Parquet reader construction + * — the goal is decoupling, not duplication. + */ +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. + */ + public static CloseableIterable changelogRecordsForTask( + SerializableChangelogTask task, + Table table, + IcebergScanConfig scanConfig, + boolean useProjectedSchema) { + String dataFilePath = task.getDataFile().getPath(); + Schema outputSchema = 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/CdcResolver.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java new file mode 100644 index 000000000000..7fe7983613b6 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java @@ -0,0 +1,158 @@ +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.data.Record; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Helper class to reconcile CDC rows. Used by {@link ResolveChanges} (with Beam {@link Row}s) and + * {@link LocalResolveDoFn} (with Iceberg {@link Record}s). + * + *

We determine the output ValueKind as follows: + * + *

    + *
  • (delete, insert) pairs become {@code UPDATE_BEFORE} + {@code UPDATE_AFTER} + *
  • singletons remain {@code DELETE} or {@code INSERT} + *
  • matching delete+insert with identical non-PK fields are considered Copy-on-Write side + * effects and are dropped + *
+ * + *

General implementation: + * + *

    + *
  1. Hash-index inserts by their non-PK field hash, for efficient Copy-on-Write detection. + *
  2. Skip matching (delete, insert) pairs with identical non-PK columns. A CoW operation deletes + * and rewrites the whole file (minus some records that are actually marked for deletion). + * Unchanged records are no-ops and should not be mistaken for updates. + *
  3. Walk the remaining deletes and inserts, emitting matched pairs as {@link + * ChangeKind#UPDATE_BEFORE} / {@link ChangeKind#UPDATE_AFTER}. + *
  4. Emit any unmatched extras as {@link ChangeKind#DELETE} / {@link ChangeKind#INSERT}. + *
+ */ +abstract class CdcResolver { + + // TODO: replace this with actual ValueKind when it lands + enum ChangeKind { + INSERT, + DELETE, + UPDATE_BEFORE, + UPDATE_AFTER + } + + /** Hashes the non-PK fields of an element. Used as the index for O(n+m) CoW deduplication. */ + protected abstract int nonPkHash(T element); + + /** + * Returns true if two records (already known to share a PK) share identical non-PK fields. Called + * only when the two elements collide in the {@link #nonPkHash} index, so the implementation can + * stay simple (linear scan of non-PK fields). + */ + protected abstract boolean nonPkEquals(T delete, T insert); + + /** + * Resolves a Primary Key group of deletes and inserts. Caller provides {@code emit} which decides + * how to materialize each output. + * + *

Both input lists are inspected in their given order. + */ + final void resolve(List deletes, List inserts, BiConsumer emit) { + boolean hasDeletes = !deletes.isEmpty(); + boolean hasInserts = !inserts.isEmpty(); + + if (hasInserts && hasDeletes) { + // First, check if any (delete, insert) pairs are duplicates that should not be + // included in the output + boolean[] dupDeletes = new boolean[deletes.size()]; + boolean[] dupInserts = new boolean[inserts.size()]; + + // Map hash to insert-indices + Map> insertHashToIdx = new HashMap<>(); + for (int insertIdx = 0; insertIdx < inserts.size(); insertIdx++) { + int insertHash = nonPkHash(inserts.get(insertIdx)); + insertHashToIdx.computeIfAbsent(insertHash, k -> new ArrayList<>()).add(insertIdx); + } + for (int deleteIdx = 0; deleteIdx < deletes.size(); deleteIdx++) { + int deleteHash = nonPkHash(deletes.get(deleteIdx)); + @Nullable List candidates = insertHashToIdx.get(deleteHash); + if (candidates != null) { + // check if candidates are just duplicates (e.g. from CoW) + for (int idx = 0; idx < candidates.size(); idx++) { + int insertIdx = candidates.get(idx); + if (!dupInserts[insertIdx] + && nonPkEquals(deletes.get(deleteIdx), inserts.get(insertIdx))) { + // this (delete, insert) pair is a duplicate --> should be skipped + dupDeletes[deleteIdx] = true; + dupInserts[insertIdx] = true; + candidates.remove(idx); + break; + } + } + } + } + + // Emit matched pairs as UPDATE_BEFORE / UPDATE_AFTER. + int d = 0; + int i = 0; + while (d < deletes.size() && i < inserts.size()) { + // skip duplicates + while (d < deletes.size() && dupDeletes[d]) { + d++; + } + while (i < inserts.size() && dupInserts[i]) { + i++; + } + + if (d < deletes.size() && i < inserts.size()) { + emit.accept(ChangeKind.UPDATE_BEFORE, deletes.get(d)); + emit.accept(ChangeKind.UPDATE_AFTER, inserts.get(i)); + d++; + i++; + } + } + + // emit unmatched extras as DELETE / INSERT. + while (d < deletes.size()) { + if (!dupDeletes[d]) { + emit.accept(ChangeKind.DELETE, deletes.get(d)); + } + d++; + } + while (i < inserts.size()) { + if (!dupInserts[i]) { + emit.accept(ChangeKind.INSERT, inserts.get(i)); + } + i++; + } + } else if (hasInserts) { + for (T r : inserts) { + emit.accept(ChangeKind.INSERT, r); + } + } else if (hasDeletes) { + for (T r : deletes) { + emit.accept(ChangeKind.DELETE, r); + } + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java new file mode 100644 index 000000000000..103d38be4012 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java @@ -0,0 +1,65 @@ +/* + * 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 com.google.auto.value.AutoValue; +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.values.Row; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Descriptor for a set of {@link SerializableChangelogTask}s. */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class ChangelogDescriptor { + public static Builder builder() { + return new AutoValue_ChangelogDescriptor.Builder(); + } + + public static SchemaCoder coder() { + try { + return SchemaRegistry.createDefault().getSchemaCoder(ChangelogDescriptor.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @SchemaFieldNumber("0") + abstract String getTableIdentifierString(); + + @SchemaFieldNumber("1") + public abstract @Nullable Row getOverlapLower(); + + @SchemaFieldNumber("2") + public abstract @Nullable Row getOverlapUpper(); + + @AutoValue.Builder + public abstract static class Builder { + abstract Builder setTableIdentifierString(String table); + + abstract Builder setOverlapLower(@Nullable Row overlapLower); + + abstract Builder setOverlapUpper(@Nullable Row overlapUpper); + + abstract ChangelogDescriptor build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java new file mode 100644 index 000000000000..676d21a3da5a --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java @@ -0,0 +1,1047 @@ +/* + * 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 java.lang.String.format; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.Type.ADDED_ROWS; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.getAddedDeleteFiles; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.getDataFile; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.getLength; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.getPartition; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.getSpec; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.getType; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +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.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.TableCache; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +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.Iterables; +import org.apache.iceberg.AddedRowsScanTask; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DeletedDataFileScanTask; +import org.apache.iceberg.DeletedRowsScanTask; +import org.apache.iceberg.IncrementalChangelogScan; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.StructLikeMap; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DoFn that takes incoming Iceberg snapshots and scans them for changelogs using Iceberg's {@link + * IncrementalChangelogScan}. Changelog tasks are organized into batches and routed to different + * downstream PCollections based on complexity. + * + *

The Iceberg scan generates batches of changelog scan tasks, each of size {@link + * TableProperties#SPLIT_SIZE}. This can be configured with the table's read.split.target-size + * property. + * + *

This DoFn analyzes the nature of changes within the snapshot, partition, and file level, then + * routes the changes accordingly: + * + *

    + *
  1. Unidirectional (Fast Path): If an isolated level contains only inserts OR only + * deletes, its tasks are emitted to {@link #UNIDIRECTIONAL_TASKS}. These records bypass + * the CoGBK shuffle and are output immediately. + *
  2. Small Bidirectional (Medium Path): If an isolated level contains a mix of inserts + * and deletes, and is small enough, its tasks are emitted to {@link + * #SMALL_BIDIRECTIONAL_TASKS}. These records are resolved in memory to identify potential + * updates. Task groups are considered small enough if the estimated overlap region is within + * {@link TableProperties#SPLIT_SIZE}. + *
  3. Bidirectional (Slow Path): If an isolated level contains a mix of inserts and + * deletes, and is too large, its tasks are emitted to {@link #LARGE_BIDIRECTIONAL_TASKS}. + * These records are grouped by Primary Key and processed by {@link ResolveChanges} to + * identify potential updates. + *
+ * + *

Optimizing by Shuffling Less Data

+ * + *

We take a three-layered approach to identify data that can bypass the expensive downstream + * CoGroupByKey shuffle: + * + *

Snapshots

+ * + * We start by analyzing the nature of changes at the snapshot level. If a snapshot's operation is + * not of type {@link DataOperations#OVERWRITE}, then it's a uni-directional change. + * + *

Pinned Partitions

+ * + *

If the table's partition fields are derived entirely from Primary Key fields, we know that a + * record will not migrate between partitions. This narrows down the isolated level and allows us to + * only check for bi-directional changes within a partition. Doing this will allow partitions + * with uni-directional changes to bypass the expensive CoGBK shuffle. It also gives partitions with + * small bi-directional changes a chance to be processed in-memory instead of needing to pass + * through the CoGBK. + * + *

Optimization for Individual Files

+ * + * When we have narrowed down our group of tasks with bi-directional changes, we start analyzing the + * metadata of their underlying files. We compare the upper and lower bounds of Partition Keys + * relevant to each file, and consider any overlaps as potentially containing an update. If a given + * task's Primary Key bounds has no overlap with any opposing task's Primary Key bounds, then we + * know it's not possible to create an (insert, delete) pair with it. Such a task can safely bypass + * the shuffle. + * + *

Note: "opposing" refers to a change that happens in the opposite direction (e.g. insert is + * "positive", delete is "negative") + * + *

For example, say we have a group of tasks: + * + *

    + *
  1. Task A (adds rows): bounds [3, 8] + *
  2. Task B (adds rows): bounds [2, 4] + *
  3. Task C (deletes rows): bounds [1, 5] + *
  4. Task D (adds rows): bounds [6, 12] + *
+ * + *

Tasks A and B add rows, and overlap with Task C which deletes row. We need to resolve the rows + * in these 3 tasks because they might all contain (insert, delete) pairs that lead to an update. + * + *

Task D however, does not overlap with any delete rows. It will never produce an (insert, + * delete) pair, so we can directly emit it without resolving its output rows. + */ +class ChangelogScanner + extends DoFn>> { + private static final Logger LOG = LoggerFactory.getLogger(ChangelogScanner.class); + private static final Counter totalChangelogScanTasks = + Metrics.counter(ChangelogScanner.class, "totalChangelogScanTasks"); + private static final Counter numAddedRowsScanTasks = + Metrics.counter(ChangelogScanner.class, "numAddedRowsScanTasks"); + private static final Counter numDeletedRowsScanTasks = + Metrics.counter(ChangelogScanner.class, "numDeletedRowsScanTasks"); + private static final Counter numDeletedDataFileScanTasks = + Metrics.counter(ChangelogScanner.class, "numDeletedDataFileScanTasks"); + private static final Counter numUniDirectionalTasks = + Metrics.counter(ChangelogScanner.class, "numUniDirectionalTasks"); + private static final Counter numLargeBiDirectionalTasks = + Metrics.counter(ChangelogScanner.class, "numLargeBiDirectionalTasks"); + private static final Counter numSmallBiDirectionalTasks = + Metrics.counter(ChangelogScanner.class, "numSmallBiDirectionalTasks"); + static final TupleTag>> + UNIDIRECTIONAL_TASKS = new TupleTag<>(); + static final TupleTag>> + SMALL_BIDIRECTIONAL_TASKS = new TupleTag<>(); + static final TupleTag>> + LARGE_BIDIRECTIONAL_TASKS = new TupleTag<>(); + + static final KvCoder> OUTPUT_CODER = + KvCoder.of(ChangelogDescriptor.coder(), ListCoder.of(SerializableChangelogTask.coder())); + private final IcebergScanConfig scanConfig; + private @MonotonicNonNull Table table; + private @MonotonicNonNull Snapshot snapshot; + private transient @MonotonicNonNull TaskBatcher uniBatcher; + private boolean canDoPartitionOptimization = false; + // for metrics + private int numAddedRowsTasks = 0; + private int numDeletedRowsTasks = 0; + private int numDeletedFileTasks = 0; + private int numUniDirTasks = 0; + private int numSmallBiDirTasks = 0; + private int numLargeBiDirTasks = 0; + private int numUniDirSplits = 0; + private int numSmallBiDirSplits = 0; + private int numLargeBiDirSplits = 0; + + ChangelogScanner(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @Setup + public void setup() { + TableCache.setup(scanConfig); + } + + @ProcessElement + public void process(@Element Long snapshotId, MultiOutputReceiver out) throws IOException { + resetLocalMetrics(); + long millis = System.currentTimeMillis(); + System.out.println("xxx started processing at: " + Instant.ofEpochMilli(millis)); + // not using getRefreshed because upstream Watch should have already refreshed the + // table to a state where this snapshot exists + this.table = SerializableTable.copyOf(TableCache.get(scanConfig.getTableIdentifier())); + this.snapshot = + checkStateNotNull( + table.snapshot(snapshotId), "Could not retrieve table snapshot: %s", snapshotId); + + @Nullable Long fromSnapshotId = snapshot.parentId(); + @Nullable Expression filter = scanConfig.getFilter(); + + IncrementalChangelogScan scan = + table + .newIncrementalChangelogScan() + .toSnapshot(snapshotId) + .project(scanConfig.getProjectedSchema()); + if (fromSnapshotId != null) { + scan = scan.fromSnapshotExclusive(fromSnapshotId); + } + if (filter != null) { + scan = scan.filter(filter); + } + + // configure the scan to store upper/lower bound metrics only + // if it's available for primary key fields + scan = maybeIncludeColumnStats(scan, table); + + createAndOutputReadTasks(scan, out); + long endMillis = System.currentTimeMillis(); + System.out.printf( + "xxx finished processing (%s) after: %s seconds%n", + Instant.ofEpochMilli(endMillis), Duration.millis(endMillis - millis).getStandardSeconds()); + } + + private IncrementalChangelogScan maybeIncludeColumnStats( + IncrementalChangelogScan scan, Table table) { + boolean metricsAvailable = true; + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + Collection pkFields = table.schema().identifierFieldNames(); + for (String field : pkFields) { + MetricsModes.MetricsMode mode = metricsConfig.columnMode(field); + if (!(mode instanceof MetricsModes.Full) && !(mode instanceof MetricsModes.Truncate)) { + metricsAvailable = false; + break; + } + } + if (metricsAvailable) { + scan = scan.includeColumnStats(pkFields); + } + return scan; + } + + @SuppressWarnings("Slf4jFormatShouldBeConst") + private void createAndOutputReadTasks( + IncrementalChangelogScan scan, MultiOutputReceiver multiOutputReceiver) throws IOException { + Snapshot snapshot = checkStateNotNull(this.snapshot); + Table table = checkStateNotNull(this.table); + + // ******** Partition Optimization ******** + // Determine which partition specs "pin" records to their partition + // (i.e. partition fields are sourced entirely from a record's PK). + // If records are pinned, we can optimize by only shuffling bi-directional changes + // *within* a partition, since no cross-partition changes will occur. + Set pinnedSpecs = + table.specs().entrySet().stream() + .filter(e -> doesSpecPinRecordsToPartition(e.getValue())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + boolean tableHasPinnedSpecs = !pinnedSpecs.isEmpty(); + + // The optimization cannot apply if any file in this snapshot uses an unpinned spec + boolean snapshotHasUnpinnedSpec = false; + Set specsInSnapshot = new HashSet<>(); + ChangeTypesInPartition changeTypesInPartition = new ChangeTypesInPartition(); + + // Buffer tasks from OVERWRITE snapshots, because they are potentially bi-directional + OverwriteTasks overwriteTasks = new OverwriteTasks(); + + // batcher for uni-directional tasks, which can be directly emitted when splitSize is reached + uniBatcher = + new TaskBatcher( + scanConfig.getTableIdentifier(), + snapshot.timestampMillis(), + splitSize(), + multiOutputReceiver.get(UNIDIRECTIONAL_TASKS)); + + // === collect partition metadata and route/buffer tasks === + LOG.info( + "Planning to scan snapshot {} (seq: {})", snapshot.snapshotId(), snapshot.sequenceNumber()); + try (CloseableIterable> scanTaskGroups = scan.planTasks()) { + for (ScanTaskGroup scanTaskGroup : scanTaskGroups) { + for (ChangelogScanTask task : scanTaskGroup.tasks()) { + SerializableChangelogTask.Type type = getType(task); + StructLike partition = getPartition(task); + PartitionSpec spec = getSpec(task); + gatherTaskTypeMetrics(type); + + // Collect partition metadata for pinned-spec optimization + if (tableHasPinnedSpecs) { + if (!pinnedSpecs.contains(spec.specId())) { + snapshotHasUnpinnedSpec = true; + } else { + changeTypesInPartition.add(spec, partition, type); + specsInSnapshot.add(spec.specId()); + } + } + + // non-overwrite tasks are always unidirectional (the scan planner + // skips REPLACE ops). + if (!DataOperations.OVERWRITE.equals(snapshot.operation())) { + uniBatcher.add(makeTask(task, table), getLength(task)); + numUniDirTasks++; + continue; + } + + // Overwrite tasks need further analysis — buffer for post-loop processing + overwriteTasks.add(spec, partition, task); + + // TODO: remove debug printing + System.out.printf("\tBuffering overwrite task with partition '%s':%n", partition); + System.out.printf( + "\t\t(%s) DF: %s%n", + task.getClass().getSimpleName(), name(getDataFile(task).location())); + for (DeleteFile delf : getAddedDeleteFiles(task)) { + System.out.println("\t\t\tAdded DelF: " + name(delf.location())); + } + } + } + } + // a snapshot using multiple specs is also not safe for the partition optimization, + // unless we account for the spec ID in the file-to-file comparison, which complicates things + canDoPartitionOptimization = + tableHasPinnedSpecs && !snapshotHasUnpinnedSpec && specsInSnapshot.size() <= 1; + + // === analyze buffered overwrite tasks using the partition metadata === + processOverwriteTasks(overwriteTasks, changeTypesInPartition, multiOutputReceiver); + uniBatcher.flush(); + + int totalTasks = updateTaskCounters(); + + LOG.info(scanResultMessage(totalTasks)); + } + + private void processOverwriteTasks( + OverwriteTasks overwriteTasks, + ChangeTypesInPartition changeTypesInPartition, + MultiOutputReceiver multiOutputReceiver) { + if (overwriteTasks.isEmpty()) { + return; + } + Snapshot snapshot = checkStateNotNull(this.snapshot); + Table table = checkStateNotNull(this.table); + + TaskBatcher uniBatcher = checkStateNotNull(this.uniBatcher); + TaskBatcher largeBiBatcher = + new TaskBatcher( + scanConfig.getTableIdentifier(), + snapshot.timestampMillis(), + splitSize(), + multiOutputReceiver.get(LARGE_BIDIRECTIONAL_TASKS)); + + if (!canDoPartitionOptimization) { + // Records are not pinned to partition (or no pinned specs at all). + // We need to compare underlying files across the whole snapshot. + List tasks = overwriteTasks.allTasks(); + + AnalysisResult result = + analyzeFiles(tasks, scanConfig.recordIdSchema(), scanConfig.recordIdComparator()); + + uniBatcher.add(result.unidirectional, table); + numUniDirTasks += result.unidirectional.size(); + + routeBidirectional(result, largeBiBatcher, multiOutputReceiver); + + // TODO: remove debug printing + System.out.println("\t\tUnpinned spec:"); + for (ChangelogScanTask task : tasks) { + System.out.printf( + "\t\t\t(%s) DF: %s%n", + task.getClass().getSimpleName(), name(getDataFile(task).location())); + for (DeleteFile delf : getAddedDeleteFiles(task)) { + System.out.println("\t\t\tAdded DelF: " + name(delf.location())); + } + } + } else { + // Records are pinned to partition. + // Narrow down by comparing the files within each partition independently. + for (Map.Entry>> tasksPerSpec : + overwriteTasks.tasks.entrySet()) { + int specId = tasksPerSpec.getKey(); + for (Map.Entry> tasksInPartition : + tasksPerSpec.getValue().entrySet()) { + StructLike partition = tasksInPartition.getKey(); + @Nullable + Set partitionChangeTypes = + changeTypesInPartition.typesFor(specId, partition); + + // If this partition has only uni-directional changes, output to UNIDIRECTIONAL and bypass + // file analysis + if (partitionChangeTypes != null && !containsBiDirectionalChanges(partitionChangeTypes)) { + uniBatcher.add(tasksInPartition.getValue(), table); + numUniDirTasks += tasksInPartition.getValue().size(); + continue; + } + + // Partition has bi-directional changes — analyze file-level overlaps + AnalysisResult result = + analyzeFiles( + tasksInPartition.getValue(), + scanConfig.recordIdSchema(), + scanConfig.recordIdComparator()); + + uniBatcher.add(result.unidirectional, table); + routeBidirectional(result, largeBiBatcher, multiOutputReceiver); + + // metrics + numUniDirTasks += result.unidirectional.size(); + numLargeBiDirTasks += result.bidirectional.size(); + + // TODO: remove debug printing + System.out.printf("\t\tPinned Partition '%s' bidirectional:%n", partition); + for (ChangelogScanTask task : tasksInPartition.getValue()) { + System.out.printf( + "\t\t\t(%s) DF: %s%n", + task.getClass().getSimpleName(), name(getDataFile(task).location())); + for (DeleteFile delf : getAddedDeleteFiles(task)) { + System.out.println("\t\t\tAdded DelF: " + name(delf.location())); + } + } + } + } + } + largeBiBatcher.flush(); + numLargeBiDirSplits = largeBiBatcher.totalSplits; + } + + /** + * Helper class for storing + processing {@link ChangelogScanTask}s organized by partition and + * spec ID. + */ + static class OverwriteTasks { + Map>> tasks = new HashMap<>(); + + void add(PartitionSpec spec, StructLike partition, ChangelogScanTask task) { + tasks + .computeIfAbsent(spec.specId(), id -> StructLikeMap.create(spec.partitionType())) + .computeIfAbsent(partition, p -> new ArrayList<>()) + .add(task); + } + + boolean isEmpty() { + return tasks.isEmpty(); + } + + List allTasks() { + return tasks.values().stream() + .flatMap(taskMap -> taskMap.values().stream()) + .flatMap(List::stream) + .collect(Collectors.toList()); + } + } + + /** + * Helper class for identifying types of {@link ChangelogScanTask} per spec ID and partition. This + * is used to determine whether this snapshot is eligible for partition optimization. + */ + static class ChangeTypesInPartition { + Map>> changeTypesPerPartition = + new HashMap<>(); + + void add(PartitionSpec spec, StructLike partition, SerializableChangelogTask.Type type) { + changeTypesPerPartition + .computeIfAbsent(spec.specId(), id -> StructLikeMap.create(spec.partitionType())) + .computeIfAbsent(partition, p -> new HashSet<>()) + .add(type); + } + + @Nullable + Set typesFor(Integer specId, StructLike partition) { + if (!changeTypesPerPartition.containsKey(specId)) { + return null; + } + return checkStateNotNull(changeTypesPerPartition.get(specId)).get(partition); + } + } + + /** Checks if a set of change types include both inserts and deletes. */ + private static boolean containsBiDirectionalChanges( + Set changeTypes) { + return changeTypes.contains(ADDED_ROWS) && changeTypes.size() > 1; + } + + /** Helper class for analyzing overlaps between opposing tasks. */ + static class AnalysisResult { + final List unidirectional; + final List bidirectional; + final @Nullable StructLike overlapLower; + final @Nullable StructLike overlapUpper; + + AnalysisResult( + List unidirectional, + List bidirectional, + @Nullable StructLike overlapLower, + @Nullable StructLike overlapUpper) { + this.unidirectional = unidirectional; + this.bidirectional = bidirectional; + this.overlapLower = overlapLower; + this.overlapUpper = overlapUpper; + } + + @Nullable + Row overlapLowerRow(org.apache.beam.sdk.schemas.Schema idSchema) { + return this.overlapLower == null + ? null + : IcebergUtils.icebergRecordToBeamRow(idSchema, (Record) this.overlapLower); + } + + @Nullable + Row overlapUpperRow(org.apache.beam.sdk.schemas.Schema idSchema) { + return this.overlapUpper == null + ? null + : IcebergUtils.icebergRecordToBeamRow(idSchema, (Record) this.overlapUpper); + } + } + + /** + * Analyzes all tasks in the given list by comparing the bounds of each task's underlying files. + * If a task's partition key bounds overlap with an opposing task's partition key bounds, they are + * both considered bi-directional changes. If a task's bounds do not overlap with any opposing + * task's bounds, it is considered a uni-directional change. + */ + static AnalysisResult analyzeFiles( + List tasks, Schema recIdSchema, Comparator idComp) { + List insertTasks = new ArrayList<>(); + List deleteTasks = new ArrayList<>(); + + try { + for (ChangelogScanTask task : tasks) { + if (task instanceof AddedRowsScanTask) { + insertTasks.add(TaskAndBounds.of(task, recIdSchema, idComp)); + } else if (task instanceof DeletedDataFileScanTask || task instanceof DeletedRowsScanTask) { + deleteTasks.add(TaskAndBounds.of(task, recIdSchema, idComp)); + } else { + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + } + } catch (TaskAndBounds.NoBoundMetricsException e) { + // if metrics are not fully available, we need to play it safe and shuffle all the tasks. + return new AnalysisResult(Collections.emptyList(), tasks, null, null); + } + + if (!insertTasks.isEmpty() && !deleteTasks.isEmpty()) { + Comparator lowerBoundComp = (t1, t2) -> idComp.compare(t1.lowerId, t2.lowerId); + Comparator upperBoundComp = (t1, t2) -> idComp.compare(t1.upperId, t2.upperId); + + insertTasks.sort(lowerBoundComp); + deleteTasks.sort(lowerBoundComp); + + TaskAndBounds firstInsert = insertTasks.get(0); + TaskAndBounds firstDelete = deleteTasks.get(0); + TaskAndBounds lastInsert = insertTasks.stream().max(upperBoundComp).orElseThrow(); + TaskAndBounds lastDelete = deleteTasks.stream().max(upperBoundComp).orElseThrow(); + + boolean overlapExists = + idComp.compare(firstDelete.lowerId, lastInsert.upperId) < 0 + && idComp.compare(firstInsert.lowerId, lastDelete.upperId) < 0; + + if (overlapExists) { + // Iterate through inserts and only check relevant deletes + for (TaskAndBounds insert : insertTasks) { + // First check if the insert task overlaps with the global delete window. + // If not, we can just skip it. + if (idComp.compare(insert.upperId, firstDelete.lowerId) < 0 + || idComp.compare(insert.lowerId, lastDelete.upperId) > 0) { + continue; + } + + for (TaskAndBounds del : deleteTasks) { + // if the delete task's lower bound is already past the insert task's upper bound, + // no subsequent delete can overlap this insert (because we sorted above). + // We can break inner loop. + if (idComp.compare(del.lowerId, insert.upperId) > 0) { + break; + } + + del.checkOverlapWith(insert, idComp); + } + } + } + } + + // collect results and return. + // overlapping tasks are bidirectional. + // otherwise they are unidirectional. + List unidirectional = new ArrayList<>(); + List bidirectional = new ArrayList<>(); + + for (TaskAndBounds taskAndBounds : Iterables.concat(deleteTasks, insertTasks)) { + String msg = ""; + if (taskAndBounds.overlaps) { + msg += + format( + "overlapping task: (%s, %s)", + taskAndBounds.task.commitSnapshotId(), + taskAndBounds.task.getClass().getSimpleName()); + bidirectional.add(taskAndBounds.task); + } else { + unidirectional.add(taskAndBounds.task); + msg += + format( + "NON-overlapping task: (%s, %s)", + taskAndBounds.task.commitSnapshotId(), + taskAndBounds.task.getClass().getSimpleName()); + } + msg += "\n\tDF: " + name(getDataFile(taskAndBounds.task).location()); + msg += "\n\t\tlower: " + taskAndBounds.lowerId + ", upper: " + taskAndBounds.upperId; + if (!getAddedDeleteFiles(taskAndBounds.task).isEmpty()) { + for (DeleteFile df : getAddedDeleteFiles(taskAndBounds.task)) { + msg += "\n\tAdded DelF: " + name(df.location()); + msg += "\n\t\tlower: " + taskAndBounds.lowerId + ", upper: " + taskAndBounds.upperId; + } + } + System.out.println(msg); + } + + StructLike overlapLower = null; + StructLike overlapUpper = null; + if (!bidirectional.isEmpty()) { + StructLike globalInsertLower = + insertTasks.stream() + .filter(t -> t.overlaps) + .map(t -> t.lowerId) + .min(idComp) + .orElseThrow(); + StructLike globalInsertUpper = + insertTasks.stream() + .filter(t -> t.overlaps) + .map(t -> t.upperId) + .max(idComp) + .orElseThrow(); + StructLike globalDeleteLower = + deleteTasks.stream() + .filter(t -> t.overlaps) + .map(t -> t.lowerId) + .min(idComp) + .orElseThrow(); + StructLike globalDeleteUpper = + deleteTasks.stream() + .filter(t -> t.overlaps) + .map(t -> t.upperId) + .max(idComp) + .orElseThrow(); + + overlapLower = + idComp.compare(globalInsertLower, globalDeleteLower) > 0 + ? globalInsertLower + : globalDeleteLower; + overlapUpper = + idComp.compare(globalInsertUpper, globalDeleteUpper) < 0 + ? globalInsertUpper + : globalDeleteUpper; + } + + return new AnalysisResult(unidirectional, bidirectional, overlapLower, overlapUpper); + } + + /** + * Routes bi-directional tasks from an {@link AnalysisResult} to either the in-memory local + * resolve path (when the estimated overlap region fits in one split) or the CoGroupByKey shuffle + * path otherwise. + * + *

For LOCAL routing, all bi-directional tasks for this snapshot/partition group are emitted as + * a batch so that the downstream {@link LocalResolveDoFn} can resolve them together in-memory. // + * * The total byte size may exceed {@code splitSize}, but the in-memory // * footprint is bounded + * by the overlap byte estimate (the local resolver still does per-record PK // * routing to avoid + * buffering records outside the overlap range). + * + *

Returns the number of tasks routed to LOCAL so the caller can update counters. + */ + private void routeBidirectional( + AnalysisResult result, TaskBatcher largeBiBatcher, MultiOutputReceiver multiOutputReceiver) { + Snapshot snapshot = checkStateNotNull(this.snapshot); + + if (result.bidirectional.isEmpty()) { + return; + } + + long totalBytes = + result.bidirectional.stream().mapToLong(SerializableChangelogTask::getLength).sum(); + + @Nullable Row overlapLowerRow = result.overlapLowerRow(scanConfig.rowIdBeamSchema()); + @Nullable Row overlapUpperRow = result.overlapUpperRow(scanConfig.rowIdBeamSchema()); + ChangelogDescriptor descriptor = + ChangelogDescriptor.builder() + .setTableIdentifierString(scanConfig.getTableIdentifier()) + .setOverlapLower(overlapLowerRow) + .setOverlapUpper(overlapUpperRow) + .build(); + + List serializedTasks = + result.bidirectional.stream() + .map(t -> makeTask(t, checkStateNotNull(table))) + .collect(Collectors.toList()); + + // If the batch is small enough, we can route to LOCAL (in-memory) resolver + if (totalBytes <= splitSize()) { + Instant ts = Instant.ofEpochMilli(snapshot.timestampMillis()); + multiOutputReceiver + .get(SMALL_BIDIRECTIONAL_TASKS) + .outputWithTimestamp(KV.of(descriptor, serializedTasks), ts); + System.out.printf( + "xxx LOCAL_RESOLVE (snap=%d, %d tasks, ~%d total bytes)%n", + snapshot.snapshotId(), result.bidirectional.size(), totalBytes); + numSmallBiDirTasks += result.bidirectional.size(); + numSmallBiDirSplits++; + return; + } + + // If the batch is too big, we need to route to the CoGBK for distributed resolution + for (SerializableChangelogTask t : serializedTasks) { + largeBiBatcher.add(descriptor, t, t.getLength()); + } + System.out.printf( + "xxx BIDIRECTIONAL (snap=%d, %d tasks, ~%d total bytes)%n", + snapshot.snapshotId(), result.bidirectional.size(), totalBytes); + } + + private static SerializableChangelogTask makeTask(ChangelogScanTask task, Table table) { + return SerializableChangelogTask.from(task, table.specs()); + } + + /** + * Wraps the {@link ChangelogScanTask}, and stores its lower and upper Primary Keys. Identifies + * overlaps with other tasks by comparing lower and upper keys using Iceberg libraries. + */ + static class TaskAndBounds { + ChangelogScanTask task; + StructLike lowerId; + StructLike upperId; + boolean overlaps = false; + + private TaskAndBounds(ChangelogScanTask task, StructLike lowerId, StructLike upperId) { + this.task = task; + this.lowerId = lowerId; + this.upperId = upperId; + } + + static TaskAndBounds of( + ChangelogScanTask task, Schema recIdSchema, Comparator idComp) + throws NoBoundMetricsException { + @MonotonicNonNull GenericRecord lowerId = null; + @MonotonicNonNull GenericRecord upperId = null; + + if (task instanceof AddedRowsScanTask || task instanceof DeletedDataFileScanTask) { + // just store the bounds of the DataFile + DataFile df = getDataFile(task); + @Nullable Map lowerBounds = df.lowerBounds(); + @Nullable Map upperBounds = df.upperBounds(); + if (lowerBounds == null || upperBounds == null) { + throw new NoBoundMetricsException( + format( + "Upper and/or lower bounds are missing for %s with DataFile: %s.", + task.getClass().getSimpleName(), df.location())); + } + + lowerId = createRecId(recIdSchema, lowerBounds); + upperId = createRecId(recIdSchema, upperBounds); + } else if (task instanceof DeletedRowsScanTask) { + // iterate over all added DeleteFiles and keep track of only the + // minimum and maximum bounds over the list + for (DeleteFile deleteFile : ((DeletedRowsScanTask) task).addedDeletes()) { + @Nullable Map lowerDelBounds = deleteFile.lowerBounds(); + @Nullable Map upperDelBounds = deleteFile.upperBounds(); + if (lowerDelBounds == null || upperDelBounds == null) { + throw new NoBoundMetricsException( + format( + "Upper and/or lower bounds are missing for %s with " + + "DataFile '%s' and DeleteFile '%s'", + task.getClass().getSimpleName(), + getDataFile(task).location(), + deleteFile.location())); + } + + GenericRecord delFileLower = createRecId(recIdSchema, lowerDelBounds); + GenericRecord delFileUpper = createRecId(recIdSchema, upperDelBounds); + + if (lowerId == null || idComp.compare(delFileLower, lowerId) < 0) { + lowerId = delFileLower; + } + if (upperId == null || idComp.compare(delFileUpper, upperId) > 0) { + upperId = delFileUpper; + } + } + } else { + throw new UnsupportedOperationException( + "Unsupported task type: " + task.getClass().getSimpleName()); + } + + if (lowerId == null || upperId == null) { + throw new NoBoundMetricsException( + format( + "Could not compute min and/or max bounds for %s with DataFile: %s", + task.getClass().getSimpleName(), getDataFile(task).location())); + } + return new TaskAndBounds(task, lowerId, upperId); + } + + /** + * Compares itself with another task. If the bounds overlap, sets {@link #overlaps} to true for + * both tasks. + */ + private void checkOverlapWith(TaskAndBounds other, Comparator idComp) { + if (overlaps && other.overlaps) { + return; + } + + int left = idComp.compare(lowerId, other.upperId); + int right = idComp.compare(other.lowerId, upperId); + + if (left <= 0 && right <= 0) { + overlaps = true; + other.overlaps = true; + } + } + + private static GenericRecord createRecId(Schema recIdSchema, Map bounds) + throws NoBoundMetricsException { + GenericRecord recId = GenericRecord.create(recIdSchema); + + for (Types.NestedField field : recIdSchema.columns()) { + int fieldId = field.fieldId(); + Type type = field.type(); + String name = field.name(); + @Nullable ByteBuffer value = bounds.get(fieldId); + if (value == null) { + throw new NoBoundMetricsException("Could not fetch metric value for column: " + name); + } + Object data = checkStateNotNull(Conversions.fromByteBuffer(type, value)); + recId.setField(name, data); + } + return recId; + } + + static class NoBoundMetricsException extends Exception { + public NoBoundMetricsException(String msg) { + super(msg); + } + } + } + + /** Checks if all partition fields are derived from record identifier fields. */ + private static boolean doesSpecPinRecordsToPartition(PartitionSpec spec) { + Set identifierFieldsIds = spec.schema().identifierFieldIds(); + if (spec.isUnpartitioned() || identifierFieldsIds.isEmpty()) { + return false; + } + + for (PartitionField field : spec.fields()) { + if (!identifierFieldsIds.contains(field.sourceId())) { + return false; + } + } + + return true; + } + + /** + * Helper class to batch tasks going to the same tagged PCollection. + * + *

Used to create batches of uni-directional tasks to send to {@link #UNIDIRECTIONAL_TASKS} + * tag. + * + *

Also used to create batches of large bi-directional tasks to send to {@link + * #LARGE_BIDIRECTIONAL_TASKS} tag. + * + *

A batch is emitted once it reaches {@link #splitSize()}. + * + *

Note: This is not used by small bi-directional tasks. Instead, they are emitted immediately + * to {@link #SMALL_BIDIRECTIONAL_TASKS}. + */ + static class TaskBatcher { + Map> tasks = new HashMap<>(); + long byteSize = 0L; + final long maxSplitSize; + final String tableIdentifier; + final Instant timestamp; + final OutputReceiver>> output; + int totalSplits = 0; + + TaskBatcher( + String tableIdentifier, + Long timestampMillis, + long maxSplitSize, + OutputReceiver>> output) { + this.tableIdentifier = tableIdentifier; + this.timestamp = Instant.ofEpochMilli(timestampMillis); + this.maxSplitSize = maxSplitSize; + this.output = output; + } + + boolean canTake(long sizeBytes) { + return byteSize + sizeBytes <= maxSplitSize; + } + + void add(List tasks, Table table) { + tasks.forEach(t -> add(makeTask(t, table), getLength(t))); + } + + void add(SerializableChangelogTask task, long sizeBytes) { + add( + ChangelogDescriptor.builder().setTableIdentifierString(tableIdentifier).build(), + task, + sizeBytes); + } + + void add(ChangelogDescriptor descriptor, SerializableChangelogTask task, long sizeBytes) { + if (!canTake(sizeBytes)) { + flush(); + } + byteSize += sizeBytes; + tasks.computeIfAbsent(descriptor, d -> new ArrayList<>()).add(task); + } + + void flush() { + if (tasks.isEmpty()) { + return; + } + + for (Map.Entry> entry : + tasks.entrySet()) { + ChangelogDescriptor descriptor = entry.getKey(); + List taskList = entry.getValue(); + output.outputWithTimestamp(KV.of(descriptor, taskList), timestamp); + } + + byteSize = 0; + tasks = new HashMap<>(); + totalSplits++; + } + } + + /** + * Fetch the desired split size for downstream read DoFn. We do our best to put tasks into groups + * of that size. This allows the user to control load per worker by tuning `read.split.target-size` + */ + long splitSize() { + return PropertyUtil.propertyAsLong( + checkStateNotNull(table).properties(), + TableProperties.SPLIT_SIZE, + TableProperties.SPLIT_SIZE_DEFAULT); + } + + static String name(String path) { + return Iterables.getLast(Splitter.on("-").split(path)); + } + + private void resetLocalMetrics() { + numAddedRowsTasks = 0; + numDeletedRowsTasks = 0; + numDeletedFileTasks = 0; + numUniDirTasks = 0; + numLargeBiDirTasks = 0; + numSmallBiDirTasks = 0; + numUniDirSplits = 0; + numSmallBiDirSplits = 0; + numLargeBiDirSplits = 0; + } + + private void gatherTaskTypeMetrics(SerializableChangelogTask.Type type) { + switch (type) { + case ADDED_ROWS: + numAddedRowsTasks++; + break; + case DELETED_ROWS: + numDeletedRowsTasks++; + break; + case DELETED_FILE: + numDeletedFileTasks++; + break; + } + } + + private int updateTaskCounters() { + int totalTasks = numAddedRowsTasks + numDeletedRowsTasks + numDeletedFileTasks; + numUniDirSplits = checkStateNotNull(uniBatcher).totalSplits; + totalChangelogScanTasks.inc(totalTasks); + numAddedRowsScanTasks.inc(numAddedRowsTasks); + numDeletedRowsScanTasks.inc(numDeletedRowsTasks); + numDeletedDataFileScanTasks.inc(numDeletedFileTasks); + numUniDirectionalTasks.inc(numUniDirTasks); + numSmallBiDirectionalTasks.inc(numSmallBiDirTasks); + numLargeBiDirectionalTasks.inc(numLargeBiDirTasks - numSmallBiDirTasks); + + return totalTasks; + } + + private String scanResultMessage(int totalTasks) { + StringBuilder message = new StringBuilder(); + message.append( + format( + "Snapshot %s (seq: %s) produced %s changelog tasks.", + checkStateNotNull(snapshot).snapshotId(), + checkStateNotNull(snapshot).sequenceNumber(), + totalTasks)); + if (totalTasks > 0) { + message.append("Emitted:"); + if (numUniDirTasks > 0) { + message.append( + format( + "%n\t%s splits containing %s uni-directional tasks", + numUniDirSplits, numUniDirTasks)); + } + if (numSmallBiDirTasks > 0) { + message.append( + format( + "%n\t%s splits containing %s small bi-directional tasks (for local resolution)", + numSmallBiDirSplits, numSmallBiDirTasks)); + } + if (numLargeBiDirTasks > 0) { + message.append( + format( + "%n\t%s splits containing %s large bi-directional tasks (to be shuffled)", + numLargeBiDirSplits, numLargeBiDirTasks)); + } + } + return message.toString(); + } +} 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/IncrementalChangelogSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java new file mode 100644 index 000000000000..5c45d09bab1f --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java @@ -0,0 +1,197 @@ +/* + * 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.io.iceberg.cdc.ChangelogScanner.LARGE_BIDIRECTIONAL_TASKS; +import static org.apache.beam.sdk.io.iceberg.cdc.ChangelogScanner.SMALL_BIDIRECTIONAL_TASKS; +import static org.apache.beam.sdk.io.iceberg.cdc.ChangelogScanner.UNIDIRECTIONAL_TASKS; +import static org.apache.beam.sdk.io.iceberg.cdc.ResolveChanges.DELETES; +import static org.apache.beam.sdk.io.iceberg.cdc.ResolveChanges.INSERTS; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.ReadUtils; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Redistribute; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * An Iceberg source that incrementally reads a table's changelogs, processing one snapshot at a + * time. + * + *

The streaming path uses {@link WatchForSnapshotsSdf} for proper per-snapshot watermarks. The + * bounded path creates the snapshot range up front. + */ +public class IncrementalChangelogSource extends PTransform> { + private static final Duration DEFAULT_POLL_INTERVAL = Duration.standardSeconds(60); + + private final IcebergScanConfig scanConfig; + + public IncrementalChangelogSource(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @Override + public PCollection expand(PBegin input) { + // emit one SnapshotInfo per element, with element timestamp -> snapshot commit time. + PCollection snapshots = + MoreObjects.firstNonNull(scanConfig.getStreaming(), false) + ? unboundedSnapshots(input) + : boundedSnapshots(input); + + // process one snapshot at a time and produce batches of changelog scan tasks. + // tasks are emitted to three outputs: + // 1. unidirectional tasks: we know these won't have any updates + // 2. small bidirectional tasks: these may contain an update, but the batch is small enough to + // resolve in-memory + // 2. large bidirectional tasks: may contain an update, but are too large for in-memory + // resolution. will + // need to run these output rows through a CoGBK + PCollectionTuple changelogTasks = + snapshots.apply( + "Create Changelog Tasks", + ParDo.of(new ChangelogScanner(scanConfig)) + .withOutputTags( + UNIDIRECTIONAL_TASKS, + TupleTagList.of(LARGE_BIDIRECTIONAL_TASKS).and(SMALL_BIDIRECTIONAL_TASKS))); + changelogTasks.get(UNIDIRECTIONAL_TASKS).setCoder(ChangelogScanner.OUTPUT_CODER); + changelogTasks.get(SMALL_BIDIRECTIONAL_TASKS).setCoder(ChangelogScanner.OUTPUT_CODER); + changelogTasks.get(LARGE_BIDIRECTIONAL_TASKS).setCoder(ChangelogScanner.OUTPUT_CODER); + + Schema projectedRowSchema = + IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); + + // reads UNIDIRECTIONAL and BIDIRECTIONAL tags and produces rows. + ReadFromChangelogs.CdcOutput outputRows = + changelogTasks.apply(new ReadFromChangelogs(scanConfig)); + + // Small overlapping groups get resolved entirely in memory with no shuffle. + PCollection smallBidirectionalCdcRows = + changelogTasks + .get(SMALL_BIDIRECTIONAL_TASKS) + .apply("Redistribute Small Bidirectional Changes", Redistribute.arbitrarily()) + .apply("Resolve Locally", ParDo.of(new LocalResolveDoFn(scanConfig))) + .setRowSchema(projectedRowSchema); + + // BIDIRECTIONAL records go through a CoGBK and ResolveChanges + // We window locally using a custom WindowFn based on the snapsot's commit time. Each snapshot + // exists in its own window. + // We re-window the resolved output back to GlobalWindows before the final Flatten + // to align with the other branches. + Window, TimestampedValue>> keyedWindowing = + Window., TimestampedValue>>into(new SnapshotWindowFn()) + .triggering(AfterWatermark.pastEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes(); + PCollection, TimestampedValue>> keyedInserts = + outputRows.biDirectionalInserts().apply("Window Inserts", keyedWindowing); + PCollection, TimestampedValue>> keyedDeletes = + outputRows.biDirectionalDeletes().apply("Window Deletes", keyedWindowing); + PCollection biDirectionalCdcRows = + KeyedPCollectionTuple.of(INSERTS, keyedInserts) + .and(DELETES, keyedDeletes) + .apply("CoGroupBy Primary Key", CoGroupByKey.create()) + .apply("Resolve Delete-Insert Pairs", ParDo.of(new ResolveChanges(scanConfig))) + .setRowSchema(projectedRowSchema) + .apply("Re-window to Global", Window.into(new GlobalWindows())); + + // Merge all three paths into a single output. All three are in GlobalWindows. + PCollection merged = + PCollectionList.of(outputRows.uniDirectionalRows()) + .and(smallBidirectionalCdcRows) + .and(biDirectionalCdcRows) + .apply(Flatten.pCollections()); + + // If the user configures a watermark column, restamp each record by + // that column's value. Output watermark then advances per-record rather than per-snapshot. + @Nullable String watermarkColumn = scanConfig.getWatermarkColumn(); + if (watermarkColumn != null) { + merged = + merged.apply( + "Apply Watermark Column", ParDo.of(new ApplyWatermarkColumn(watermarkColumn))); + } + + return merged.setRowSchema(projectedRowSchema); + } + + /** + * Continuously watches the Iceberg table for new snapshots via {@link WatchForSnapshotsSdf} and + * emits per snapshot. + */ + private PCollection unboundedSnapshots(PBegin input) { + Duration pollInterval = + MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); + return input + .apply("Impulse", Create.of("")) + .apply("Watch for Snapshots", ParDo.of(new WatchForSnapshotsSdf(scanConfig, pollInterval))); + } + + /** + * Reads the full snapshot range up front and emits each snapshot individually, each carrying its + * own commit time as the element timestamp. + */ + private PCollection boundedSnapshots(PBegin input) { + Table table = + scanConfig + .getCatalogConfig() + .catalog() + .loadTable(TableIdentifier.parse(scanConfig.getTableIdentifier())); + checkStateNotNull( + table.currentSnapshot(), + "Table %s does not have any snapshots to read from.", + scanConfig.getTableIdentifier()); + + @Nullable Long from = ReadUtils.getFromSnapshotExclusive(table, scanConfig); + long to = + MoreObjects.firstNonNull( + ReadUtils.getToSnapshot(table, scanConfig), table.currentSnapshot().snapshotId()); + List> timestamped = + ReadUtils.snapshotsBetween(table, scanConfig.getTableIdentifier(), from, to).stream() + .map( + s -> + TimestampedValue.of( + s.getSnapshotId(), Instant.ofEpochMilli(s.getTimestampMillis()))) + .collect(Collectors.toList()); + return input.apply("Create Snapshot Range", Create.timestamped(timestamped)); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java new file mode 100644 index 000000000000..96d08b4b8d0a --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java @@ -0,0 +1,237 @@ +/* + * 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.io.iceberg.IcebergUtils.icebergSchemaToBeamSchema; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.Type.ADDED_ROWS; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.TableCache; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.util.StructLikeUtil; +import org.apache.iceberg.util.StructProjection; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Resolves a small bi-directional changelog group entirely in memory. This is the equivalent of + * {@link ReadFromChangelogs} + {@link CoGroupByKey} + {@link ResolveChanges}. + * + *

All tasks in a changelog group belong to the same Iceberg {@link Snapshot}. The upstream + * {@link ChangelogScanner} routes here only when the total size of the bi-directional group fits + * within {@link TableProperties#SPLIT_SIZE}. + * + *

The incoming batch's overlap region has already been computed in the scanning phase by {@link + * ChangelogScanner}. In this DoFn, we just process each task and route records: + * + *

    + *
  • Records whose PK falls outside the overlap range cannot have an opposing-side match, + * so they are emitted directly with {@code INSERT} or {@code DELETE} kind. + *
  • Records whose PK falls inside the overlap range are stashed in a {@link + * StructLikeMap} keyed by PK, then resolved by {@link CdcResolver}. + *
+ */ +class LocalResolveDoFn extends DoFn>, Row> { + private final IcebergScanConfig scanConfig; + private final org.apache.beam.sdk.schemas.Schema projectedBeamSchema; + + private transient @MonotonicNonNull OverlapRange overlap; + private transient @MonotonicNonNull List nonPkFields; + private transient @MonotonicNonNull StructProjection projector; + + LocalResolveDoFn(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + this.projectedBeamSchema = icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); + } + + @Setup + public void setup() { + TableCache.setup(scanConfig); + Schema fullSchema = TableCache.get(scanConfig.getTableIdentifier()).schema(); + this.overlap = OverlapRange.forScanConfig(scanConfig); + Set pkFieldNames = new HashSet<>(overlap.recordIdSchema().identifierFieldNames()); + // The dedup logic only inspects non-PK fields, so precompute them once. + List nonPk = new ArrayList<>(); + for (Types.NestedField f : fullSchema.columns()) { + if (!pkFieldNames.contains(f.name())) { + nonPk.add(f); + } + } + this.nonPkFields = nonPk; + this.projector = StructProjection.create(fullSchema, scanConfig.getProjectedSchema()); + } + + @ProcessElement + public void process( + @Element KV> element, + OutputReceiver out) + throws IOException { + ChangelogDescriptor descriptor = element.getKey(); + Table table = TableCache.get(scanConfig.getTableIdentifier()); + OverlapRange ovl = checkStateNotNull(overlap); + + // {PK: (inserts | deletes)} for in-overlap records that need resolution. + // Records outside the overlap are emitted directly + StructLikeMap pkGroups = StructLikeMap.create(ovl.recordIdSchema().asStruct()); + + @Nullable StructLike overlapLower = ovl.toStructLike(descriptor.getOverlapLower()); + @Nullable StructLike overlapUpper = ovl.toStructLike(descriptor.getOverlapUpper()); + for (SerializableChangelogTask task : element.getValue()) { + readAndRoute(task, table, overlapLower, overlapUpper, pkGroups, out); + } + + resolveAndEmit(pkGroups, table.schema(), out); + } + + /** + * Processes a {@link SerializableChangelogTask} and routes each record. + * + *
    + *
  • Out of overlap: emit directly + *
  • Inside overlap: stash in {@code pkGroups} to resolve in {@link #resolveAndEmit} + *
+ */ + private void readAndRoute( + SerializableChangelogTask task, + Table table, + @Nullable StructLike overlapLower, + @Nullable StructLike overlapUpper, + StructLikeMap pkGroups, + OutputReceiver out) + throws IOException { + OverlapRange ovl = checkStateNotNull(overlap); + try (CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig, false)) { + for (Record rec : records) { + if (ovl.contains(rec, overlapLower, overlapUpper)) { // needs resolution + StructLike pk = StructLikeUtil.copy(ovl.recordIdProjection()); + PkGroup group = pkGroups.computeIfAbsent(pk, k -> new PkGroup()); + if (task.getType() == ADDED_ROWS) { + group.inserts.add(rec); + } else { + group.deletes.add(rec); + } + } else { // safe to emit directly + emit(rec, out); + } + } + } + } + + /** Resolves each PK group using {@link CdcResolver}. */ + private void resolveAndEmit( + StructLikeMap pkGroups, Schema fullSchema, OutputReceiver out) { + CdcResolver resolver = new RecordResolver(checkStateNotNull(nonPkFields), fullSchema); + for (PkGroup group : pkGroups.values()) { + resolver.resolve( + group.deletes, + group.inserts, + (kind, rec) -> { + // TODO: emit with proper UPDATE_BEFORE / UPDATE_AFTER / DELETE / INSERT row kinds. + emit(rec, out); + logEmit(kind, rec); + }); + } + } + + /** Resolver specialization that hashes Iceberg Record non-PK fields. */ + private static final class RecordResolver extends CdcResolver { + private final List nonPkFields; + private final Comparator nonPkComparator; + private final StructProjection left; + private final StructProjection right; + + RecordResolver(List nonPkFields, Schema recSchema) { + this.nonPkFields = nonPkFields; + Set nonPkFieldIds = + nonPkFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + this.left = StructProjection.create(recSchema, nonPkFieldIds); + this.right = StructProjection.create(recSchema, nonPkFieldIds); + this.nonPkComparator = + Comparators.forType(TypeUtil.select(recSchema, nonPkFieldIds).asStruct()); + } + + @Override + protected int nonPkHash(Record rec) { + int hash = 1; + for (Types.NestedField field : nonPkFields) { + hash = 31 * hash + Objects.hashCode(rec.getField(field.name())); + } + return hash; + } + + @Override + protected boolean nonPkEquals(Record delete, Record insert) { + return nonPkComparator.compare(left.wrap(delete), right.wrap(insert)) == 0; + } + } + + /** Debug-only logging hook so the existing CoW / update / extra prints survive the refactor. */ + private static void logEmit(CdcResolver.ChangeKind kind, Record rec) { + switch (kind) { + case UPDATE_BEFORE: + System.out.printf("[LOCAL_RESOLVE] -- UpdateBefore:%n\t%s%n", rec); + break; + case UPDATE_AFTER: + System.out.printf("[LOCAL_RESOLVE] -- UpdateAfter%n\t%s%n", rec); + break; + case DELETE: + System.out.printf("[LOCAL_RESOLVE] -- Deleted%n%s%n", rec); + break; + case INSERT: + System.out.printf("[LOCAL_RESOLVE] -- Added%n%s%n", rec); + break; + } + } + + /** Prune to get the final projected record then output as a Beam Row. */ + private void emit(Record rec, OutputReceiver out) { + StructLike projected = checkStateNotNull(projector).wrap(rec); + out.output(IcebergUtils.structToRow(projectedBeamSchema, projected)); + } + + /** Two parallel lists of inserts/deletes that share a primary key. */ + private static final class PkGroup { + final List inserts = new ArrayList<>(); + final List deletes = new ArrayList<>(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRange.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRange.java new file mode 100644 index 000000000000..efc200b4d04f --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRange.java @@ -0,0 +1,92 @@ +/* + * 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.Comparator; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.TableCache; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.util.StructProjection; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Primary-key-projection and overlap-range comparison helper. + * + *

Used by {@link LocalResolveDoFn} and {@link ReadFromChangelogs} to decide whether a record's + * PK falls within an overlap of two opposing tasks. If so, the record needs to be compared with + * others to determine if it is part of an update pair. + */ +final class OverlapRange { + private final Schema recordIdSchema; + private final StructProjection recordIdProjection; + private final Comparator idComp; + + private OverlapRange( + Schema recordIdSchema, StructProjection recordIdProjection, Comparator idComp) { + this.recordIdSchema = recordIdSchema; + this.recordIdProjection = recordIdProjection; + this.idComp = idComp; + } + + static OverlapRange forScanConfig(IcebergScanConfig scanConfig) { + Schema fullSchema = TableCache.get(scanConfig.getTableIdentifier()).schema(); + StructProjection projection = StructProjection.create(fullSchema, scanConfig.recordIdSchema()); + return new OverlapRange( + scanConfig.recordIdSchema(), projection, scanConfig.recordIdComparator()); + } + + StructProjection recordIdProjection() { + return recordIdProjection; + } + + Schema recordIdSchema() { + return recordIdSchema; + } + + /** Converts a Beam Row (overlap bound) back to an Iceberg {@link StructLike}. */ + @Nullable + StructLike toStructLike(@Nullable Row beamBound) { + if (beamBound == null) { + return null; + } + return IcebergUtils.beamRowToIcebergRecord(recordIdSchema, beamBound); + } + + /** + * Wraps the record to project its Primary Key, then checks if the PK within the overlap {@code + * [lower, upper]} (inclusive). Can be paired with a subsequent {@link #recordIdProjection()} call + * to fetch the PK value. + * + *

If either bound is null, we conservatively assume it falls within the overlap. + */ + boolean contains(Record rec, @Nullable StructLike lower, @Nullable StructLike upper) { + checkStateNotNull(recordIdProjection).wrap(rec); + + if (lower == null || upper == null) { + return true; + } + return idComp.compare(recordIdProjection, lower) >= 0 + && idComp.compare(recordIdProjection, upper) <= 0; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java new file mode 100644 index 000000000000..4b0a4f11f980 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java @@ -0,0 +1,437 @@ +/* + * 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.io.iceberg.IcebergUtils.icebergRecordToBeamRow; +import static org.apache.beam.sdk.io.iceberg.IcebergUtils.icebergSchemaToBeamSchema; +import static org.apache.beam.sdk.io.iceberg.IcebergUtils.structToRow; +import static org.apache.beam.sdk.io.iceberg.cdc.ChangelogScanner.LARGE_BIDIRECTIONAL_TASKS; +import static org.apache.beam.sdk.io.iceberg.cdc.ChangelogScanner.UNIDIRECTIONAL_TASKS; +import static org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask.Type.ADDED_ROWS; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.TableCache; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Redistribute; +import org.apache.beam.sdk.transforms.Reify; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.util.StructProjection; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A {@link PTransform} that processes batches of {@link ChangelogScanTask}s and routes them + * accordingly: + * + *

    + *
  • Records from Uni-directional batches are directly emitted, as INSERT or DELETE kind + *
  • Records from Bi-directional batches are compared against the Primary Key overlap range: + *
      + *
    • if outside the overlap, emit directly as INSERT or DELETE kind + *
    • if inside the overlap, key by (snapshot, pk) and route to downstream {@link + * CoGroupByKey} and final resolution by {@link ResolveChanges} + *
    + *
+ * + *

We first key bi-directional rows by (snapshot id, primary key) before sending to {@link + * CoGroupByKey} to ensure they stay isolated from other PKs or snapshots. Inserts are routed to + * + *

A {@link ChangelogScanTask} comes in three types: + * + *

    + *
  1. AddedRowsScanTask: Indicates records have been inserted by a new DataFile. + *
  2. DeletedRowsScanTask: Indicates records have been deleted using a DeleteFile. + *
  3. DeletedDataFileScanTask: Indicates a whole DataFile has been deleted. + *
+ * + *

Each of these types need to be processed differently. More details in {@link + * CdcReadUtils#changelogRecordsForTask}. + */ +class ReadFromChangelogs extends PTransform { + private static final Counter numAddedRowsScanTasksCompleted = + Metrics.counter(ReadFromChangelogs.class, "numAddedRowsScanTasksCompleted"); + private static final Counter numDeletedRowsScanTasksCompleted = + Metrics.counter(ReadFromChangelogs.class, "numDeletedRowsScanTasksCompleted"); + private static final Counter numDeletedDataFileScanTasksCompleted = + Metrics.counter(ReadFromChangelogs.class, "numDeletedDataFileScanTasksCompleted"); + + private static final TupleTag UNIDIRECTIONAL_ROWS = new TupleTag<>(); + private static final TupleTag, Row>> BIDIRECTIONAL_INSERTS = new TupleTag<>(); + private static final TupleTag, Row>> BIDIRECTIONAL_DELETES = new TupleTag<>(); + + private final IcebergScanConfig scanConfig; + + ReadFromChangelogs(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @Override + public CdcOutput expand(PCollectionTuple input) { + Schema fullRowSchema = scanConfig.getSchema(); + Schema projectedRowSchema = + IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); + + // === UNIDIRECTIONAL tasks === + // (i.e. only deletes, or only inserts) + // take the fast approach of just reading and emitting CDC records + PCollection uniDirectionalRows = + input + .get(UNIDIRECTIONAL_TASKS) + .apply(Redistribute.arbitrarily()) + .apply( + "Read Uni-Directional Changes", + ParDo.of(ReadDoFn.unidirectional(scanConfig)) + .withOutputTags(UNIDIRECTIONAL_ROWS, TupleTagList.empty())) + .get(UNIDIRECTIONAL_ROWS) + .setRowSchema(projectedRowSchema); + + // === BIDIRECTIONAL tasks === + // (i.e. a task group containing a mix of deletes and inserts) + // read and route records according to their PK (see class java doc) + PCollectionTuple biDirectionalRows = + input + .get(LARGE_BIDIRECTIONAL_TASKS) + .apply(Redistribute.arbitrarily()) + .apply( + "Read Bi-Directional Changes", + ParDo.of(ReadDoFn.bidirectional(scanConfig)) + .withOutputTags( + BIDIRECTIONAL_INSERTS, + TupleTagList.of(BIDIRECTIONAL_DELETES).and(UNIDIRECTIONAL_ROWS))); + // Collect pruned (non-overlapping) rows from bi-directional reader + PCollection nonOverlappingRowsFromBiDirTasks = + biDirectionalRows.get(UNIDIRECTIONAL_ROWS).setRowSchema(projectedRowSchema); + + // Flatten uni-directional rows from both sources + PCollection allUniDirectionalRows = + PCollectionList.of(uniDirectionalRows) + .and(nonOverlappingRowsFromBiDirTasks) + .apply("Flatten Uni-Directional Rows", Flatten.pCollections()); + + // Reify to preserve each record's timestamp (CoGBK overwrites timestamps with the window's + // end-of-window) + // Note: element timestamps are snapshot commit timestamp + KvCoder, Row> keyedOutputCoder = + KvCoder.of( + KvCoder.of(VarLongCoder.of(), SchemaCoder.of(scanConfig.rowIdBeamSchema())), + SchemaCoder.of(fullRowSchema)); + PCollection, TimestampedValue>> keyedInsertsWithTimestamps = + biDirectionalRows + .get(BIDIRECTIONAL_INSERTS) + .setCoder(keyedOutputCoder) + .apply("Reify INSERT Timestamps", Reify.timestampsInValue()); + PCollection, TimestampedValue>> keyedDeletesWithTimestamps = + biDirectionalRows + .get(BIDIRECTIONAL_DELETES) + .setCoder(keyedOutputCoder) + .apply("Reify DELETE Timestamps", Reify.timestampsInValue()); + + return new CdcOutput( + input.getPipeline(), + allUniDirectionalRows, + keyedInsertsWithTimestamps, + keyedDeletesWithTimestamps); + } + + public static class CdcOutput implements POutput { + private final Pipeline pipeline; + private final PCollection uniDirectionalRows; + private final PCollection, TimestampedValue>> biDirectionalInserts; + private final PCollection, TimestampedValue>> biDirectionalDeletes; + + CdcOutput( + Pipeline p, + PCollection uniDirectionalRows, + PCollection, TimestampedValue>> biDirectionalInserts, + PCollection, TimestampedValue>> biDirectionalDeletes) { + this.pipeline = p; + this.uniDirectionalRows = uniDirectionalRows; + this.biDirectionalInserts = biDirectionalInserts; + this.biDirectionalDeletes = biDirectionalDeletes; + } + + PCollection uniDirectionalRows() { + return uniDirectionalRows; + } + + PCollection, TimestampedValue>> biDirectionalInserts() { + return biDirectionalInserts; + } + + PCollection, TimestampedValue>> biDirectionalDeletes() { + return biDirectionalDeletes; + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of( + UNIDIRECTIONAL_ROWS, + uniDirectionalRows, + BIDIRECTIONAL_INSERTS, + biDirectionalInserts, + BIDIRECTIONAL_DELETES, + biDirectionalDeletes); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + } + + @DoFn.BoundedPerElement + private static class ReadDoFn + extends DoFn>, OutT> { + private final IcebergScanConfig scanConfig; + private final boolean keyedOutput; + private final Schema projectedBeamRowSchema; + private final Schema fullBeamRowSchema; + private transient @MonotonicNonNull OverlapRange overlap; + private transient @MonotonicNonNull StructProjection outputProjector; + private transient @MonotonicNonNull StructProjection pkProjector; + + /** Used for uni-directional changes. Records are output immediately as-is. */ + static ReadDoFn unidirectional(IcebergScanConfig scanConfig) { + return new ReadDoFn<>(scanConfig, false); + } + + /** + * Used for bi-directional changes. Records are keyed by (snapshot ID, primary key) and sent to + * a CoGBK. + */ + static ReadDoFn, Row>> bidirectional(IcebergScanConfig scanConfig) { + return new ReadDoFn<>(scanConfig, true); + } + + private ReadDoFn(IcebergScanConfig scanConfig, boolean keyedOutput) { + this.scanConfig = scanConfig; + this.keyedOutput = keyedOutput; + + this.projectedBeamRowSchema = icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); + this.fullBeamRowSchema = scanConfig.getSchema(); + } + + @Setup + public void setup() { + TableCache.setup(scanConfig); + this.overlap = OverlapRange.forScanConfig(scanConfig); + } + + @ProcessElement + public void process( + @Element KV> element, + RestrictionTracker tracker, + MultiOutputReceiver out) + throws IOException { + Table table = TableCache.get(scanConfig.getTableIdentifier()); + + List tasks = element.getValue(); + @Nullable Row overlapLower = element.getKey().getOverlapLower(); + @Nullable Row overlapUpper = element.getKey().getOverlapUpper(); + + for (long l = tracker.currentRestriction().getFrom(); + l < tracker.currentRestriction().getTo(); + l++) { + if (!tracker.tryClaim(l)) { + return; + } + + SerializableChangelogTask task = tasks.get((int) l); + processTaskRecords(task, overlapLower, overlapUpper, table, out); + } + } + + /** + * Processes a ChangelogScanTask and routes records accordingly: + * + *

If this DoFn is configured with {@link #unidirectional}, we simply read records and output + * directly to {@link #UNIDIRECTIONAL_ROWS}. + * + *

If this DoFn is configured with {@link #bidirectional}, we compare against the Primary Key + * overlap range. If within the overlap, we key by (snapshotId, PK) and out to either {@link + * #BIDIRECTIONAL_INSERTS} or {@link #BIDIRECTIONAL_DELETES}. Otherwise (not in overlap), we + * output the record directly to {@link #UNIDIRECTIONAL_ROWS}. + */ + private void processTaskRecords( + SerializableChangelogTask task, + @Nullable Row overlapLowerRow, + @Nullable Row overlapUpperRow, + Table table, + MultiOutputReceiver outputReceiver) + throws IOException { + OverlapRange ovl = checkStateNotNull(overlap); + @Nullable StructLike overlapLower = ovl.toStructLike(overlapLowerRow); + @Nullable StructLike overlapUpper = ovl.toStructLike(overlapUpperRow); + + TupleTag, Row>> taggedOutput = + task.getType() == ADDED_ROWS ? BIDIRECTIONAL_INSERTS : BIDIRECTIONAL_DELETES; + + Schema outputSchema = keyedOutput ? fullBeamRowSchema : projectedBeamRowSchema; + try (CloseableIterable records = + CdcReadUtils.changelogRecordsForTask(task, table, scanConfig, !keyedOutput)) { + for (Record rec : records) { + // uni-directional -- just output records (they are already projected by read pushdown) + if (!keyedOutput) { + Row row = icebergRecordToBeamRow(projectedBeamRowSchema, rec); + outputReceiver.get(UNIDIRECTIONAL_ROWS).output(row); + continue; + } + + // bi-directional -- compare overlap + if (ovl.contains(rec, overlapLower, overlapUpper)) { + // inside overlap -- read full row and output KV + Row row = icebergRecordToBeamRow(outputSchema, rec); + Row pk = structToRow(scanConfig.rowIdBeamSchema(), pkProjector().wrap(rec)); + long snapshotId = task.getCommitSnapshotId(); + outputReceiver.get(taggedOutput).output(KV.of(KV.of(snapshotId, pk), row)); + + System.out.printf( + "[LARGE BIIDIRECTIONAL OVERLAP] -- %s(%s)%n%s%n", + getKind(task.getType()), snapshotId, row); + } else { + // outside overlap -- get projected record and output + StructLike projected = outputProjector().wrap(rec); + Row row = structToRow(projectedBeamRowSchema, projected); + System.out.printf( + "[LARGE BIIDIRECTIONAL NO-OVERLAP] -- %s(%s)%n%s%n", + getKind(task.getType()), task.getCommitSnapshotId(), row); + outputReceiver.get(UNIDIRECTIONAL_ROWS).output(row); + } + } + } + + trackMetrics(task.getType()); + } + + private StructProjection outputProjector() { + if (outputProjector == null) { + outputProjector = + StructProjection.create( + TableCache.get(scanConfig.getTableIdentifier()).schema(), + scanConfig.getProjectedSchema()); + } + return outputProjector; + } + + private StructProjection pkProjector() { + if (pkProjector == null) { + pkProjector = + StructProjection.create( + TableCache.get(scanConfig.getTableIdentifier()).schema(), + scanConfig.recordIdSchema()); + } + return pkProjector; + } + + private void trackMetrics(SerializableChangelogTask.Type type) { + switch (type) { + case ADDED_ROWS: + numAddedRowsScanTasksCompleted.inc(); + break; + case DELETED_ROWS: + numDeletedRowsScanTasksCompleted.inc(); + break; + case DELETED_FILE: + numDeletedDataFileScanTasksCompleted.inc(); + break; + } + } + + private String getKind(SerializableChangelogTask.Type taskType) { + switch (taskType) { + case ADDED_ROWS: + return "INSERT"; + case DELETED_ROWS: + return "DELETE"; + case DELETED_FILE: + default: + return "DELETE-DF"; + } + } + + @GetSize + public double getSize( + @Element KV> element, + @Restriction OffsetRange restriction) { + // TODO(ahmedabu98): this is just the compressed DataFile byte size. find a way to make a + // better byte size estimate + long size = 0; + + for (long l = restriction.getFrom(); l < restriction.getTo(); l++) { + size += element.getValue().get((int) l).getDataFile().getFileSizeInBytes(); + } + + return size; + } + + @GetInitialRestriction + public OffsetRange getInitialRange( + @Element KV> element) { + return new OffsetRange(0, element.getValue().size()); + } + + // commenting out for now because i think doing max split will lead to OOMs, + // each thread will try to buffer its task in memory at the same time + // @SplitRestriction + // public void splitRestriction( + // @Restriction OffsetRange restriction, OutputReceiver out) { + // // Split into individual tasks for maximum initial parallelism + // for (long i = restriction.getFrom(); i < restriction.getTo(); i++) { + // out.output(new OffsetRange(i, i + 1)); + // } + // } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java new file mode 100644 index 000000000000..52964b6a5f69 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java @@ -0,0 +1,146 @@ +/* + * 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.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.util.RowFilter; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; + +/** + * Receives a {@link CoGbkResult} containing inserts and deletes sharing the same snapshot ID and + * Primary Key, and uses {@link CdcResolver} to identify logical updates. + * + *

Input elements are pre-prepared with reified timestamps. This is because CoGroupByKey assigns + * all elements in a window with the same timestamp, erasing individual record timestamps. This DoFn + * enriches the record with the reified value to preserve its timestamp. TODO(ahmedabu98): is this ^ + * necessary anymore, now that we window each group by its snapshot commit timestamp? All rows + * coming from a group should all have the same timestamp at this point. And we only potentially + * assign watermark column timestamp downstream from this DoFn. + */ +public class ResolveChanges extends DoFn, CoGbkResult>, Row> { + public static final TupleTag> DELETES = new TupleTag<>() {}; + public static final TupleTag> INSERTS = new TupleTag<>() {}; + private final RowFilter rowFilter; + + ResolveChanges(IcebergScanConfig scanConfig) { + this.rowFilter = + new RowFilter(scanConfig.getSchema()) + .keep( + scanConfig.getProjectedSchema().columns().stream() + .map(Types.NestedField::name) + .collect(Collectors.toList())); + } + + @ProcessElement + public void processElement( + @Element KV, CoGbkResult> element, OutputReceiver out) { + Row primaryKey = element.getKey().getValue(); + Set pkFields = new HashSet<>(primaryKey.getSchema().getFieldNames()); + CoGbkResult result = element.getValue(); + + // should be okay to materialize these lists. a PK collision will likely be a handful of records + // at most + List> deletes = Lists.newArrayList(result.getAll(DELETES)); + List> inserts = Lists.newArrayList(result.getAll(INSERTS)); + // TODO(ahmedabu98): do we need to sort anymore? all records should have the same timestamp now. + deletes.sort(Comparator.comparing(TimestampedValue::getTimestamp)); + inserts.sort(Comparator.comparing(TimestampedValue::getTimestamp)); + + new RowResolver(pkFields) + .resolve( + deletes, + inserts, + (kind, tv) -> { + // TODO: emit with proper UPDATE_BEFORE / UPDATE_AFTER / DELETE / INSERT row kinds. + Row projectedRow = rowFilter.filter(tv.getValue()); + out.outputWithTimestamp(projectedRow, tv.getTimestamp()); + logEmit(kind, tv); + }); + } + + private static final class RowResolver extends CdcResolver> { + private final Set pkFields; + + RowResolver(Set pkFields) { + this.pkFields = pkFields; + } + + @Override + protected int nonPkHash(TimestampedValue element) { + int hash = 1; + for (String field : element.getValue().getSchema().getFieldNames()) { + if (pkFields.contains(field)) { + continue; + } + hash = 31 * hash + Objects.hashCode(element.getValue().getValue(field)); + } + return hash; + } + + @Override + protected boolean nonPkEquals(TimestampedValue delete, TimestampedValue insert) { + Schema schema = insert.getValue().getSchema(); + for (String field : schema.getFieldNames()) { + // we already know PK values are equal + if (pkFields.contains(field)) { + continue; + } + // return early if two values are not equal + if (!Row.Equals.deepEquals( + insert.getValue().getValue(field), + delete.getValue().getValue(field), + schema.getField(field).getType())) { + return false; + } + } + return true; + } + } + + /** Debug-only logging hook so the existing CoW / update / extra prints survive the refactor. */ + private static void logEmit(CdcResolver.ChangeKind kind, TimestampedValue tv) { + switch (kind) { + case UPDATE_BEFORE: + System.out.printf("[BIDIRECTIONAL] -- UpdateBefore:%n\t%s%n", tv); + break; + case UPDATE_AFTER: + System.out.printf("[BIDIRECTIONAL] -- UpdateAfter%n\t%s%n", tv); + break; + case DELETE: + System.out.printf("[BIDIRECTIONAL] -- Deleted%n%s%n", tv); + break; + case INSERT: + System.out.printf("[BIDIRECTIONAL] -- Inserted%n%s%n", tv); + break; + } + } +} 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/SnapshotWindowFn.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFn.java new file mode 100644 index 000000000000..f9985c136264 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFn.java @@ -0,0 +1,87 @@ +/* + * 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.Collection; +import java.util.Collections; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * A {@link WindowFn} that assigns each element to a 1-millisecond {@link IntervalWindow} anchored + * at the element's event timestamp. + * + *

We set the element's timestamp as its snapshot commit timestamp. All tasks/records from the + * same snapshot land in the same window. + * + *

With the per-snapshot watermark from {@link WatchForSnapshotsSdf}, the CoGroupByKey fires when + * a snapshot is fully drained. The watermark advances past the snapshot's commit time only after + * every downstream stage has finished processing that snapshot's records. + * + *

Two snapshots committed within the same millisecond may collapse into the same window. But + * that's okay because {@link ReadFromChangelogs} includes snapshot id in the key before routing to + * the CoGBK, so it won't produce incorrect joins. + */ +public class SnapshotWindowFn extends NonMergingWindowFn { + private static final Duration WINDOW_LENGTH = Duration.millis(1); + + @Override + public Collection assignWindows(AssignContext c) { + Instant ts = c.timestamp(); + return Collections.singletonList(new IntervalWindow(ts, ts.plus(WINDOW_LENGTH))); + } + + @Override + public boolean isCompatible(WindowFn other) { + return other instanceof SnapshotWindowFn; + } + + @Override + public Coder windowCoder() { + return IntervalWindow.getCoder(); + } + + @Override + public WindowMappingFn getDefaultWindowMappingFn() { + // Just return a window covering the main-input window's end timestamp. + return new WindowMappingFn<>() { + @Override + public IntervalWindow getSideInputWindow(BoundedWindow mainWindow) { + Instant end = mainWindow.maxTimestamp(); + return new IntervalWindow(end, end.plus(WINDOW_LENGTH)); + } + }; + } + + @Override + public boolean equals(@Nullable Object obj) { + return obj instanceof SnapshotWindowFn; + } + + @Override + public int hashCode() { + return SnapshotWindowFn.class.hashCode(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java new file mode 100644 index 000000000000..8642a0f0b5fa --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java @@ -0,0 +1,286 @@ +/* + * 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.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.ReadUtils; +import org.apache.beam.sdk.io.iceberg.SnapshotInfo; +import org.apache.beam.sdk.io.iceberg.TableCache; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Gauge; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.splittabledofn.GrowableOffsetRangeTracker; +import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; +import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.Manual; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.util.SnapshotUtil; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SplittableDoFn that watches an Iceberg table for new {@link Snapshot}s and emits them one at a + * time, advancing the watermark per snapshot. Each snapshot is later processed by {@link + * ChangelogScanner}. + * + *

The restriction tracks Snapshots via their sequence numbers, which are monotonic, + * unlike snapshot IDs. The initial range starts at the sequence number of the user-configured + * starting snapshot (or one if none configured) and runs to {@link Long#MAX_VALUE}. Each call to + * {@code @ProcessElement} claims the sequence numbers of newly discovered snapshots in + * chronological order. + * + *

Uses a {@link Manual} watermark estimator. After emitting a snapshot, the watermark is set to + * that snapshot's commit time. On empty polls, the watermark is bumped to {@code now() - + * MAX_SNAPSHOT_DISCOVERY_DELAY} to prevent downstream windows from stalling indefinitely during + * quiet periods. + */ +@DoFn.UnboundedPerElement +class WatchForSnapshotsSdf extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(WatchForSnapshotsSdf.class); + private static final Counter snapshotsEmitted = + Metrics.counter(WatchForSnapshotsSdf.class, "snapshotsEmitted"); + private static final Gauge latestEmittedSnapshotId = + Metrics.gauge(WatchForSnapshotsSdf.class, "latestEmittedSnapshotId"); + // TODO(ahmedabu98): consider exposing this as a config option + private static final Duration MAX_SNAPSHOT_DISCOVERY_DELAY = Duration.standardMinutes(5); + private static final Long POLL_FOREVER = Long.MAX_VALUE; + + private final IcebergScanConfig scanConfig; + private final Duration pollInterval; + + WatchForSnapshotsSdf(IcebergScanConfig scanConfig, Duration pollInterval) { + this.scanConfig = scanConfig; + this.pollInterval = pollInterval; + } + + @GetInitialRestriction + public OffsetRange initialRestriction() { + TableCache.setup(scanConfig); + Table table = TableCache.getRefreshed(scanConfig.getTableIdentifier()); + + long toSnapshotExclusiveSeq = POLL_FOREVER; + @Nullable Long toSnapshotId = ReadUtils.getToSnapshot(table, scanConfig); + if (toSnapshotId != null) { + toSnapshotExclusiveSeq = + Preconditions.checkStateNotNull( + table.snapshot(toSnapshotId), + "Configured end snapshot %s does not exist", + toSnapshotId) + .sequenceNumber() + + 1; + } + + @Nullable Long fromSnapshotInclusiveId = ReadUtils.getFromSnapshotInclusive(table, scanConfig); + long fromSnapshotInclusiveSeq; + if (fromSnapshotInclusiveId == null) { + fromSnapshotInclusiveSeq = 1L; // sequence numbers start at 1 + } else { + Snapshot fromSnapshotInclusive = + Preconditions.checkArgumentNotNull( + table.snapshot(fromSnapshotInclusiveId), + "The specified starting snapshot %s does not exist", + fromSnapshotInclusiveId); + fromSnapshotInclusiveSeq = fromSnapshotInclusive.sequenceNumber(); + + boolean sameLineage = + toSnapshotId == null + ? SnapshotUtil.isAncestorOf(table, fromSnapshotInclusiveId) + : SnapshotUtil.isAncestorOf(table, toSnapshotId, fromSnapshotInclusiveId); + checkArgument( + sameLineage, + "Configured starting snapshot %s is not an ancestor of %s", + fromSnapshotInclusiveId, + toSnapshotId == null ? "the current table" : "end snapshot " + toSnapshotId); + } + + return new OffsetRange( + fromSnapshotInclusiveSeq, Math.max(fromSnapshotInclusiveSeq, toSnapshotExclusiveSeq)); + } + + @NewTracker + public RestrictionTracker newTracker(@Restriction OffsetRange restriction) { + if (restriction.getTo() == POLL_FOREVER) { + return new GrowableOffsetRangeTracker( + restriction.getFrom(), this::estimateCurrentRangeEndExclusive); + } + + return new OffsetRangeTracker(restriction); + } + + private long estimateCurrentRangeEndExclusive() { + TableCache.setup(scanConfig); + Table table = TableCache.get(scanConfig.getTableIdentifier()); + + @Nullable Long toSnapshotId = ReadUtils.getToSnapshot(table, scanConfig); + if (toSnapshotId != null) { + @Nullable Snapshot toSnapshot = table.snapshot(toSnapshotId); + return toSnapshot == null ? Long.MIN_VALUE : toSnapshot.sequenceNumber() + 1; + } + + @Nullable Snapshot current = table.currentSnapshot(); + return current == null ? Long.MIN_VALUE : current.sequenceNumber() + 1; + } + + @GetRestrictionCoder + public Coder restrictionCoder() { + return new OffsetRange.Coder(); + } + + @GetInitialWatermarkEstimatorState + public Instant initialWatermarkState() { + return BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + @NewWatermarkEstimator + public ManualWatermarkEstimator newWatermarkEstimator( + @WatermarkEstimatorState Instant state) { + return new Manual(state); + } + + @ProcessElement + public ProcessContinuation process( + RestrictionTracker tracker, + ManualWatermarkEstimator watermark, + OutputReceiver out) { + TableCache.setup(scanConfig); + Table table = TableCache.getRefreshed(scanConfig.getTableIdentifier()); + + @Nullable Long userToSnapshotId = ReadUtils.getToSnapshot(table, scanConfig); + boolean bounded = userToSnapshotId != null; + + @Nullable Snapshot current = table.currentSnapshot(); + if (current == null) { + // no snapshots yet. + LOG.info("Skipping scan: table is empty with no snapshots yet"); + return pauseOrStop(watermark, bounded); + } + + // Resolve the upper bound: user-specified bounded mode, or "current" for unbounded. + long toSnapshotId; + long toSnapshotSeq; + if (userToSnapshotId != null) { + toSnapshotId = userToSnapshotId; + toSnapshotSeq = + Preconditions.checkStateNotNull( + table.snapshot(userToSnapshotId), + "Configured toSnapshotId %s does not exist", + userToSnapshotId) + .sequenceNumber(); + } else { + toSnapshotId = current.snapshotId(); + toSnapshotSeq = current.sequenceNumber(); + } + + long nextSeqInclusive = tracker.currentRestriction().getFrom(); + if (toSnapshotSeq < nextSeqInclusive) { + // Nothing new since last poll. + LOG.info("Skipping scan: nothing new since last poll."); + return pauseOrStop(watermark, bounded); + } + + // Collect snapshots in [nextSeqInclusive, toSnapshotSeq] chronologically + String tableId = scanConfig.getTableIdentifier(); + List fresh = snapshotsAfter(table, tableId, nextSeqInclusive, toSnapshotId); + LOG.info("Collected snapshots: {}", fresh); + + for (SnapshotInfo snap : fresh) { + if (!tracker.tryClaim(snap.getSequenceNumber())) { + return ProcessContinuation.stop(); + } + Instant ts = Instant.ofEpochMilli(snap.getTimestampMillis()); + out.outputWithTimestamp(snap.getSnapshotId(), ts); + + if (watermark.currentWatermark().isBefore(ts)) { + watermark.setWatermark(ts); + } + snapshotsEmitted.inc(); + latestEmittedSnapshotId.set(snap.getSnapshotId()); + LOG.info( + "Emitted snapshot {} (sequence id: {}, commit ts: {})", + snap.getSnapshotId(), + snap.getSequenceNumber(), + ts); + } + + return pauseOrStop(watermark, bounded); + } + + /** + * On an empty poll, bump the watermark to {@code now() - MAX_SNAPSHOT_DISCOVERY_DELAY} so + * downstream windows can still fire. Returns {@code stop()} when end snapshot has been reached, + * otherwise {@code resume()} after the poll interval. + */ + private ProcessContinuation pauseOrStop( + ManualWatermarkEstimator watermark, boolean bounded) { + Duration delay = + MoreObjects.firstNonNull( + scanConfig.getMaxSnapshotDiscoveryDelay(), MAX_SNAPSHOT_DISCOVERY_DELAY); + Instant idleWatermark = Instant.now().minus(delay); + if (watermark.currentWatermark().isBefore(idleWatermark)) { + LOG.info( + "Sitting idle for {} seconds. Bumping watermark to {}", + TimeUnit.MILLISECONDS.toSeconds( + Instant.now().getMillis() - watermark.currentWatermark().getMillis()), + idleWatermark); + watermark.setWatermark(idleWatermark); + } + return bounded + ? ProcessContinuation.stop() + : ProcessContinuation.resume().withResumeDelay(pollInterval); + } + + /** + * Returns snapshots with sequence number in {@code [nextSeqInclusive, toSnapshotSeq]}, keyed off + * the lineage ending at {@code toSnapshotId}. + */ + @SuppressWarnings("return") // ancestorsOf accepts null returns as a "stop" signal + static List snapshotsAfter( + Table table, String tableIdentifier, long nextSeqInclusive, long toSnapshotId) { + + List snapshots = new ArrayList<>(); + // ancestorsOf returns an iterable of snapshots looking backwards. + // we'll need to reverse it to process snapshots chronologically. + for (Snapshot snapshot : + SnapshotUtil.ancestorsOf( + toSnapshotId, snapshotId -> snapshotId != null ? table.snapshot(snapshotId) : null)) { + if (snapshot.sequenceNumber() < nextSeqInclusive) { + break; + } + snapshots.add(SnapshotInfo.fromSnapshot(snapshot, tableIdentifier)); + } + Collections.reverse(snapshots); + return snapshots; + } +} 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/ReadUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java index 73a0fd19e893..6287a6e06197 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 @@ -40,7 +40,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 +74,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); } 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/TestFixtures.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java index 820cab5ddcd6..bdaa2770c5f8 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java @@ -21,12 +21,14 @@ import static org.apache.iceberg.types.Types.NestedField.required; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; 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.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.apache.iceberg.Schema; import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Types; @@ -34,7 +36,9 @@ public class TestFixtures { public static final Schema SCHEMA = new Schema( - required(1, "id", Types.LongType.get()), optional(2, "data", Types.StringType.get())); + Arrays.asList( + required(1, "id", Types.LongType.get()), optional(2, "data", Types.StringType.get())), + Sets.newHashSet(1)); public static final Schema NESTED_SCHEMA = new Schema( 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..32d0893fa06f --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReaderTest.java @@ -0,0 +1,302 @@ +/* + * 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.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.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.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(); + + // ------------------------------------------------------------------------ + // Test infrastructure + // ------------------------------------------------------------------------ + + /** {@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; + } + + @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. Unused arms return empty. */ + private static class StubLoader implements DeleteLoader { + private final PositionDeleteIndex posIndex; + private final StructLikeSet eqSet; + private int posLoadCount = 0; + private int eqLoadCount = 0; + + StubLoader(PositionDeleteIndex posIndex, StructLikeSet eqSet) { + this.posIndex = posIndex; + this.eqSet = eqSet; + } + + @Override + public PositionDeleteIndex loadPositionDeletes(Iterable files, CharSequence path) { + posLoadCount++; + return posIndex; + } + + @Override + public StructLikeSet loadEqualityDeletes(Iterable files, Schema schema) { + eqLoadCount++; + return eqSet; + } + } + + /** 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; + } + + /** 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() { + StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); + StubDeleteReader 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() { + StubLoader loader = new StubLoader(posIndexOf(1L, 3L), eqSetOfIds()); + StubDeleteReader 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() { + StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds(2, 4)); + StubDeleteReader 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() { + StubLoader loader = new StubLoader(posIndexOf(0L, 4L), eqSetOfIds(2, 4)); + StubDeleteReader 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); + StubDeleteReader 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)); + StubDeleteReader 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); + } +} From 960876840a64bf5e0c04fec70f6a034d88134e56 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 24 May 2026 13:09:00 -0400 Subject: [PATCH 2/9] cleanup --- .../main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java | 2 +- .../main/java/org/apache/beam/sdk/io/iceberg/TableCache.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java index b51e5be9a5a4..bab5405cd4a5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java @@ -78,7 +78,7 @@ public Row toRow() { private static @MonotonicNonNull SchemaCoder coder; private static @MonotonicNonNull Schema schema; - public static SchemaCoder getCoder() { + static SchemaCoder getCoder() { if (coder == null) { initSchemaAndCoder(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java index e37157c6ddfe..d9d8802e2b49 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java @@ -33,7 +33,6 @@ import org.apache.iceberg.catalog.TableIdentifier; /** Utility to fetch and cache Iceberg {@link Table}s. */ -// TODO(ahmedabu98): add periodic refresh public class TableCache { private static final Map CATALOG_CACHE = new ConcurrentHashMap<>(); private static final LoadingCache INTERNAL_CACHE = From 61403ba2a29575e0bb9cd3e9cc71a7251a17b4af Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 24 May 2026 15:10:15 -0400 Subject: [PATCH 3/9] switch to real ValueKind --- .../beam/sdk/io/iceberg/cdc/CdcResolver.java | 28 +++++++------------ .../sdk/io/iceberg/cdc/LocalResolveDoFn.java | 18 +++++++----- .../io/iceberg/cdc/ReadFromChangelogs.java | 15 +++++++--- .../sdk/io/iceberg/cdc/ResolveChanges.java | 6 ++-- 4 files changed, 35 insertions(+), 32 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java index 7fe7983613b6..0323c621e169 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolver.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.function.BiConsumer; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.ValueKind; import org.apache.iceberg.data.Record; import org.checkerframework.checker.nullness.qual.Nullable; @@ -47,20 +48,11 @@ * and rewrites the whole file (minus some records that are actually marked for deletion). * Unchanged records are no-ops and should not be mistaken for updates. *

  • Walk the remaining deletes and inserts, emitting matched pairs as {@link - * ChangeKind#UPDATE_BEFORE} / {@link ChangeKind#UPDATE_AFTER}. - *
  • Emit any unmatched extras as {@link ChangeKind#DELETE} / {@link ChangeKind#INSERT}. + * ValueKind#UPDATE_BEFORE} / {@link ValueKind#UPDATE_AFTER}. + *
  • Emit any unmatched extras as {@link ValueKind#DELETE} / {@link ValueKind#INSERT}. * */ abstract class CdcResolver { - - // TODO: replace this with actual ValueKind when it lands - enum ChangeKind { - INSERT, - DELETE, - UPDATE_BEFORE, - UPDATE_AFTER - } - /** Hashes the non-PK fields of an element. Used as the index for O(n+m) CoW deduplication. */ protected abstract int nonPkHash(T element); @@ -77,7 +69,7 @@ enum ChangeKind { * *

    Both input lists are inspected in their given order. */ - final void resolve(List deletes, List inserts, BiConsumer emit) { + final void resolve(List deletes, List inserts, BiConsumer emit) { boolean hasDeletes = !deletes.isEmpty(); boolean hasInserts = !inserts.isEmpty(); @@ -125,8 +117,8 @@ && nonPkEquals(deletes.get(deleteIdx), inserts.get(insertIdx))) { } if (d < deletes.size() && i < inserts.size()) { - emit.accept(ChangeKind.UPDATE_BEFORE, deletes.get(d)); - emit.accept(ChangeKind.UPDATE_AFTER, inserts.get(i)); + emit.accept(ValueKind.UPDATE_BEFORE, deletes.get(d)); + emit.accept(ValueKind.UPDATE_AFTER, inserts.get(i)); d++; i++; } @@ -135,23 +127,23 @@ && nonPkEquals(deletes.get(deleteIdx), inserts.get(insertIdx))) { // emit unmatched extras as DELETE / INSERT. while (d < deletes.size()) { if (!dupDeletes[d]) { - emit.accept(ChangeKind.DELETE, deletes.get(d)); + emit.accept(ValueKind.DELETE, deletes.get(d)); } d++; } while (i < inserts.size()) { if (!dupInserts[i]) { - emit.accept(ChangeKind.INSERT, inserts.get(i)); + emit.accept(ValueKind.INSERT, inserts.get(i)); } i++; } } else if (hasInserts) { for (T r : inserts) { - emit.accept(ChangeKind.INSERT, r); + emit.accept(ValueKind.INSERT, r); } } else if (hasDeletes) { for (T r : deletes) { - emit.accept(ChangeKind.DELETE, r); + emit.accept(ValueKind.DELETE, r); } } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java index 96d08b4b8d0a..54b6f9460507 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFn.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.ValueKind; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; @@ -139,19 +140,21 @@ private void readAndRoute( OutputReceiver out) throws IOException { OverlapRange ovl = checkStateNotNull(overlap); + boolean isInsert = task.getType() == ADDED_ROWS; try (CloseableIterable records = CdcReadUtils.changelogRecordsForTask(task, table, scanConfig, false)) { for (Record rec : records) { if (ovl.contains(rec, overlapLower, overlapUpper)) { // needs resolution StructLike pk = StructLikeUtil.copy(ovl.recordIdProjection()); PkGroup group = pkGroups.computeIfAbsent(pk, k -> new PkGroup()); - if (task.getType() == ADDED_ROWS) { + if (isInsert) { group.inserts.add(rec); } else { group.deletes.add(rec); } } else { // safe to emit directly - emit(rec, out); + emit(rec, isInsert ? ValueKind.INSERT : ValueKind.DELETE, out); + logEmit(isInsert ? ValueKind.INSERT : ValueKind.DELETE, rec); } } } @@ -166,8 +169,7 @@ private void resolveAndEmit( group.deletes, group.inserts, (kind, rec) -> { - // TODO: emit with proper UPDATE_BEFORE / UPDATE_AFTER / DELETE / INSERT row kinds. - emit(rec, out); + emit(rec, kind, out); logEmit(kind, rec); }); } @@ -206,7 +208,7 @@ protected boolean nonPkEquals(Record delete, Record insert) { } /** Debug-only logging hook so the existing CoW / update / extra prints survive the refactor. */ - private static void logEmit(CdcResolver.ChangeKind kind, Record rec) { + private static void logEmit(ValueKind kind, Record rec) { switch (kind) { case UPDATE_BEFORE: System.out.printf("[LOCAL_RESOLVE] -- UpdateBefore:%n\t%s%n", rec); @@ -224,9 +226,11 @@ private static void logEmit(CdcResolver.ChangeKind kind, Record rec) { } /** Prune to get the final projected record then output as a Beam Row. */ - private void emit(Record rec, OutputReceiver out) { + private void emit(Record rec, ValueKind kind, OutputReceiver out) { StructLike projected = checkStateNotNull(projector).wrap(rec); - out.output(IcebergUtils.structToRow(projectedBeamSchema, projected)); + out.builder(IcebergUtils.structToRow(projectedBeamSchema, projected)) + .setValueKind(kind) + .output(); } /** Two parallel lists of inserts/deletes that share a primary key. */ diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java index 4b0a4f11f980..d38a987e5650 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java @@ -58,6 +58,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.ValueKind; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.ChangelogScanTask; import org.apache.iceberg.StructLike; @@ -316,8 +317,10 @@ private void processTaskRecords( @Nullable StructLike overlapLower = ovl.toStructLike(overlapLowerRow); @Nullable StructLike overlapUpper = ovl.toStructLike(overlapUpperRow); + boolean isInsert = task.getType() == ADDED_ROWS; TupleTag, Row>> taggedOutput = - task.getType() == ADDED_ROWS ? BIDIRECTIONAL_INSERTS : BIDIRECTIONAL_DELETES; + isInsert ? BIDIRECTIONAL_INSERTS : BIDIRECTIONAL_DELETES; + ValueKind kind = isInsert ? ValueKind.INSERT : ValueKind.DELETE; Schema outputSchema = keyedOutput ? fullBeamRowSchema : projectedBeamRowSchema; try (CloseableIterable records = @@ -326,7 +329,7 @@ private void processTaskRecords( // uni-directional -- just output records (they are already projected by read pushdown) if (!keyedOutput) { Row row = icebergRecordToBeamRow(projectedBeamRowSchema, rec); - outputReceiver.get(UNIDIRECTIONAL_ROWS).output(row); + outputReceiver.get(UNIDIRECTIONAL_ROWS).builder(row).setValueKind(kind).output(); continue; } @@ -336,7 +339,11 @@ private void processTaskRecords( Row row = icebergRecordToBeamRow(outputSchema, rec); Row pk = structToRow(scanConfig.rowIdBeamSchema(), pkProjector().wrap(rec)); long snapshotId = task.getCommitSnapshotId(); - outputReceiver.get(taggedOutput).output(KV.of(KV.of(snapshotId, pk), row)); + outputReceiver + .get(taggedOutput) + .builder(KV.of(KV.of(snapshotId, pk), row)) + .setValueKind(kind) + .output(); System.out.printf( "[LARGE BIIDIRECTIONAL OVERLAP] -- %s(%s)%n%s%n", @@ -348,7 +355,7 @@ private void processTaskRecords( System.out.printf( "[LARGE BIIDIRECTIONAL NO-OVERLAP] -- %s(%s)%n%s%n", getKind(task.getType()), task.getCommitSnapshotId(), row); - outputReceiver.get(UNIDIRECTIONAL_ROWS).output(row); + outputReceiver.get(UNIDIRECTIONAL_ROWS).builder(row).setValueKind(kind).output(); } } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java index 52964b6a5f69..bf792f268c02 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.ValueKind; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; @@ -80,9 +81,8 @@ public void processElement( deletes, inserts, (kind, tv) -> { - // TODO: emit with proper UPDATE_BEFORE / UPDATE_AFTER / DELETE / INSERT row kinds. Row projectedRow = rowFilter.filter(tv.getValue()); - out.outputWithTimestamp(projectedRow, tv.getTimestamp()); + out.builder(projectedRow).setValueKind(kind).setTimestamp(tv.getTimestamp()).output(); logEmit(kind, tv); }); } @@ -127,7 +127,7 @@ protected boolean nonPkEquals(TimestampedValue delete, TimestampedValue tv) { + private static void logEmit(ValueKind kind, TimestampedValue tv) { switch (kind) { case UPDATE_BEFORE: System.out.printf("[BIDIRECTIONAL] -- UpdateBefore:%n\t%s%n", tv); From 70a12ede3a1acbbd3196cc3f5fb1bab59bae9a9d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 24 May 2026 17:47:21 -0400 Subject: [PATCH 4/9] small fixes --- .../sdk/io/iceberg/IcebergScanConfig.java | 3 +- .../cdc/IncrementalChangelogSource.java | 8 +-- .../io/iceberg/cdc/ReadFromChangelogs.java | 22 ++++--- .../sdk/io/iceberg/cdc/ResolveChanges.java | 57 ++++++++----------- 4 files changed, 40 insertions(+), 50 deletions(-) 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 8d98197caf54..c7cfacc5d124 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 @@ -470,13 +470,14 @@ void validate(Table table) { field != null, error("'watermark_column' refers to unknown column: %s"), watermarkColumn); checkArgument( field.isRequired(), - error("'watermark_column' refers to a nullable column: %s"), + error("'watermark_column' needs to be a non-nullable column: %s"), watermarkColumn); checkArgument( field.type().typeId() == TIMESTAMP || field.type().typeId() == LONG, error("'watermark_column' must be a timestamp-typed column, but '%s' has type %s"), watermarkColumn, field.type().typeId()); + checkArgumentNotNull(getProjectedSchema().findField(watermarkColumn), "'watermark_column' column should not be dropped."); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java index 5c45d09bab1f..003465bbfe78 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java @@ -118,14 +118,14 @@ public PCollection expand(PBegin input) { // exists in its own window. // We re-window the resolved output back to GlobalWindows before the final Flatten // to align with the other branches. - Window, TimestampedValue>> keyedWindowing = - Window., TimestampedValue>>into(new SnapshotWindowFn()) + Window, Row>> keyedWindowing = + Window., Row>>into(new SnapshotWindowFn()) .triggering(AfterWatermark.pastEndOfWindow()) .withAllowedLateness(Duration.ZERO) .discardingFiredPanes(); - PCollection, TimestampedValue>> keyedInserts = + PCollection, Row>> keyedInserts = outputRows.biDirectionalInserts().apply("Window Inserts", keyedWindowing); - PCollection, TimestampedValue>> keyedDeletes = + PCollection, Row>> keyedDeletes = outputRows.biDirectionalDeletes().apply("Window Deletes", keyedWindowing); PCollection biDirectionalCdcRows = KeyedPCollectionTuple.of(INSERTS, keyedInserts) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java index d38a987e5650..bcf9522faf68 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java @@ -165,16 +165,14 @@ public CdcOutput expand(PCollectionTuple input) { KvCoder.of( KvCoder.of(VarLongCoder.of(), SchemaCoder.of(scanConfig.rowIdBeamSchema())), SchemaCoder.of(fullRowSchema)); - PCollection, TimestampedValue>> keyedInsertsWithTimestamps = + PCollection, Row>> keyedInsertsWithTimestamps = biDirectionalRows .get(BIDIRECTIONAL_INSERTS) - .setCoder(keyedOutputCoder) - .apply("Reify INSERT Timestamps", Reify.timestampsInValue()); - PCollection, TimestampedValue>> keyedDeletesWithTimestamps = + .setCoder(keyedOutputCoder); + PCollection, Row>> keyedDeletesWithTimestamps = biDirectionalRows .get(BIDIRECTIONAL_DELETES) - .setCoder(keyedOutputCoder) - .apply("Reify DELETE Timestamps", Reify.timestampsInValue()); + .setCoder(keyedOutputCoder); return new CdcOutput( input.getPipeline(), @@ -186,14 +184,14 @@ public CdcOutput expand(PCollectionTuple input) { public static class CdcOutput implements POutput { private final Pipeline pipeline; private final PCollection uniDirectionalRows; - private final PCollection, TimestampedValue>> biDirectionalInserts; - private final PCollection, TimestampedValue>> biDirectionalDeletes; + private final PCollection, Row>> biDirectionalInserts; + private final PCollection, Row>> biDirectionalDeletes; CdcOutput( Pipeline p, PCollection uniDirectionalRows, - PCollection, TimestampedValue>> biDirectionalInserts, - PCollection, TimestampedValue>> biDirectionalDeletes) { + PCollection, Row>> biDirectionalInserts, + PCollection, Row>> biDirectionalDeletes) { this.pipeline = p; this.uniDirectionalRows = uniDirectionalRows; this.biDirectionalInserts = biDirectionalInserts; @@ -204,11 +202,11 @@ PCollection uniDirectionalRows() { return uniDirectionalRows; } - PCollection, TimestampedValue>> biDirectionalInserts() { + PCollection, Row>> biDirectionalInserts() { return biDirectionalInserts; } - PCollection, TimestampedValue>> biDirectionalDeletes() { + PCollection, Row>> biDirectionalDeletes() { return biDirectionalDeletes; } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java index bf792f268c02..2e233b8dfdf3 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java @@ -35,21 +35,15 @@ import org.apache.beam.sdk.values.ValueKind; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.joda.time.Instant; /** * Receives a {@link CoGbkResult} containing inserts and deletes sharing the same snapshot ID and * Primary Key, and uses {@link CdcResolver} to identify logical updates. - * - *

    Input elements are pre-prepared with reified timestamps. This is because CoGroupByKey assigns - * all elements in a window with the same timestamp, erasing individual record timestamps. This DoFn - * enriches the record with the reified value to preserve its timestamp. TODO(ahmedabu98): is this ^ - * necessary anymore, now that we window each group by its snapshot commit timestamp? All rows - * coming from a group should all have the same timestamp at this point. And we only potentially - * assign watermark column timestamp downstream from this DoFn. */ -public class ResolveChanges extends DoFn, CoGbkResult>, Row> { - public static final TupleTag> DELETES = new TupleTag<>() {}; - public static final TupleTag> INSERTS = new TupleTag<>() {}; +class ResolveChanges extends DoFn, CoGbkResult>, Row> { + static final TupleTag DELETES = new TupleTag<>() {}; + static final TupleTag INSERTS = new TupleTag<>() {}; private final RowFilter rowFilter; ResolveChanges(IcebergScanConfig scanConfig) { @@ -63,31 +57,28 @@ public class ResolveChanges extends DoFn, CoGbkResult>, Row> { @ProcessElement public void processElement( - @Element KV, CoGbkResult> element, OutputReceiver out) { + @Element KV, CoGbkResult> element, @Timestamp Instant timestamp, OutputReceiver out) { Row primaryKey = element.getKey().getValue(); Set pkFields = new HashSet<>(primaryKey.getSchema().getFieldNames()); CoGbkResult result = element.getValue(); // should be okay to materialize these lists. a PK collision will likely be a handful of records // at most - List> deletes = Lists.newArrayList(result.getAll(DELETES)); - List> inserts = Lists.newArrayList(result.getAll(INSERTS)); - // TODO(ahmedabu98): do we need to sort anymore? all records should have the same timestamp now. - deletes.sort(Comparator.comparing(TimestampedValue::getTimestamp)); - inserts.sort(Comparator.comparing(TimestampedValue::getTimestamp)); + List deletes = Lists.newArrayList(result.getAll(DELETES)); + List inserts = Lists.newArrayList(result.getAll(INSERTS)); new RowResolver(pkFields) .resolve( deletes, inserts, - (kind, tv) -> { - Row projectedRow = rowFilter.filter(tv.getValue()); - out.builder(projectedRow).setValueKind(kind).setTimestamp(tv.getTimestamp()).output(); - logEmit(kind, tv); + (kind, row) -> { + Row projectedRow = rowFilter.filter(row); + out.builder(projectedRow).setValueKind(kind).setTimestamp(timestamp).output(); + logEmit(kind, row); }); } - private static final class RowResolver extends CdcResolver> { + private static final class RowResolver extends CdcResolver { private final Set pkFields; RowResolver(Set pkFields) { @@ -95,20 +86,20 @@ private static final class RowResolver extends CdcResolver } @Override - protected int nonPkHash(TimestampedValue element) { + protected int nonPkHash(Row element) { int hash = 1; - for (String field : element.getValue().getSchema().getFieldNames()) { + for (String field : element.getSchema().getFieldNames()) { if (pkFields.contains(field)) { continue; } - hash = 31 * hash + Objects.hashCode(element.getValue().getValue(field)); + hash = 31 * hash + Objects.hashCode(element.getValue(field)); } return hash; } @Override - protected boolean nonPkEquals(TimestampedValue delete, TimestampedValue insert) { - Schema schema = insert.getValue().getSchema(); + protected boolean nonPkEquals(Row delete, Row insert) { + Schema schema = insert.getSchema(); for (String field : schema.getFieldNames()) { // we already know PK values are equal if (pkFields.contains(field)) { @@ -116,8 +107,8 @@ protected boolean nonPkEquals(TimestampedValue delete, TimestampedValue delete, TimestampedValue tv) { + private static void logEmit(ValueKind kind, Row row) { switch (kind) { case UPDATE_BEFORE: - System.out.printf("[BIDIRECTIONAL] -- UpdateBefore:%n\t%s%n", tv); + System.out.printf("[BIDIRECTIONAL] -- UpdateBefore:%n\t%s%n", row); break; case UPDATE_AFTER: - System.out.printf("[BIDIRECTIONAL] -- UpdateAfter%n\t%s%n", tv); + System.out.printf("[BIDIRECTIONAL] -- UpdateAfter%n\t%s%n", row); break; case DELETE: - System.out.printf("[BIDIRECTIONAL] -- Deleted%n%s%n", tv); + System.out.printf("[BIDIRECTIONAL] -- Deleted%n%s%n", row); break; case INSERT: - System.out.printf("[BIDIRECTIONAL] -- Inserted%n%s%n", tv); + System.out.printf("[BIDIRECTIONAL] -- Inserted%n%s%n", row); break; } } From 9ce456a8282df47281e5f5df62e7db9b6338392c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 1 Jun 2026 14:34:57 -0400 Subject: [PATCH 5/9] add tests --- .../IO_Iceberg_Integration_Tests.json | 2 +- .../sdk/io/iceberg/IcebergScanConfig.java | 8 +- .../io/iceberg/SerializableDeleteFile.java | 2 +- .../io/iceberg/cdc/ChangelogDescriptor.java | 37 +- .../sdk/io/iceberg/cdc/ChangelogScanner.java | 9 +- .../cdc/IncrementalChangelogSource.java | 22 +- .../io/iceberg/cdc/ReadFromChangelogs.java | 14 +- .../sdk/io/iceberg/cdc/ResolveChanges.java | 16 +- .../io/iceberg/cdc/WatchForSnapshotsSdf.java | 7 +- ...ergCdcReadSchemaTransformProviderTest.java | 85 +++- .../sdk/io/iceberg/IcebergIOReadTest.java | 83 +++- .../sdk/io/iceberg/IcebergScanConfigTest.java | 213 ++++++++ ...IcebergSchemaTransformTranslationTest.java | 35 +- .../sdk/io/iceberg/PartitionUtilsTest.java | 68 +++ .../beam/sdk/io/iceberg/ReadUtilsTest.java | 43 ++ .../iceberg/SerializableDeleteFileTest.java | 222 +++++++++ .../sdk/io/iceberg/TestDataWarehouse.java | 2 +- .../iceberg/catalog/IcebergCatalogBaseIT.java | 213 ++++++++ .../iceberg/cdc/ApplyWatermarkColumnTest.java | 155 ++++++ .../sdk/io/iceberg/cdc/CdcReadUtilsTest.java | 381 +++++++++++++++ .../sdk/io/iceberg/cdc/CdcResolverTest.java | 118 +++++ .../io/iceberg/cdc/ChangelogScannerTest.java | 454 ++++++++++++++++++ .../sdk/io/iceberg/cdc/DeleteReaderTest.java | 151 +++++- .../cdc/IncrementalChangelogSourceTest.java | 279 +++++++++++ .../io/iceberg/cdc/LocalResolveDoFnIT.java | 224 +++++++++ .../sdk/io/iceberg/cdc/OverlapRangeTest.java | 163 +++++++ .../iceberg/cdc/ReadFromChangelogsTest.java | 356 ++++++++++++++ .../io/iceberg/cdc/ResolveChangesTest.java | 210 ++++++++ .../cdc/SerializableChangelogTaskTest.java | 256 ++++++++++ .../io/iceberg/cdc/SnapshotWindowFnTest.java | 93 ++++ .../iceberg/cdc/WatchForSnapshotsSdfTest.java | 265 ++++++++++ 31 files changed, 4096 insertions(+), 90 deletions(-) create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfigTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFileTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumnTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcReadUtilsTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolverTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScannerTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSourceTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFnIT.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRangeTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogsTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChangesTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTaskTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFnTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdfTest.java diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 7ab7bcd9a9c6..37dd25bf9029 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": 3 } 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 c7cfacc5d124..9518197a4134 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 @@ -420,8 +420,8 @@ void validate(Table table) { checkState( !primaryKeyIds.isEmpty(), "Cannot read CDC records as the table schema does not specified any primary key fields."); - Set projectedPrimaryKeyIds = getProjectedSchema().identifierFieldIds(); - primaryKeyIds.removeAll(projectedPrimaryKeyIds); + Set projectedFieldIds = TypeUtil.getProjectedIds(getProjectedSchema()); + primaryKeyIds.removeAll(projectedFieldIds); checkArgument( primaryKeyIds.isEmpty(), "When reading CDC records, the projected schema must not drop primary key fields. " @@ -477,7 +477,9 @@ void validate(Table table) { error("'watermark_column' must be a timestamp-typed column, but '%s' has type %s"), watermarkColumn, field.type().typeId()); - checkArgumentNotNull(getProjectedSchema().findField(watermarkColumn), "'watermark_column' column should not be dropped."); + checkArgumentNotNull( + getProjectedSchema().findField(watermarkColumn), + "'watermark_column' column should not be dropped."); } } 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 index e2aeeff6d7fd..d05b7d51d515 100644 --- 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 @@ -130,7 +130,7 @@ abstract static class Builder { abstract Builder setPartitionSpecId(int partitionSpec); - abstract Builder setSortOrderId(int sortOrderId); + abstract Builder setSortOrderId(@Nullable Integer sortOrderId); abstract Builder setEqualityFieldIds(List equalityFieldIds); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java index 103d38be4012..9caa5cced462 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java @@ -19,12 +19,12 @@ import com.google.auto.value.AutoValue; import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.Schema; 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.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; import org.checkerframework.checker.nullness.qual.Nullable; /** Descriptor for a set of {@link SerializableChangelogTask}s. */ @@ -35,16 +35,35 @@ public static Builder builder() { return new AutoValue_ChangelogDescriptor.Builder(); } - public static SchemaCoder coder() { - try { - return SchemaRegistry.createDefault().getSchemaCoder(ChangelogDescriptor.class); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); - } + @SuppressWarnings("nullness") + public static SchemaCoder coder(Schema overlapSchema) { + Schema descriptorSchema = + Schema.builder() + .addStringField("tableIdentifierString") + .addNullableField("overlapLower", Schema.FieldType.row(overlapSchema)) + .addNullableField("overlapUpper", Schema.FieldType.row(overlapSchema)) + .build(); + + return SchemaCoder.of( + descriptorSchema, + TypeDescriptor.of(ChangelogDescriptor.class), + descriptor -> + Row.withSchema(descriptorSchema) + .addValues( + descriptor.getTableIdentifierString(), + descriptor.getOverlapLower(), + descriptor.getOverlapUpper()) + .build(), + row -> + ChangelogDescriptor.builder() + .setTableIdentifierString(row.getString("tableIdentifierString")) + .setOverlapLower(row.getRow("overlapLower")) + .setOverlapUpper(row.getRow("overlapUpper")) + .build()); } @SchemaFieldNumber("0") - abstract String getTableIdentifierString(); + public abstract String getTableIdentifierString(); @SchemaFieldNumber("1") public abstract @Nullable Row getOverlapLower(); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java index 676d21a3da5a..9dba01fa7e6e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java @@ -185,8 +185,6 @@ class ChangelogScanner static final TupleTag>> LARGE_BIDIRECTIONAL_TASKS = new TupleTag<>(); - static final KvCoder> OUTPUT_CODER = - KvCoder.of(ChangelogDescriptor.coder(), ListCoder.of(SerializableChangelogTask.coder())); private final IcebergScanConfig scanConfig; private @MonotonicNonNull Table table; private @MonotonicNonNull Snapshot snapshot; @@ -207,6 +205,13 @@ class ChangelogScanner this.scanConfig = scanConfig; } + static KvCoder> coder( + org.apache.beam.sdk.schemas.Schema rowIdBeamSchema) { + return KvCoder.of( + ChangelogDescriptor.coder(rowIdBeamSchema), + ListCoder.of(SerializableChangelogTask.coder())); + } + @Setup public void setup() { TableCache.setup(scanConfig); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java index 003465bbfe78..2656b3baeae0 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; import org.apache.beam.sdk.io.iceberg.IcebergUtils; import org.apache.beam.sdk.io.iceberg.ReadUtils; @@ -38,6 +39,7 @@ import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; @@ -63,8 +65,6 @@ * bounded path creates the snapshot range up front. */ public class IncrementalChangelogSource extends PTransform> { - private static final Duration DEFAULT_POLL_INTERVAL = Duration.standardSeconds(60); - private final IcebergScanConfig scanConfig; public IncrementalChangelogSource(IcebergScanConfig scanConfig) { @@ -94,9 +94,11 @@ public PCollection expand(PBegin input) { .withOutputTags( UNIDIRECTIONAL_TASKS, TupleTagList.of(LARGE_BIDIRECTIONAL_TASKS).and(SMALL_BIDIRECTIONAL_TASKS))); - changelogTasks.get(UNIDIRECTIONAL_TASKS).setCoder(ChangelogScanner.OUTPUT_CODER); - changelogTasks.get(SMALL_BIDIRECTIONAL_TASKS).setCoder(ChangelogScanner.OUTPUT_CODER); - changelogTasks.get(LARGE_BIDIRECTIONAL_TASKS).setCoder(ChangelogScanner.OUTPUT_CODER); + KvCoder> tasksCoder = + ChangelogScanner.coder(scanConfig.rowIdBeamSchema()); + changelogTasks.get(UNIDIRECTIONAL_TASKS).setCoder(tasksCoder); + changelogTasks.get(SMALL_BIDIRECTIONAL_TASKS).setCoder(tasksCoder); + changelogTasks.get(LARGE_BIDIRECTIONAL_TASKS).setCoder(tasksCoder); Schema projectedRowSchema = IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); @@ -133,7 +135,11 @@ public PCollection expand(PBegin input) { .apply("CoGroupBy Primary Key", CoGroupByKey.create()) .apply("Resolve Delete-Insert Pairs", ParDo.of(new ResolveChanges(scanConfig))) .setRowSchema(projectedRowSchema) - .apply("Re-window to Global", Window.into(new GlobalWindows())); + .apply( + "Re-window to Global", + Window.into(new GlobalWindows()) + .triggering(DefaultTrigger.of()) + .discardingFiredPanes()); // Merge all three paths into a single output. All three are in GlobalWindows. PCollection merged = @@ -159,11 +165,9 @@ public PCollection expand(PBegin input) { * emits per snapshot. */ private PCollection unboundedSnapshots(PBegin input) { - Duration pollInterval = - MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); return input .apply("Impulse", Create.of("")) - .apply("Watch for Snapshots", ParDo.of(new WatchForSnapshotsSdf(scanConfig, pollInterval))); + .apply("Watch for Snapshots", ParDo.of(new WatchForSnapshotsSdf(scanConfig))); } /** diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java index bcf9522faf68..ade32693d39e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Redistribute; -import org.apache.beam.sdk.transforms.Reify; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.values.KV; @@ -55,7 +54,6 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.ValueKind; @@ -127,7 +125,7 @@ public CdcOutput expand(PCollectionTuple input) { PCollection uniDirectionalRows = input .get(UNIDIRECTIONAL_TASKS) - .apply(Redistribute.arbitrarily()) + .apply("Redistribute Uni-Directional Changes", Redistribute.arbitrarily()) .apply( "Read Uni-Directional Changes", ParDo.of(ReadDoFn.unidirectional(scanConfig)) @@ -141,7 +139,7 @@ public CdcOutput expand(PCollectionTuple input) { PCollectionTuple biDirectionalRows = input .get(LARGE_BIDIRECTIONAL_TASKS) - .apply(Redistribute.arbitrarily()) + .apply("Redistribute Large Bi-Directional Changes", Redistribute.arbitrarily()) .apply( "Read Bi-Directional Changes", ParDo.of(ReadDoFn.bidirectional(scanConfig)) @@ -166,13 +164,9 @@ public CdcOutput expand(PCollectionTuple input) { KvCoder.of(VarLongCoder.of(), SchemaCoder.of(scanConfig.rowIdBeamSchema())), SchemaCoder.of(fullRowSchema)); PCollection, Row>> keyedInsertsWithTimestamps = - biDirectionalRows - .get(BIDIRECTIONAL_INSERTS) - .setCoder(keyedOutputCoder); + biDirectionalRows.get(BIDIRECTIONAL_INSERTS).setCoder(keyedOutputCoder); PCollection, Row>> keyedDeletesWithTimestamps = - biDirectionalRows - .get(BIDIRECTIONAL_DELETES) - .setCoder(keyedOutputCoder); + biDirectionalRows.get(BIDIRECTIONAL_DELETES).setCoder(keyedOutputCoder); return new CdcOutput( input.getPipeline(), diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java index 2e233b8dfdf3..c132cc7876f6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChanges.java @@ -17,10 +17,8 @@ */ package org.apache.beam.sdk.io.iceberg.cdc; -import java.util.Comparator; import java.util.HashSet; import java.util.List; -import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; @@ -30,7 +28,6 @@ import org.apache.beam.sdk.util.RowFilter; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.ValueKind; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -57,7 +54,9 @@ class ResolveChanges extends DoFn, CoGbkResult>, Row> { @ProcessElement public void processElement( - @Element KV, CoGbkResult> element, @Timestamp Instant timestamp, OutputReceiver out) { + @Element KV, CoGbkResult> element, + @Timestamp Instant timestamp, + OutputReceiver out) { Row primaryKey = element.getKey().getValue(); Set pkFields = new HashSet<>(primaryKey.getSchema().getFieldNames()); CoGbkResult result = element.getValue(); @@ -92,7 +91,10 @@ protected int nonPkHash(Row element) { if (pkFields.contains(field)) { continue; } - hash = 31 * hash + Objects.hashCode(element.getValue(field)); + hash = + 31 * hash + + Row.Equals.deepHashCode( + element.getValue(field), element.getSchema().getField(field).getType()); } return hash; } @@ -107,9 +109,7 @@ protected boolean nonPkEquals(Row delete, Row insert) { } // return early if two values are not equal if (!Row.Equals.deepEquals( - insert.getValue(field), - delete.getValue(field), - schema.getField(field).getType())) { + insert.getValue(field), delete.getValue(field), schema.getField(field).getType())) { return false; } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java index 8642a0f0b5fa..0d232c64053f 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdf.java @@ -69,6 +69,8 @@ @DoFn.UnboundedPerElement class WatchForSnapshotsSdf extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(WatchForSnapshotsSdf.class); + private static final Duration DEFAULT_POLL_INTERVAL = Duration.standardSeconds(60); + private static final Counter snapshotsEmitted = Metrics.counter(WatchForSnapshotsSdf.class, "snapshotsEmitted"); private static final Gauge latestEmittedSnapshotId = @@ -80,9 +82,10 @@ class WatchForSnapshotsSdf extends DoFn { private final IcebergScanConfig scanConfig; private final Duration pollInterval; - WatchForSnapshotsSdf(IcebergScanConfig scanConfig, Duration pollInterval) { + WatchForSnapshotsSdf(IcebergScanConfig scanConfig) { this.scanConfig = scanConfig; - this.pollInterval = pollInterval; + this.pollInterval = + MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); } @GetInitialRestriction 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..5f69b1717337 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,12 +35,16 @@ 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.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.iceberg.CatalogUtil; import org.apache.iceberg.Snapshot; 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; @@ -52,6 +56,18 @@ 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)); + + private static final org.apache.iceberg.Schema CDC_CONFIG_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.optional(3, "category", Types.StringType.get()), + Types.NestedField.required(4, "event_micros", Types.LongType.get())), + ImmutableSet.of(1)); + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); @Rule public TestPipeline testPipeline = TestPipeline.create(); @@ -73,6 +89,10 @@ public void testBuildTransformWithRow() { .withFieldValue("to_timestamp", 456L) .withFieldValue("starting_strategy", "earliest") .withFieldValue("poll_interval_seconds", 789) + .withFieldValue("keep", ImmutableList.of("id", "data", "event_micros")) + .withFieldValue("filter", "\"category\" = 'include'") + .withFieldValue("watermark_column", "event_micros") + .withFieldValue("max_snapshot_discovery_delay", 321L) .build(); new IcebergCdcReadSchemaTransformProvider().from(config); @@ -83,8 +103,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 +142,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()); @@ -159,4 +179,61 @@ public void testStreamingReadUsingManagedTransform() throws Exception { testPipeline.run(); } + + @Test + public void testManagedReadWithProjectionFilterWatermarkAndSnapshotRange() throws Exception { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + TableIdentifier tableId = TableIdentifier.parse(identifier); + + Table table = warehouse.createTable(tableId, CDC_CONFIG_SCHEMA); + long eventMicros = (System.currentTimeMillis() - 1_000L) * 1_000L; + List records = + ImmutableList.of( + record(1L, "keep-a", "include", eventMicros), + record(2L, "drop", "exclude", eventMicros + 1_000L), + record(3L, "keep-b", "include", eventMicros + 2_000L)); + table + .newFastAppend() + .appendFile(warehouse.writeRecords("cdc-managed-config.parquet", table.schema(), records)) + .commit(); + + Map properties = new HashMap<>(); + properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); + properties.put("warehouse", warehouse.location); + + Map configMap = new HashMap<>(); + configMap.put("table", identifier); + configMap.put("catalog_name", "test-name"); + configMap.put("catalog_properties", properties); + configMap.put("from_snapshot", table.currentSnapshot().snapshotId()); + configMap.put("to_snapshot", table.currentSnapshot().snapshotId()); + configMap.put("keep", ImmutableList.of("id", "data", "event_micros")); + configMap.put("filter", "\"category\" = 'include'"); + configMap.put("watermark_column", "event_micros"); + configMap.put("max_snapshot_discovery_delay", 30L); + + org.apache.iceberg.Schema projectedSchema = table.schema().select("id", "data", "event_micros"); + Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(projectedSchema); + List expectedRows = + records.stream() + .filter(record -> "include".equals(record.getField("category"))) + .map(record -> IcebergUtils.icebergRecordToBeamRow(beamSchema, record)) + .collect(Collectors.toList()); + + PCollection output = + testPipeline + .apply(Managed.read(Managed.ICEBERG_CDC).withConfig(configMap)) + .getSinglePCollection(); + + assertThat(output.isBounded(), equalTo(BOUNDED)); + PAssert.that(output).containsInAnyOrder(expectedRows); + + testPipeline.run(); + } + + private static Record record(long id, String data, String category, long eventMicros) { + return TestFixtures.createRecord( + CDC_CONFIG_SCHEMA, + ImmutableMap.of("id", id, "data", data, "category", category, "event_micros", eventMicros)); + } } 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/IcebergScanConfigTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfigTest.java new file mode 100644 index 000000000000..742e26737447 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfigTest.java @@ -0,0 +1,213 @@ +/* + * 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.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import java.util.List; +import java.util.UUID; +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.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.types.Types; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** Tests for {@link IcebergScanConfig}. */ +public class IcebergScanConfigTest { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + 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()), + Types.NestedField.optional(3, "category", Types.StringType.get()), + Types.NestedField.required(4, "event_time", Types.TimestampType.withoutZone()), + Types.NestedField.required(5, "event_micros", Types.LongType.get()), + Types.NestedField.optional(6, "optional_time", Types.TimestampType.withoutZone()), + Types.NestedField.required(7, "required_text", Types.StringType.get()), + Types.NestedField.required( + 8, + "nested", + Types.StructType.of( + Types.NestedField.required( + 9, "nested_time", Types.TimestampType.withoutZone())))), + ImmutableSet.of(1)); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Test + public void cdcValidationRequiresIdentifierFields() { + TableIdentifier tableId = uniqueTableId(); + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergScanConfig scanConfig = + scanConfigBuilder(tableId, TestFixtures.SCHEMA).setUseCdc(true).build(); + + IllegalStateException thrown = + assertThrows(IllegalStateException.class, () -> scanConfig.validate(table)); + assertThat(thrown.getMessage(), containsString("Cannot read CDC records")); + assertThat(thrown.getMessage(), containsString("primary key fields")); + } + + @Test + public void cdcValidationRejectsProjectionDroppingIdentifierFields() { + TableIdentifier tableId = uniqueTableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + + IcebergScanConfig keepWithoutPk = + scanConfigBuilder(tableId, CDC_SCHEMA) + .setUseCdc(true) + .setKeepFields(ImmutableList.of("data")) + .build(); + IllegalArgumentException keepException = + assertThrows(IllegalArgumentException.class, () -> keepWithoutPk.validate(table)); + assertThat( + keepException.getMessage(), + containsString("projected schema must not drop primary key fields")); + + IcebergScanConfig dropPk = + scanConfigBuilder(tableId, CDC_SCHEMA) + .setUseCdc(true) + .setDropFields(ImmutableList.of("id")) + .build(); + IllegalArgumentException dropException = + assertThrows(IllegalArgumentException.class, () -> dropPk.validate(table)); + assertThat( + dropException.getMessage(), + containsString("projected schema must not drop primary key fields")); + } + + @Test + public void requiredSchemaIncludesFilterOnlyFieldsWithoutChangingProjection() { + TableIdentifier tableId = uniqueTableId(); + warehouse.createTable(tableId, CDC_SCHEMA); + IcebergScanConfig scanConfig = + scanConfigBuilder(tableId, CDC_SCHEMA) + .setUseCdc(true) + .setKeepFields(ImmutableList.of("id")) + .setFilterString("\"data\" = 'keep' AND \"category\" = 'include'") + .build(); + + assertEquals(ImmutableList.of("id"), fieldNames(scanConfig.getProjectedSchema())); + assertEquals( + ImmutableSet.of("id", "data", "category"), + ImmutableSet.copyOf(fieldNames(scanConfig.getRequiredSchema()))); + } + + @Test + public void watermarkColumnAcceptsRequiredTimestampAndLongColumns() { + TableIdentifier tableId = uniqueTableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + + scanConfigBuilder(tableId, CDC_SCHEMA) + .setUseCdc(true) + .setKeepFields(ImmutableList.of("id", "event_time")) + .setWatermarkColumn("event_time") + .build() + .validate(table); + + scanConfigBuilder(tableId, CDC_SCHEMA) + .setUseCdc(true) + .setKeepFields(ImmutableList.of("id", "event_micros")) + .setWatermarkColumn("event_micros") + .build() + .validate(table); + } + + @Test + public void watermarkColumnRejectsInvalidConfigurations() { + TableIdentifier tableId = uniqueTableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA); + + assertInvalidWatermark( + tableId, table, "event_time", false, ImmutableList.of("id", "event_time"), "CDC mode"); + assertInvalidWatermark( + tableId, table, "missing", true, ImmutableList.of("id"), "unknown column"); + assertInvalidWatermark( + tableId, + table, + "optional_time", + true, + ImmutableList.of("id", "optional_time"), + "non-nullable"); + assertInvalidWatermark( + tableId, + table, + "required_text", + true, + ImmutableList.of("id", "required_text"), + "must be a timestamp-typed column"); + assertInvalidWatermark( + tableId, table, "event_time", true, ImmutableList.of("id"), "should not be dropped"); + } + + private void assertInvalidWatermark( + TableIdentifier tableId, + Table table, + String watermarkColumn, + boolean useCdc, + List keepFields, + String expectedMessage) { + IcebergScanConfig scanConfig = + scanConfigBuilder(tableId, CDC_SCHEMA) + .setUseCdc(useCdc) + .setKeepFields(keepFields) + .setWatermarkColumn(watermarkColumn) + .build(); + + IllegalArgumentException thrown = + assertThrows(IllegalArgumentException.class, () -> scanConfig.validate(table)); + assertThat(thrown.getMessage(), containsString(expectedMessage)); + } + + private IcebergScanConfig.Builder scanConfigBuilder( + TableIdentifier tableId, org.apache.iceberg.Schema schema) { + return IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties( + ImmutableMap.of( + "type", + CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, + "warehouse", + warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(schema)); + } + + private static List fieldNames(org.apache.iceberg.Schema schema) { + return schema.columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); + } + + private static TableIdentifier uniqueTableId() { + return TableIdentifier.of( + "default", "table_" + Long.toString(UUID.randomUUID().hashCode(), 16)); + } +} 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..64f16f83ab5d 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,10 @@ 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'") + .withFieldValue("watermark_column", "event_micros") + .withFieldValue("max_snapshot_discovery_delay", 321L) .build(); @Test @@ -269,7 +285,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 +302,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 +347,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 6287a6e06197..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; @@ -104,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()); @@ -249,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/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..b83d1b5eef07 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 @@ -62,7 +62,9 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.PeriodicImpulse; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; @@ -74,10 +76,14 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.ValueKind; 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.iceberg.AppendFiles; import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.NullOrder; import org.apache.iceberg.PartitionSpec; @@ -85,15 +91,22 @@ import org.apache.iceberg.SortDirection; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; +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.encryption.InputFilesDecryptor; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DataWriter; @@ -277,6 +290,8 @@ public void cleanUp() throws Exception { .addLogicalTypeField("date", SqlTypes.DATE) .addLogicalTypeField("time", SqlTypes.TIME) .build(); + private static final Schema CDC_BEAM_SCHEMA = + Schema.builder().addInt64Field("id").addStringField("data").build(); private static final SimpleFunction ROW_FUNC = new SimpleFunction() { @@ -319,6 +334,9 @@ public Row apply(Long num) { protected static final org.apache.iceberg.Schema ICEBERG_SCHEMA = beamSchemaToIcebergSchema(BEAM_SCHEMA); + private static final org.apache.iceberg.Schema CDC_ICEBERG_SCHEMA = + new org.apache.iceberg.Schema( + beamSchemaToIcebergSchema(CDC_BEAM_SCHEMA).columns(), Collections.singleton(1)); protected static final SimpleFunction RECORD_FUNC = new SimpleFunction() { @Override @@ -559,6 +577,81 @@ public void testStreamingReadWithColumnPruning_drop() throws Exception { pipeline.run().waitUntilFinish(); } + @Test + public void testBatchCdcReadMixedDeleteAndOverwriteSnapshots() throws Exception { + Table table = createCdcTable(); + DataFile firstFile = + commitCdcAppend( + table, + "cdc-first-data.parquet", + Arrays.asList( + cdcRecord(table.schema(), 1L, "first-file-update-before"), + cdcRecord(table.schema(), 2L, "first-file-delete"))); + Snapshot firstSnapshot = checkStateNotNull(table.currentSnapshot()); + + DataFile secondFile = + commitCdcAppend( + table, + "cdc-second-data.parquet", + Arrays.asList( + cdcRecord(table.schema(), 3L, "second-file-unchanged"), + cdcRecord(table.schema(), 4L, "second-file-update-before"))); + Snapshot secondSnapshot = checkStateNotNull(table.currentSnapshot()); + + DeleteFile equalityDelete = writeCdcEqualityDelete(table, "cdc-equality-delete.parquet", 2L); + DeleteFile positionDelete = + writeCdcPositionDelete(table, "cdc-position-delete.parquet", firstFile, 0L); + DataFile thirdFile = + writeCdcDataFile( + table, + "cdc-third-data.parquet", + Collections.singletonList(cdcRecord(table.schema(), 1L, "third-file-update-after"))); + table + .newRowDelta() + .addDeletes(equalityDelete) + .addDeletes(positionDelete) + .addRows(thirdFile) + .commit(); + table.refresh(); + Snapshot thirdSnapshot = checkStateNotNull(table.currentSnapshot()); + + DataFile fourthFile = + writeCdcDataFile( + table, + "cdc-fourth-data.parquet", + Arrays.asList( + cdcRecord(table.schema(), 3L, "second-file-unchanged"), + cdcRecord(table.schema(), 4L, "fourth-file-update-after"))); + table.newOverwrite().deleteFile(secondFile).addFile(fourthFile).commit(); + table.refresh(); + Snapshot fourthSnapshot = checkStateNotNull(table.currentSnapshot()); + + Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("from_snapshot", firstSnapshot.snapshotId()); + config.put("to_snapshot", fourthSnapshot.snapshotId()); + config.put("streaming", true); + + PCollection rows = + pipeline.apply(Managed.read(ICEBERG_CDC).withConfig(config)).getSinglePCollection(); + + PCollection changes = rows.apply("Format CDC Changes", ParDo.of(new FormatCdcChange())); + + assertThat(rows.isBounded(), equalTo(BOUNDED)); + assertEquals(CDC_BEAM_SCHEMA, rows.getSchema()); + PAssert.that(changes) + .containsInAnyOrder( + cdcChange(ValueKind.INSERT, firstSnapshot, 1L, "first-file-update-before"), + cdcChange(ValueKind.INSERT, firstSnapshot, 2L, "first-file-delete"), + cdcChange(ValueKind.INSERT, secondSnapshot, 3L, "second-file-unchanged"), + cdcChange(ValueKind.INSERT, secondSnapshot, 4L, "second-file-update-before"), + cdcChange(ValueKind.UPDATE_BEFORE, thirdSnapshot, 1L, "first-file-update-before"), + cdcChange(ValueKind.UPDATE_AFTER, thirdSnapshot, 1L, "third-file-update-after"), + cdcChange(ValueKind.DELETE, thirdSnapshot, 2L, "first-file-delete"), + cdcChange(ValueKind.UPDATE_BEFORE, fourthSnapshot, 4L, "second-file-update-before"), + cdcChange(ValueKind.UPDATE_AFTER, fourthSnapshot, 4L, "fourth-file-update-after")); + pipeline.run().waitUntilFinish(); + } + @Test public void testBatchReadBetweenSnapshots() throws Exception { runReadBetween(true, false); @@ -1067,6 +1160,126 @@ public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throw pipeline.run().waitUntilFinish(); } + private Table createCdcTable() { + return catalog.createTable( + TableIdentifier.parse(tableId()), + CDC_ICEBERG_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + "2", + TableProperties.SPLIT_SIZE, + "1", + TableProperties.DEFAULT_WRITE_METRICS_MODE, + "full")); + } + + private Map cdcReadConfig(long fromSnapshotId, long toSnapshotId) { + Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("from_snapshot", fromSnapshotId); + config.put("to_snapshot", toSnapshotId); + return config; + } + + private static String cdcChange(ValueKind valueKind, Snapshot snapshot, long id, String data) { + return String.format("%s:%d:%d:%s", valueKind, snapshot.timestampMillis(), id, data); + } + + private static Record cdcRecord(org.apache.iceberg.Schema schema, long id, String data) { + GenericRecord record = GenericRecord.create(schema); + record.setField("id", id); + if (schema.findField("data") != null) { + record.setField("data", data); + } + return record; + } + + private DataFile commitCdcAppend(Table table, String filename, List records) + throws IOException { + DataFile dataFile = writeCdcDataFile(table, filename, records); + table.newFastAppend().appendFile(dataFile).commit(); + table.refresh(); + return dataFile; + } + + private DataFile writeCdcDataFile(Table table, String filename, List records) + throws IOException { + OutputFile file = + table.io().newOutputFile(table.location() + "/" + UUID.randomUUID() + "-" + filename); + DataWriter writer = + Parquet.writeData(file) + .schema(table.schema()) + .createWriterFunc(GenericParquetWriter::create) + .overwrite() + .withSpec(table.spec()) + .build(); + + try (writer) { + for (Record record : records) { + writer.write(record); + } + } + + return writer.toDataFile(); + } + + private DeleteFile writeCdcEqualityDelete(Table table, String filename, long id) + throws IOException { + org.apache.iceberg.Schema deleteSchema = table.schema().select("id"); + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(table.schema(), table.spec(), new int[] {1}, deleteSchema, null); + EqualityDeleteWriter writer = + appenderFactory.newEqDeleteWriter( + EncryptedFiles.plainAsEncryptedOutput( + table + .io() + .newOutputFile(table.location() + "/" + UUID.randomUUID() + "-" + filename)), + FileFormat.PARQUET, + null); + + try (writer) { + writer.write(cdcRecord(deleteSchema, id, null)); + } + + return writer.toDeleteFile(); + } + + private DeleteFile writeCdcPositionDelete( + Table table, String filename, DataFile dataFile, long... positions) throws IOException { + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(table.schema(), table.spec()); + PositionDeleteWriter writer = + appenderFactory.newPosDeleteWriter( + EncryptedFiles.plainAsEncryptedOutput( + table + .io() + .newOutputFile(table.location() + "/" + UUID.randomUUID() + "-" + filename)), + FileFormat.PARQUET, + null); + + try (writer) { + for (long position : positions) { + writer.write(PositionDelete.create().set(dataFile.location(), position)); + } + } + + return writer.toDeleteFile(); + } + + private static final class FormatCdcChange extends DoFn { + @ProcessElement + public void process( + @Element Row row, + ValueKind valueKind, + @Timestamp Instant timestamp, + OutputReceiver outputReceiver) { + outputReceiver.output( + String.format( + "%s:%d:%d:%s", + valueKind, timestamp.getMillis(), row.getInt64("id"), row.getString("data"))); + } + } + @Test public void testWriteWithTableProperties() throws IOException { Map config = new HashMap<>(managedIcebergConfig(tableId())); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumnTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumnTest.java new file mode 100644 index 000000000000..911e13196329 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ApplyWatermarkColumnTest.java @@ -0,0 +1,155 @@ +/* + * 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.assertThrows; + +import java.time.LocalDateTime; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; +import org.apache.beam.sdk.schemas.logicaltypes.Timestamp; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reify; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link ApplyWatermarkColumn}. */ +@RunWith(JUnit4.class) +public class ApplyWatermarkColumnTest { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + @Test + public void stampsRowsFromSupportedWatermarkTypes() { + assertWatermarkTimestamp( + "jodaDateTime", + Schema.builder().addStringField("id").addDateTimeField("wm").build(), + Row.withSchema(Schema.builder().addStringField("id").addDateTimeField("wm").build()) + .addValues("joda", new Instant(1_234L)) + .build(), + new Instant(1_234L)); + + assertWatermarkTimestamp( + "longMicros", + Schema.builder().addStringField("id").addInt64Field("wm").build(), + Row.withSchema(Schema.builder().addStringField("id").addInt64Field("wm").build()) + .addValues("long", 1_234_567L) + .build(), + new Instant(1_234L)); + + assertWatermarkTimestamp( + "localDateTime", + Schema.builder().addStringField("id").addLogicalTypeField("wm", SqlTypes.DATETIME).build(), + Row.withSchema( + Schema.builder() + .addStringField("id") + .addLogicalTypeField("wm", SqlTypes.DATETIME) + .build()) + .addValues("ldt", LocalDateTime.of(1969, 12, 31, 23, 59, 59, 123_000_000)) + .build(), + new Instant(-877L)); + + assertWatermarkTimestamp( + "javaInstant", + Schema.builder().addStringField("id").addLogicalTypeField("wm", Timestamp.MICROS).build(), + Row.withSchema( + Schema.builder() + .addStringField("id") + .addLogicalTypeField("wm", Timestamp.MICROS) + .build()) + .addValues("instant", java.time.Instant.parse("1969-12-31T23:59:59.123Z")) + .build(), + new Instant(-877L)); + + pipeline.run(); + } + + @Test + public void nullWatermarkValuePreservesInputTimestamp() { + Schema schema = + Schema.of( + Schema.Field.of("id", Schema.FieldType.STRING), + Schema.Field.nullable("wm", Schema.FieldType.DATETIME)); + Row row = Row.withSchema(schema).addValues("null", null).build(); + Instant inputTimestamp = new Instant(99L); + + PCollection output = + pipeline + .apply( + Create.timestamped(TimestampedValue.of(row, inputTimestamp)) + .withCoder(RowCoder.of(schema))) + .apply(ParDo.of(new ApplyWatermarkColumn("wm"))); + output.setCoder(RowCoder.of(schema)); + + PAssert.that(output.apply(Reify.timestamps())) + .containsInAnyOrder(TimestampedValue.of(row, inputTimestamp)); + + pipeline.run(); + } + + @Test + public void unsupportedWatermarkTypeThrows() { + Schema schema = Schema.builder().addStringField("wm").build(); + Row row = Row.withSchema(schema).addValue("2026-05-24T00:00:00Z").build(); + + assertThrows( + UnsupportedOperationException.class, + () -> { + try (DoFnTester tester = DoFnTester.of(new ApplyWatermarkColumn("wm"))) { + tester.processElement(row); + } + }); + } + + @Test + public void missingWatermarkColumnThrows() { + Schema schema = Schema.builder().addStringField("other").build(); + Row row = Row.withSchema(schema).addValue("value").build(); + + assertThrows( + IllegalArgumentException.class, + () -> { + try (DoFnTester tester = DoFnTester.of(new ApplyWatermarkColumn("wm"))) { + tester.processElement(row); + } + }); + } + + private void assertWatermarkTimestamp( + String name, Schema schema, Row row, Instant expectedTimestamp) { + PCollection output = + pipeline + .apply(name + "Create", Create.of(row).withCoder(RowCoder.of(schema))) + .apply(name + "ApplyWatermark", ParDo.of(new ApplyWatermarkColumn("wm"))); + output.setCoder(RowCoder.of(schema)); + + PCollection> timestamps = + output.apply(name + "ReifyTimestamp", Reify.timestamps()); + PAssert.that(timestamps).containsInAnyOrder(TimestampedValue.of(row, expectedTimestamp)); + } +} 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/CdcResolverTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolverTest.java new file mode 100644 index 000000000000..4f1340efb604 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/CdcResolverTest.java @@ -0,0 +1,118 @@ +/* + * 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.empty; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link CdcResolver}. */ +@RunWith(JUnit4.class) +public class CdcResolverTest { + private static final TestResolver RESOLVER = new TestResolver(); + + @Test + public void duplicateDeleteInsertIsDropped() { + List emitted = + resolve( + Collections.singletonList(item("same", 7)), Collections.singletonList(item("same", 7))); + + assertThat(emitted, empty()); + } + + @Test + public void changedDeleteInsertBecomesUpdatePair() { + List emitted = + resolve( + Collections.singletonList(item("before", 1)), + Collections.singletonList(item("after", 2))); + + assertThat(emitted, contains("UPDATE_BEFORE:before", "UPDATE_AFTER:after")); + } + + @Test + public void duplicateUpdateAndSingletonsResolveByMultiplicity() { + List emitted = + resolve( + Arrays.asList(item("copy", 1), item("old", 2), item("deleted-only", 3)), + Arrays.asList(item("copy", 1), item("new", 4))); + + assertThat(emitted, contains("UPDATE_BEFORE:old", "UPDATE_AFTER:new", "DELETE:deleted-only")); + } + + @Test + public void hashCollisionOnlyConsumesEqualInsertOnce() { + List emitted = + resolve( + Arrays.asList(item("copy", 9), item("deleted-only", 9)), + Collections.singletonList(item("copy", 9))); + + assertThat(emitted, contains("DELETE:deleted-only")); + } + + @Test + public void hashMatchAloneDoesNotDeduplicate() { + List emitted = + resolve( + Collections.singletonList(item("before", 42)), + Collections.singletonList(item("after", 42))); + + assertThat(emitted, contains("UPDATE_BEFORE:before", "UPDATE_AFTER:after")); + } + + private static Item item(String nonPkValue, int hash) { + return new Item(nonPkValue, hash); + } + + private static List resolve(List deletes, List inserts) { + List emitted = new ArrayList<>(); + RESOLVER.resolve( + deletes, inserts, (kind, item) -> emitted.add(kind.name() + ":" + item.nonPkValue)); + return emitted; + } + + private static class TestResolver extends CdcResolver { + @Override + protected int nonPkHash(Item element) { + return element.hash; + } + + @Override + protected boolean nonPkEquals(Item delete, Item insert) { + return delete.nonPkValue.equals(insert.nonPkValue); + } + } + + private static class Item { + private final String nonPkValue; + private final int hash; + + private Item(String nonPkValue, int hash) { + this.nonPkValue = nonPkValue; + this.hash = hash; + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScannerTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScannerTest.java new file mode 100644 index 000000000000..ea9749e8f357 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScannerTest.java @@ -0,0 +1,454 @@ +/* + * 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.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.io.iceberg.SerializableDataFile; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.OutputBuilder; +import org.apache.beam.sdk.values.TimestampedValue; +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.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.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link ChangelogScanner}. */ +@RunWith(JUnit4.class) +public class ChangelogScannerTest { + private static final Schema SINGLE_PK_SCHEMA = + new Schema( + ImmutableList.of( + required(1, "id", Types.LongType.get()), optional(2, "data", Types.StringType.get())), + ImmutableSet.of(1)); + private static final Schema COMPOSITE_PK_SCHEMA = + new Schema( + ImmutableList.of( + required(1, "account", Types.StringType.get()), + required(2, "sequence", Types.IntegerType.get()), + optional(3, "data", Types.StringType.get())), + ImmutableSet.of(1, 2)); + private static final Schema SINGLE_RECORD_ID_SCHEMA = recordIdSchema(SINGLE_PK_SCHEMA); + private static final Schema COMPOSITE_RECORD_ID_SCHEMA = recordIdSchema(COMPOSITE_PK_SCHEMA); + private static final PartitionSpec UNPARTITIONED_SPEC = PartitionSpec.unpartitioned(); + private static final PartitionSpec IDENTITY_ID_SPEC = + PartitionSpec.builderFor(SINGLE_PK_SCHEMA).identity("id").build(); + + @Test + public void analyzeFilesPrunesNonOverlappingOpposingTasksToUnidirectional() { + FakeAddedRowsTask insert = new FakeAddedRowsTask(dataFile("insert", 10L, 20L), 11L); + FakeDeletedDataFileTask delete = new FakeDeletedDataFileTask(dataFile("delete", 30L, 40L), 13L); + + ChangelogScanner.AnalysisResult result = + ChangelogScanner.analyzeFiles( + ImmutableList.of(insert, delete), + SINGLE_RECORD_ID_SCHEMA, + comparator(SINGLE_RECORD_ID_SCHEMA)); + + assertThat(result.bidirectional, empty()); + assertThat(result.unidirectional, contains(delete, insert)); + assertNull(result.overlapLower); + assertNull(result.overlapUpper); + } + + @Test + public void analyzeFilesFindsOverlapDespiteInputOrder() { + FakeDeletedDataFileTask laterDelete = + new FakeDeletedDataFileTask(dataFile("delete-later", 30L, 40L), 13L); + FakeDeletedDataFileTask overlappingDelete = + new FakeDeletedDataFileTask(dataFile("delete-overlap", 15L, 18L), 17L); + FakeAddedRowsTask insert = new FakeAddedRowsTask(dataFile("insert", 10L, 20L), 19L); + + ChangelogScanner.AnalysisResult result = + ChangelogScanner.analyzeFiles( + ImmutableList.of(laterDelete, overlappingDelete, insert), + SINGLE_RECORD_ID_SCHEMA, + comparator(SINGLE_RECORD_ID_SCHEMA)); + + assertThat(result.unidirectional, contains(laterDelete)); + assertThat(result.bidirectional, containsInAnyOrder(overlappingDelete, insert)); + assertEquals(15L, record(result.overlapLower).getField("id")); + assertEquals(18L, record(result.overlapUpper).getField("id")); + } + + @Test + public void analyzeFilesUsesLexicographicCompositePrimaryKeyRanges() { + FakeAddedRowsTask insert = + new FakeAddedRowsTask( + dataFile( + COMPOSITE_PK_SCHEMA, + "insert", + ImmutableMap.of("account", "acct-a", "sequence", 2), + ImmutableMap.of("account", "acct-a", "sequence", 8), + 11L), + 11L); + FakeDeletedDataFileTask overlappingDelete = + new FakeDeletedDataFileTask( + dataFile( + COMPOSITE_PK_SCHEMA, + "delete-overlap", + ImmutableMap.of("account", "acct-a", "sequence", 5), + ImmutableMap.of("account", "acct-b", "sequence", 1), + 13L), + 13L); + FakeDeletedDataFileTask farDelete = + new FakeDeletedDataFileTask( + dataFile( + COMPOSITE_PK_SCHEMA, + "delete-far", + ImmutableMap.of("account", "acct-c", "sequence", 1), + ImmutableMap.of("account", "acct-c", "sequence", 2), + 17L), + 17L); + + ChangelogScanner.AnalysisResult result = + ChangelogScanner.analyzeFiles( + ImmutableList.of(insert, farDelete, overlappingDelete), + COMPOSITE_RECORD_ID_SCHEMA, + comparator(COMPOSITE_RECORD_ID_SCHEMA)); + + assertThat(result.unidirectional, contains(farDelete)); + assertThat(result.bidirectional, containsInAnyOrder(insert, overlappingDelete)); + assertEquals("acct-a", record(result.overlapLower).getField("account").toString()); + assertEquals(5, record(result.overlapLower).getField("sequence")); + assertEquals("acct-a", record(result.overlapUpper).getField("account").toString()); + assertEquals(8, record(result.overlapUpper).getField("sequence")); + } + + @Test + public void analyzeFilesConservativelyRoutesAllTasksWhenMetricsAreMissing() { + FakeAddedRowsTask insert = new FakeAddedRowsTask(dataFile("insert", 10L, 20L), 11L); + FakeDeletedDataFileTask deleteWithMissingMetrics = + new FakeDeletedDataFileTask(dataFileWithoutBounds("delete-missing-metrics"), 13L); + + ChangelogScanner.AnalysisResult result = + ChangelogScanner.analyzeFiles( + ImmutableList.of(insert, deleteWithMissingMetrics), + SINGLE_RECORD_ID_SCHEMA, + comparator(SINGLE_RECORD_ID_SCHEMA)); + + assertThat(result.unidirectional, empty()); + assertThat(result.bidirectional, contains(insert, deleteWithMissingMetrics)); + assertNull(result.overlapLower); + assertNull(result.overlapUpper); + } + + @Test + public void taskBatcherFlushesAtSplitBoundariesWithoutEmptyBatches() { + CapturingOutputReceiver out = new CapturingOutputReceiver(); + ChangelogScanner.TaskBatcher batcher = + new ChangelogScanner.TaskBatcher("default.table", 1234L, 100L, out); + SerializableChangelogTask first = serializableTask("first", 40L); + SerializableChangelogTask second = serializableTask("second", 60L); + SerializableChangelogTask third = serializableTask("third", 1L); + + batcher.add(first, 40L); + batcher.add(second, 60L); + assertThat(out.values, empty()); + + batcher.add(third, 1L); + assertEquals(1, out.values.size()); + batcher.flush(); + + assertEquals(2, out.values.size()); + assertEquals(2, batcher.totalSplits); + assertEquals(new Instant(1234L), out.values.get(0).getTimestamp()); + assertThat(tasksInOutput(out, 0), contains(first, second)); + assertThat(tasksInOutput(out, 1), contains(third)); + } + + @Test + public void taskBatcherAllowsOversizeSingleTaskWithoutEmittingEmptyBatch() { + CapturingOutputReceiver out = new CapturingOutputReceiver(); + ChangelogScanner.TaskBatcher batcher = + new ChangelogScanner.TaskBatcher("default.table", 1234L, 100L, out); + SerializableChangelogTask oversize = serializableTask("oversize", 150L); + + batcher.add(oversize, 150L); + assertThat(out.values, empty()); + + batcher.flush(); + + assertEquals(1, out.values.size()); + assertEquals(1, batcher.totalSplits); + assertThat(tasksInOutput(out, 0), contains(oversize)); + } + + private static Comparator comparator(Schema schema) { + return Comparators.forType(schema.asStruct()); + } + + private static Schema recordIdSchema(Schema schema) { + return TypeUtil.select(schema, schema.identifierFieldIds()); + } + + private static Record record(StructLike structLike) { + return (Record) structLike; + } + + private static List tasksInOutput( + CapturingOutputReceiver out, int index) { + return out.values.get(index).getValue().getValue(); + } + + private static DataFile dataFile(String name, long lower, long upper) { + return dataFile(UNPARTITIONED_SPEC, null, name, lower, upper); + } + + private static DataFile dataFile( + PartitionSpec spec, StructLike partition, String name, long lower, long upper) { + return dataFile( + SINGLE_PK_SCHEMA, + spec, + partition, + name, + ImmutableMap.of("id", lower), + ImmutableMap.of("id", upper), + 100L); + } + + private static DataFile dataFile( + Schema schema, String name, Map lower, Map upper, long size) { + return dataFile(schema, UNPARTITIONED_SPEC, null, name, lower, upper, size); + } + + private static DataFile dataFile( + Schema schema, + PartitionSpec spec, + StructLike partition, + String name, + Map lower, + Map upper, + long size) { + DataFiles.Builder builder = + DataFiles.builder(spec) + .withFormat(FileFormat.PARQUET) + .withPath("gs:://bucket/data/" + name + ".parquet") + .withFileSizeInBytes(size) + .withMetrics( + new Metrics( + 1L, null, null, null, null, bounds(schema, lower), bounds(schema, upper))); + if (partition != null) { + builder.withPartition(partition); + } + return builder.build(); + } + + private static DataFile dataFileWithoutBounds(String name) { + return DataFiles.builder(UNPARTITIONED_SPEC) + .withFormat(FileFormat.PARQUET) + .withPath("gs:://bucket/data/" + name + ".parquet") + .withFileSizeInBytes(100L) + .withMetrics(new Metrics(1L, null, null, null, null, null, null)) + .build(); + } + + private static Map bounds(Schema schema, Map values) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Types.NestedField field : schema.columns()) { + if (values.containsKey(field.name())) { + builder.put( + field.fieldId(), Conversions.toByteBuffer(field.type(), values.get(field.name()))); + } + } + return builder.build(); + } + + private static StructLike partition(long id) { + PartitionKey partitionKey = new PartitionKey(IDENTITY_ID_SPEC, SINGLE_PK_SCHEMA); + GenericRecord record = GenericRecord.create(SINGLE_PK_SCHEMA); + record.setField("id", id); + record.setField("data", "partition-" + id); + partitionKey.partition(record); + return partitionKey; + } + + private static SerializableChangelogTask serializableTask(String name, long length) { + DataFile file = + DataFiles.builder(UNPARTITIONED_SPEC) + .withFormat(FileFormat.PARQUET) + .withPath("gs:://bucket/data/" + name + ".parquet") + .withFileSizeInBytes(length) + .withMetrics(new Metrics(1L, null, null, null, null, null, null)) + .build(); + return SerializableChangelogTask.builder() + .setType(SerializableChangelogTask.Type.ADDED_ROWS) + .setDataFile(SerializableDataFile.from(file, "", false)) + .setSpecId(UNPARTITIONED_SPEC.specId()) + .setOperation(ChangelogOperation.INSERT) + .setOrdinal(0) + .setCommitSnapshotId(1L) + .setStart(0L) + .setLength(length) + .setJsonExpression(ExpressionParser.toJson(Expressions.alwaysTrue())) + .build(); + } + + private abstract static class FakeContentTask + implements ChangelogScanTask, ContentScanTask { + private final DataFile file; + private final PartitionSpec spec; + private final StructLike partition; + private final long length; + + FakeContentTask(DataFile file, long length) { + this(file, UNPARTITIONED_SPEC, file.partition(), length); + } + + FakeContentTask(DataFile file, PartitionSpec spec, StructLike partition, long length) { + this.file = file; + this.spec = spec; + this.partition = partition; + this.length = length; + } + + @Override + public DataFile file() { + return file; + } + + @Override + public PartitionSpec spec() { + return spec; + } + + @Override + public StructLike partition() { + return partition; + } + + @Override + public long start() { + return 0L; + } + + @Override + public long length() { + return length; + } + + @Override + public Expression residual() { + return Expressions.alwaysTrue(); + } + + @Override + public int changeOrdinal() { + return 0; + } + + @Override + public long commitSnapshotId() { + return 1L; + } + } + + private static class FakeAddedRowsTask extends FakeContentTask implements AddedRowsScanTask { + FakeAddedRowsTask(DataFile file, long length) { + super(file, length); + } + + @Override + public List deletes() { + return Collections.emptyList(); + } + + @Override + public ChangelogOperation operation() { + return ChangelogOperation.INSERT; + } + } + + private static class FakeDeletedDataFileTask extends FakeContentTask + implements DeletedDataFileScanTask { + FakeDeletedDataFileTask(DataFile file, long length) { + super(file, length); + } + + @Override + public List existingDeletes() { + return Collections.emptyList(); + } + + @Override + public ChangelogOperation operation() { + return ChangelogOperation.DELETE; + } + } + + private static final class CapturingOutputReceiver + implements DoFn.OutputReceiver>> { + private final ImmutableList.Builder< + TimestampedValue>>> + builder = ImmutableList.builder(); + private List>>> + values = ImmutableList.of(); + + @Override + public OutputBuilder>> builder( + KV> value) { + throw new UnsupportedOperationException("Use outputWithTimestamp in this test receiver."); + } + + @Override + public void outputWithTimestamp( + KV> value, Instant timestamp) { + builder.add(TimestampedValue.of(value, timestamp)); + values = builder.build(); + } + } +} 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 index 32d0893fa06f..bf3aaf414579 100644 --- 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 @@ -18,6 +18,8 @@ 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; @@ -28,6 +30,7 @@ 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; @@ -39,6 +42,7 @@ 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; @@ -75,9 +79,13 @@ public class DeleteReaderTest { .withRecordCount(2) .build(); - // ------------------------------------------------------------------------ - // Test infrastructure - // ------------------------------------------------------------------------ + 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 { @@ -102,6 +110,21 @@ private static class StubDeleteReader extends DeleteReader { 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; @@ -118,16 +141,20 @@ protected DeleteLoader newDeleteLoader() { } } - /** {@link DeleteLoader} that returns pre-built indexes. Unused arms return empty. */ + /** {@link DeleteLoader} that returns pre-built indexes. */ private static class StubLoader implements DeleteLoader { private final PositionDeleteIndex posIndex; - private final StructLikeSet eqSet; + 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.eqSet = eqSet; + this.eqSets = eqSets; } @Override @@ -139,7 +166,9 @@ public PositionDeleteIndex loadPositionDeletes(Iterable files, CharS @Override public StructLikeSet loadEqualityDeletes(Iterable files, Schema schema) { eqLoadCount++; - return eqSet; + return eqSets.getOrDefault( + Sets.newHashSet(TypeUtil.getProjectedIds(new Schema(schema.asStruct().fields()))), + StructLikeSet.create(schema.asStruct())); } } @@ -190,6 +219,17 @@ private static StructLikeSet eqSetOfIds(int... ids) { 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; @@ -217,8 +257,8 @@ private static List idsOf(CloseableIterable records) { /** With no delete files at all, {@code read()} emits nothing. */ @Test public void noDeletesEmitsNothing() { - StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); - StubDeleteReader reader = new StubDeleteReader(Collections.emptyList(), loader); + 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)); @@ -229,8 +269,8 @@ public void noDeletesEmitsNothing() { /** Pos-only emits only the pos-deleted records. */ @Test public void posOnlyEmitsPosDeletedRecords() { - StubLoader loader = new StubLoader(posIndexOf(1L, 3L), eqSetOfIds()); - StubDeleteReader reader = new StubDeleteReader(ImmutableList.of(POS_FILE), loader); + 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)); @@ -241,8 +281,8 @@ public void posOnlyEmitsPosDeletedRecords() { /** Only equality deletes, emits records matching the eq set. */ @Test public void eqOnlyEmitsEqDeletedRecords() { - StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds(2, 4)); - StubDeleteReader reader = new StubDeleteReader(ImmutableList.of(EQ_FILE_ID), loader); + 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)); @@ -253,8 +293,9 @@ public void eqOnlyEmitsEqDeletedRecords() { /** Pos-deletes plus equality deletes, emit the union without duplication. */ @Test public void posAndEqEmitUnion() { - StubLoader loader = new StubLoader(posIndexOf(0L, 4L), eqSetOfIds(2, 4)); - StubDeleteReader reader = new StubDeleteReader(ImmutableList.of(POS_FILE, EQ_FILE_ID), loader); + 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)); @@ -268,7 +309,7 @@ public void posAndEqEmitUnion() { public void preloadedPositionDeletesAvoidSecondLoad() { StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); PositionDeleteIndex preloadedPosIndex = posIndexOf(1L, 3L); - StubDeleteReader reader = + DeleteReader reader = new StubDeleteReader( ImmutableList.of(POS_FILE), loader, @@ -287,7 +328,7 @@ public void preloadedEqualityDeletesAvoidSecondLoad() { StubLoader loader = new StubLoader(posIndexOf(), eqSetOfIds()); Map, StructLikeSet> preloadedEqSets = new HashMap<>(); preloadedEqSets.put(Collections.singleton(1), eqSetOfIds(2, 4)); - StubDeleteReader reader = + DeleteReader reader = new StubDeleteReader( ImmutableList.of(EQ_FILE_ID), loader, @@ -299,4 +340,80 @@ public void preloadedEqualityDeletesAvoidSecondLoad() { 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/IncrementalChangelogSourceTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSourceTest.java new file mode 100644 index 000000000000..4f456ff3fd2b --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSourceTest.java @@ -0,0 +1,279 @@ +/* + * 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.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.TestDataWarehouse; +import org.apache.beam.sdk.io.iceberg.TestFixtures; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reify; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.ValueKind; +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.iceberg.DataFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; +import org.joda.time.Instant; +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; + +/** Integration tests for {@link IncrementalChangelogSource}. */ +@RunWith(JUnit4.class) +public class IncrementalChangelogSourceTest { + private static final org.apache.iceberg.Schema CDC_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + required(1, "id", Types.LongType.get()), optional(2, "data", Types.StringType.get())), + ImmutableSet.of(1)); + + private static final org.apache.iceberg.Schema EVENT_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "event_time", Types.TimestampType.withoutZone())), + 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(); + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void boundedSnapshotRangeEmitsOnlyRequestedSnapshotsWithProjectedSchema() + throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + commitAppend(table, "s1.parquet", records(1L, "one")); + commitAppend(table, "s2.parquet", records(2L, "two")); + commitAppend(table, "s3.parquet", records(3L, "three")); + commitAppend(table, "s4.parquet", records(4L, "four")); + List snapshots = Lists.newArrayList(table.snapshots()); + + IcebergScanConfig scanConfig = + baseConfigBuilder(table, tableId) + .setKeepFields(ImmutableList.of("id")) + .setFromSnapshotInclusive(snapshots.get(1).snapshotId()) + .setToSnapshot(snapshots.get(2).snapshotId()) + .build(); + Schema projectedSchema = Schema.builder().addInt64Field("id").build(); + + PCollection rows = pipeline.apply(new IncrementalChangelogSource(scanConfig)); + + assertThat(rows.isBounded(), equalTo(IsBounded.BOUNDED)); + assertEquals(projectedSchema, rows.getSchema()); + PAssert.that(rows) + .containsInAnyOrder( + Row.withSchema(projectedSchema).addValue(2L).build(), + Row.withSchema(projectedSchema).addValue(3L).build()); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void streamingSnapshotRangeTerminatesWithoutDuplicates() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + commitAppend(table, "s1.parquet", records(1L, "one")); + commitAppend(table, "s2.parquet", records(2L, "two")); + commitAppend(table, "s3.parquet", records(3L, "three")); + + IcebergScanConfig scanConfig = + baseConfigBuilder(table, tableId) + .setStreaming(true) + .setStartingStrategy(StartingStrategy.EARLIEST) + .setToSnapshot(table.currentSnapshot().snapshotId()) + .build(); + Schema rowSchema = IcebergUtils.icebergSchemaToBeamSchema(CDC_SCHEMA); + + PCollection rows = pipeline.apply(new IncrementalChangelogSource(scanConfig)); + + assertThat(rows.isBounded(), equalTo(IsBounded.UNBOUNDED)); + PAssert.that(rows) + .containsInAnyOrder( + Row.withSchema(rowSchema).addValues(1L, "one").build(), + Row.withSchema(rowSchema).addValues(2L, "two").build(), + Row.withSchema(rowSchema).addValues(3L, "three").build()); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void overwriteUpdatePairsAreResolvedWithinSnapshotWindow() throws Exception { + TableIdentifier tableId = tableId(); + Table table = + warehouse.createTable( + tableId, + CDC_SCHEMA, + null, + ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", TableProperties.SPLIT_SIZE, "1")); + DataFile oldFile = + commitAppend( + table, "old.parquet", ImmutableList.of(record(1L, "before"), record(2L, "same"))); + DataFile newFile = + warehouse.writeRecords( + testName.getMethodName() + "-new.parquet", + table.schema(), + ImmutableList.of(record(1L, "after"), record(2L, "same"))); + table.newOverwrite().deleteFile(oldFile).addFile(newFile).commit(); + table.refresh(); + + IcebergScanConfig scanConfig = + baseConfigBuilder(table, tableId) + .setFromSnapshotInclusive(table.currentSnapshot().snapshotId()) + .setToSnapshot(table.currentSnapshot().snapshotId()) + .build(); + + PCollection changes = + pipeline + .apply(new IncrementalChangelogSource(scanConfig)) + .apply("Format Changes", ParDo.of(new FormatValueKindAndRow())); + + PAssert.that(changes).containsInAnyOrder("UPDATE_BEFORE:1:before", "UPDATE_AFTER:1:after"); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void watermarkColumnRestampsProjectedRows() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, EVENT_SCHEMA, null, tableProperties()); + java.time.Instant firstEventInstant = + java.time.Instant.ofEpochMilli(System.currentTimeMillis() - 1_000L); + java.time.Instant secondEventInstant = firstEventInstant.plusMillis(5_000L); + LocalDateTime firstEvent = LocalDateTime.ofInstant(firstEventInstant, ZoneOffset.UTC); + LocalDateTime secondEvent = LocalDateTime.ofInstant(secondEventInstant, ZoneOffset.UTC); + commitAppend( + table, + "events.parquet", + ImmutableList.of(eventRecord(1L, "one", firstEvent), eventRecord(2L, "two", secondEvent))); + + IcebergScanConfig scanConfig = + baseConfigBuilder(table, tableId) + .setKeepFields(ImmutableList.of("id", "event_time")) + .setWatermarkColumn("event_time") + .setToSnapshot(table.currentSnapshot().snapshotId()) + .build(); + Schema projectedSchema = + Schema.builder() + .addInt64Field("id") + .addLogicalTypeField( + "event_time", org.apache.beam.sdk.schemas.logicaltypes.SqlTypes.DATETIME) + .build(); + + PCollection rows = pipeline.apply(new IncrementalChangelogSource(scanConfig)); + + assertEquals(projectedSchema, rows.getSchema()); + PAssert.that(rows.apply(Reify.timestamps())) + .containsInAnyOrder( + TimestampedValue.of( + Row.withSchema(projectedSchema).addValues(1L, firstEvent).build(), + new Instant(firstEventInstant.toEpochMilli())), + TimestampedValue.of( + Row.withSchema(projectedSchema).addValues(2L, secondEvent).build(), + new Instant(secondEventInstant.toEpochMilli()))); + + pipeline.run().waitUntilFinish(); + } + + private TableIdentifier tableId() { + return TableIdentifier.of("default", testName.getMethodName()); + } + + private IcebergScanConfig.Builder baseConfigBuilder(Table table, TableIdentifier tableId) { + return IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) + .setUseCdc(true); + } + + private static Map tableProperties() { + return ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + } + + private DataFile commitAppend(Table table, String fileName, List records) + throws IOException { + DataFile file = + warehouse.writeRecords(testName.getMethodName() + "-" + fileName, table.schema(), records); + table.newFastAppend().appendFile(file).commit(); + table.refresh(); + return file; + } + + private static List records(long id, String data) { + return ImmutableList.of(record(id, data)); + } + + private static Record record(long id, String data) { + return TestFixtures.createRecord(CDC_SCHEMA, ImmutableMap.of("id", id, "data", data)); + } + + private static Record eventRecord(long id, String data, LocalDateTime eventTime) { + return TestFixtures.createRecord( + EVENT_SCHEMA, ImmutableMap.of("id", id, "data", data, "event_time", eventTime)); + } + + private static final class FormatValueKindAndRow extends DoFn { + @ProcessElement + public void process( + @Element Row row, ValueKind valueKind, OutputReceiver outputReceiver) { + outputReceiver.output( + valueKind.name() + ":" + row.getInt64("id") + ":" + row.getString("data")); + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFnIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFnIT.java new file mode 100644 index 000000000000..24bb3f5552fa --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/LocalResolveDoFnIT.java @@ -0,0 +1,224 @@ +/* + * 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.empty; +import static org.junit.Assert.assertEquals; + +import java.util.List; +import java.util.Map; +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.TestDataWarehouse; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.ValueKind; +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.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.joda.time.Instant; +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; + +/** Integration tests for {@link LocalResolveDoFn}. */ +@RunWith(JUnit4.class) +public class LocalResolveDoFnIT { + 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, "visible", Types.StringType.get()), + Types.NestedField.optional(3, "hidden", 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 copyOnWriteRewriteOfIdenticalRowsIsDropped() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + IcebergScanConfig scanConfig = scanConfig(table, tableId); + DataFile oldFile = + warehouse.writeRecords( + testName.getMethodName() + "-old.parquet", + table.schema(), + ImmutableList.of(record(1L, "shown", "same-hidden"))); + DataFile newFile = + warehouse.writeRecords( + testName.getMethodName() + "-new.parquet", + table.schema(), + ImmutableList.of(record(1L, "shown", "same-hidden"))); + + List> output = + process( + scanConfig, + descriptor(tableId, 1L, 1L), + ImmutableList.of( + task(SerializableChangelogTask.Type.DELETED_FILE, oldFile, table, 300L), + task(SerializableChangelogTask.Type.ADDED_ROWS, newFile, table, 300L)), + new Instant(0L)); + + assertThat(output, empty()); + } + + @Test + public void hiddenOnlyUpdateIsResolvedBeforeProjection() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + IcebergScanConfig scanConfig = scanConfig(table, tableId); + DataFile oldFile = + warehouse.writeRecords( + testName.getMethodName() + "-old.parquet", + table.schema(), + ImmutableList.of(record(1L, "shown", "old-hidden"))); + DataFile newFile = + warehouse.writeRecords( + testName.getMethodName() + "-new.parquet", + table.schema(), + ImmutableList.of(record(1L, "shown", "new-hidden"))); + Instant timestamp = new Instant(1234L); + + List> output = + process( + scanConfig, + descriptor(tableId, 1L, 1L), + ImmutableList.of( + task(SerializableChangelogTask.Type.DELETED_FILE, oldFile, table, 301L), + task(SerializableChangelogTask.Type.ADDED_ROWS, newFile, table, 301L)), + timestamp); + + assertThat( + output.stream().map(LocalResolveDoFnIT::kindAndProjectedRow).collect(Collectors.toList()), + contains("UPDATE_BEFORE:1:shown:2", "UPDATE_AFTER:1:shown:2")); + assertEquals( + ImmutableList.of(timestamp, timestamp), + output.stream().map(ValueInSingleWindow::getTimestamp).collect(Collectors.toList())); + } + + private List> process( + IcebergScanConfig scanConfig, + ChangelogDescriptor descriptor, + List tasks, + Instant timestamp) + throws Exception { + try (DoFnTester>, Row> tester = + DoFnTester.of(new LocalResolveDoFn(scanConfig))) { + tester.processTimestampedElement(TimestampedValue.of(KV.of(descriptor, tasks), timestamp)); + return tester.getMutableOutput(tester.getMainOutputTag()); + } + } + + private TableIdentifier tableId() { + return TableIdentifier.of("default", testName.getMethodName()); + } + + private IcebergScanConfig scanConfig(Table table, TableIdentifier tableId) { + return IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) + .setKeepFields(ImmutableList.of("id", "visible")) + .setUseCdc(true) + .build(); + } + + private static ChangelogDescriptor descriptor( + TableIdentifier tableId, long lowerInclusive, long upperInclusive) { + org.apache.beam.sdk.schemas.Schema pkSchema = + org.apache.beam.sdk.schemas.Schema.builder().addInt64Field("id").build(); + return ChangelogDescriptor.builder() + .setTableIdentifierString(tableId.toString()) + .setOverlapLower(Row.withSchema(pkSchema).addValue(lowerInclusive).build()) + .setOverlapUpper(Row.withSchema(pkSchema).addValue(upperInclusive).build()) + .build(); + } + + private static Record record(long id, String visible, String hidden) { + GenericRecord record = GenericRecord.create(CDC_SCHEMA); + record.setField("id", id); + record.setField("visible", visible); + record.setField("hidden", hidden); + return record; + } + + private static SerializableChangelogTask task( + SerializableChangelogTask.Type type, DataFile dataFile, Table table, long snapshotId) { + return SerializableChangelogTask.builder() + .setType(type) + .setDataFile(dataFile, table.spec().partitionToPath(dataFile.partition()), true) + .setAddedDeletes(ImmutableList.of()) + .setExistingDeletes(ImmutableList.of()) + .setSpecId(table.spec().specId()) + .setOperation( + type == SerializableChangelogTask.Type.ADDED_ROWS + ? ChangelogOperation.INSERT + : ChangelogOperation.DELETE) + .setOrdinal(0) + .setCommitSnapshotId(snapshotId) + .setStart(0L) + .setLength(dataFile.fileSizeInBytes()) + .setJsonExpression(ExpressionParser.toJson(Expressions.alwaysTrue())) + .build(); + } + + private static Map tableProperties() { + return ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + } + + private static String kindAndProjectedRow(ValueInSingleWindow value) { + ValueKind kind = value.getValueKind(); + Row row = value.getValue(); + return kind.name() + + ":" + + row.getInt64("id") + + ":" + + row.getString("visible") + + ":" + + row.getSchema().getFieldCount(); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRangeTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRangeTest.java new file mode 100644 index 000000000000..8e0db8a930fa --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/OverlapRangeTest.java @@ -0,0 +1,163 @@ +/* + * 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.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +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.TableCache; +import org.apache.beam.sdk.io.iceberg.TestDataWarehouse; +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.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; +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; + +/** Unit tests for {@link OverlapRange}. */ +@RunWith(JUnit4.class) +public class OverlapRangeTest { + private static final org.apache.iceberg.Schema SINGLE_PK_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())), + ImmutableSet.of(1)); + + private static final org.apache.iceberg.Schema COMPOSITE_PK_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + Types.NestedField.optional(3, "data", Types.StringType.get()), + Types.NestedField.required(1, "account", Types.StringType.get()), + Types.NestedField.optional(4, "extra", Types.IntegerType.get()), + Types.NestedField.required(2, "sequence", Types.IntegerType.get())), + ImmutableSet.of(1, 2)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + @Rule public final TestName testName = new TestName(); + + @Test + public void containsUsesInclusiveSingleColumnBounds() throws Exception { + OverlapRange range = overlapRange(SINGLE_PK_SCHEMA); + StructLike lower = range.toStructLike(pkRow(range.recordIdSchema(), 10)); + StructLike upper = range.toStructLike(pkRow(range.recordIdSchema(), 20)); + + assertFalse(range.contains(singlePkRecord(9), lower, upper)); + assertTrue(range.contains(singlePkRecord(10), lower, upper)); + assertTrue(range.contains(singlePkRecord(15), lower, upper)); + assertTrue(range.contains(singlePkRecord(20), lower, upper)); + assertFalse(range.contains(singlePkRecord(21), lower, upper)); + } + + @Test + public void containsUsesLexicographicCompositeBounds() throws Exception { + OverlapRange range = overlapRange(COMPOSITE_PK_SCHEMA); + StructLike lower = range.toStructLike(pkRow(range.recordIdSchema(), "a", 2)); + StructLike upper = range.toStructLike(pkRow(range.recordIdSchema(), "b", 1)); + + assertFalse(range.contains(compositePkRecord("a", 1), lower, upper)); + assertTrue(range.contains(compositePkRecord("a", 2), lower, upper)); + assertTrue(range.contains(compositePkRecord("a", 9), lower, upper)); + assertTrue(range.contains(compositePkRecord("b", 0), lower, upper)); + assertTrue(range.contains(compositePkRecord("b", 1), lower, upper)); + assertFalse(range.contains(compositePkRecord("b", 2), lower, upper)); + } + + @Test + public void nullBoundsAreConservative() throws Exception { + OverlapRange range = overlapRange(SINGLE_PK_SCHEMA); + StructLike lower = range.toStructLike(pkRow(range.recordIdSchema(), 10)); + StructLike upper = range.toStructLike(pkRow(range.recordIdSchema(), 20)); + + assertNull(range.toStructLike(null)); + assertTrue(range.contains(singlePkRecord(1), null, upper)); + assertTrue(range.contains(singlePkRecord(100), lower, null)); + assertTrue(range.contains(singlePkRecord(100), null, null)); + } + + @Test + public void recordIdProjectionUsesIdentifierFieldsFromFullRecord() throws Exception { + OverlapRange range = overlapRange(COMPOSITE_PK_SCHEMA); + StructLike lower = range.toStructLike(pkRow(range.recordIdSchema(), "acct", 7)); + StructLike upper = range.toStructLike(pkRow(range.recordIdSchema(), "acct", 7)); + + assertTrue(range.contains(compositePkRecord("acct", 7), lower, upper)); + + assertEquals("acct", range.recordIdProjection().get(0, String.class)); + assertEquals(7, (int) range.recordIdProjection().get(1, Integer.class)); + } + + private OverlapRange overlapRange(org.apache.iceberg.Schema schema) throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + IcebergCatalogConfig catalogConfig = + IcebergCatalogConfig.builder() + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build(); + catalogConfig.catalog().createTable(tableId, schema); + IcebergScanConfig scanConfig = + IcebergScanConfig.builder() + .setCatalogConfig(catalogConfig) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(schema)) + .setUseCdc(true) + .build(); + TableCache.setup(scanConfig); + return OverlapRange.forScanConfig(scanConfig); + } + + private static Row pkRow(Schema recordIdSchema, Object... values) { + return Row.withSchema(IcebergUtils.icebergSchemaToBeamSchema(recordIdSchema)) + .addValues(values) + .build(); + } + + private static Record singlePkRecord(int id) { + GenericRecord record = GenericRecord.create(SINGLE_PK_SCHEMA); + record.setField("id", id); + record.setField("data", "v" + id); + return record; + } + + private static Record compositePkRecord(String account, int sequence) { + GenericRecord record = GenericRecord.create(COMPOSITE_PK_SCHEMA); + record.setField("data", "payload"); + record.setField("account", account); + record.setField("extra", 100); + record.setField("sequence", sequence); + return record; + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogsTest.java new file mode 100644 index 000000000000..44d033594994 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogsTest.java @@ -0,0 +1,356 @@ +/* + * 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.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.KvCoder; +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.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.ValueKind; +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.TableProperties; +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.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.types.TypeUtil; +import org.apache.iceberg.types.Types; +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 ReadFromChangelogs}. */ +@RunWith(JUnit4.class) +public class ReadFromChangelogsTest { + 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, "visible", Types.StringType.get()), + Types.NestedField.optional(3, "hidden", 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(); + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void unidirectionalTasksEmitProjectedRowsOnly() throws IOException { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + IcebergScanConfig scanConfig = scanConfig(table, tableId, ImmutableList.of("id", "visible")); + + DataFile addedFile = + warehouse.writeRecords( + testName.getMethodName() + "-added.parquet", + table.schema(), + ImmutableList.of(record(10L, "added", "added-hidden"))); + DataFile deletedRowsFile = + warehouse.writeRecords( + testName.getMethodName() + "-deleted-rows.parquet", + table.schema(), + ImmutableList.of( + record(20L, "deleted-row", "deleted-row-hidden"), + record(21L, "not-deleted", "not-deleted-hidden"))); + DeleteFile addedPositionDelete = + writePositionDelete(table, deletedRowsFile, "deleted-rows-pos-delete.parquet", 0L); + DataFile deletedFile = + warehouse.writeRecords( + testName.getMethodName() + "-deleted-file.parquet", + table.schema(), + ImmutableList.of(record(30L, "deleted-file", "deleted-file-hidden"))); + + List tasks = + ImmutableList.of( + task( + SerializableChangelogTask.Type.ADDED_ROWS, + addedFile, + ImmutableList.of(), + ImmutableList.of(), + table, + 100L), + task( + SerializableChangelogTask.Type.DELETED_ROWS, + deletedRowsFile, + ImmutableList.of(addedPositionDelete), + ImmutableList.of(), + table, + 100L), + task( + SerializableChangelogTask.Type.DELETED_FILE, + deletedFile, + ImmutableList.of(), + ImmutableList.of(), + table, + 100L)); + + ReadFromChangelogs.CdcOutput output = + input(ImmutableList.of(KV.of(descriptor(), tasks)), ImmutableList.of()) + .apply(new ReadFromChangelogs(scanConfig)); + + assertEquals( + IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()), + output.uniDirectionalRows().getSchema()); + PAssert.that( + output.uniDirectionalRows().apply("Format Unidirectional", ParDo.of(new FormatRow()))) + .containsInAnyOrder( + "INSERT:10:added:2", "DELETE:20:deleted-row:2", "DELETE:30:deleted-file:2"); + PAssert.that(output.biDirectionalInserts()).empty(); + PAssert.that(output.biDirectionalDeletes()).empty(); + + pipeline.run().waitUntilFinish(); + } + + @Test + public void bidirectionalTasksKeepFullRowsForDownstreamResolution() throws IOException { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + IcebergScanConfig scanConfig = scanConfig(table, tableId, ImmutableList.of("id", "visible")); + DataFile oldFile = + warehouse.writeRecords( + testName.getMethodName() + "-old.parquet", + table.schema(), + ImmutableList.of(record(1L, "shown", "old-hidden"))); + DataFile newFile = + warehouse.writeRecords( + testName.getMethodName() + "-new.parquet", + table.schema(), + ImmutableList.of(record(1L, "shown", "new-hidden"))); + List tasks = + ImmutableList.of( + task( + SerializableChangelogTask.Type.DELETED_FILE, + oldFile, + ImmutableList.of(), + ImmutableList.of(), + table, + 200L), + task( + SerializableChangelogTask.Type.ADDED_ROWS, + newFile, + ImmutableList.of(), + ImmutableList.of(), + table, + 200L)); + + ReadFromChangelogs.CdcOutput output = + input(ImmutableList.of(), ImmutableList.of(KV.of(descriptor(1L, 1L), tasks))) + .apply(new ReadFromChangelogs(scanConfig)); + + PAssert.that(output.uniDirectionalRows()).empty(); + PAssert.that( + output.biDirectionalDeletes().apply("Format Deletes", ParDo.of(new FormatKeyedRow()))) + .containsInAnyOrder("DELETE:200:1:shown:old-hidden:3"); + PAssert.that( + output.biDirectionalInserts().apply("Format Inserts", ParDo.of(new FormatKeyedRow()))) + .containsInAnyOrder("INSERT:200:1:shown:new-hidden:3"); + + pipeline.run().waitUntilFinish(); + } + + private PCollectionTuple input( + List>> unidirectional, + List>> largeBidirectional) { + Schema rowIdBeamSchema = + IcebergUtils.icebergSchemaToBeamSchema( + TypeUtil.select(CDC_SCHEMA, CDC_SCHEMA.identifierFieldIds())); + KvCoder> coder = + ChangelogScanner.coder(rowIdBeamSchema); + PCollection>> uni = + unidirectional.isEmpty() + ? pipeline.apply("Empty Unidirectional", Create.empty(coder)) + : pipeline.apply("Create Unidirectional", Create.of(unidirectional).withCoder(coder)); + PCollection>> large = + largeBidirectional.isEmpty() + ? pipeline.apply("Empty Large Bidirectional", Create.empty(coder)) + : pipeline.apply( + "Create Large Bidirectional", Create.of(largeBidirectional).withCoder(coder)); + return PCollectionTuple.of(ChangelogScanner.UNIDIRECTIONAL_TASKS, uni) + .and(ChangelogScanner.LARGE_BIDIRECTIONAL_TASKS, large); + } + + private TableIdentifier tableId() { + return TableIdentifier.of("default", testName.getMethodName()); + } + + private IcebergScanConfig scanConfig( + Table table, TableIdentifier tableId, List keepFields) { + return IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) + .setKeepFields(keepFields) + .setUseCdc(true) + .build(); + } + + private ChangelogDescriptor descriptor() { + return ChangelogDescriptor.builder().setTableIdentifierString(tableId().toString()).build(); + } + + private ChangelogDescriptor descriptor(long lowerInclusive, long upperInclusive) { + Schema pkSchema = Schema.builder().addInt64Field("id").build(); + return ChangelogDescriptor.builder() + .setTableIdentifierString(tableId().toString()) + .setOverlapLower(Row.withSchema(pkSchema).addValue(lowerInclusive).build()) + .setOverlapUpper(Row.withSchema(pkSchema).addValue(upperInclusive).build()) + .build(); + } + + private static Record record(long id, String visible, String hidden) { + GenericRecord record = GenericRecord.create(CDC_SCHEMA); + record.setField("id", id); + record.setField("visible", visible); + record.setField("hidden", hidden); + 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 { + for (long position : positions) { + writer.write(PositionDelete.create().set(dataFile.location(), position)); + } + } finally { + writer.close(); + } + return writer.toDeleteFile(); + } + + private static SerializableChangelogTask task( + SerializableChangelogTask.Type type, + DataFile dataFile, + List addedDeletes, + List existingDeletes, + Table table, + long snapshotId) { + 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(snapshotId) + .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 Map tableProperties() { + return ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + } + + private static class FormatRow extends DoFn { + @ProcessElement + public void process(@Element Row row, ValueKind kind, OutputReceiver out) { + out.output( + kind.name() + + ":" + + row.getInt64("id") + + ":" + + row.getString("visible") + + ":" + + row.getSchema().getFieldCount()); + } + } + + private static class FormatKeyedRow extends DoFn, Row>, String> { + @ProcessElement + public void process( + @Element KV, Row> element, ValueKind kind, OutputReceiver out) { + Row row = element.getValue(); + out.output( + kind.name() + + ":" + + element.getKey().getKey() + + ":" + + element.getKey().getValue().getInt64("id") + + ":" + + row.getString("visible") + + ":" + + row.getString("hidden") + + ":" + + row.getSchema().getFieldCount()); + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChangesTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChangesTest.java new file mode 100644 index 000000000000..9dc16c111818 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/ResolveChangesTest.java @@ -0,0 +1,210 @@ +/* + * 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.empty; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.Collections; +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.TestDataWarehouse; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.ValueKind; +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.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.joda.time.Instant; +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; + +/** Unit tests for {@link ResolveChanges}. */ +@RunWith(JUnit4.class) +public class ResolveChangesTest { + private static final org.apache.iceberg.Schema SIMPLE_ICEBERG_SCHEMA = + new org.apache.iceberg.Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())), + ImmutableSet.of(1)); + private static final Schema SIMPLE_BEAM_SCHEMA = + IcebergUtils.icebergSchemaToBeamSchema(SIMPLE_ICEBERG_SCHEMA); + private static final Schema PK_SCHEMA = Schema.builder().addInt32Field("id").build(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + @Rule public final TestName testName = new TestName(); + + @Test + public void fullRowDuplicateDeleteInsertEmitsNothing() throws Exception { + Row duplicate = simpleRow(1, "duplicate"); + + List> output = + process( + SIMPLE_ICEBERG_SCHEMA, + pkRow(1), + Collections.singletonList(duplicate), + Collections.singletonList(duplicate), + new Instant(0L)); + + assertThat(output, empty()); + } + + @Test + public void updatePairAndExtraRowsPreserveKindsAndTimestamp() throws Exception { + Instant timestamp = new Instant(123L); + Row before = simpleRow(1, "old"); + Row after = simpleRow(1, "new"); + Row extraDelete = simpleRow(1, "deleted-only"); + + List> output = + process( + SIMPLE_ICEBERG_SCHEMA, + pkRow(1), + Arrays.asList(before, extraDelete), + Collections.singletonList(after), + timestamp); + + assertThat( + output.stream().map(ResolveChangesTest::kindAndData).collect(Collectors.toList()), + contains("UPDATE_BEFORE:old", "UPDATE_AFTER:new", "DELETE:deleted-only")); + assertEquals( + Collections.nCopies(3, timestamp), + output.stream().map(ValueInSingleWindow::getTimestamp).collect(Collectors.toList())); + } + + @Test + public void duplicateDetectionUsesDeepEqualityForNestedValues() throws Exception { + org.apache.iceberg.Schema icebergSchema = + new org.apache.iceberg.Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional( + 2, + "nested", + Types.StructType.of( + Types.NestedField.optional(3, "name", Types.StringType.get()))), + Types.NestedField.optional( + 4, "items", Types.ListType.ofOptional(5, Types.StringType.get())), + Types.NestedField.optional( + 6, + "attrs", + Types.MapType.ofOptional( + 7, 8, Types.StringType.get(), Types.IntegerType.get())), + Types.NestedField.optional(9, "payload", Types.BinaryType.get()), + Types.NestedField.optional(10, "nullable", Types.StringType.get())), + ImmutableSet.of(1)); + Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(icebergSchema); + Schema nestedSchema = beamSchema.getField("nested").getType().getRowSchema(); + Row delete = + Row.withSchema(beamSchema) + .addValues( + 1, + Row.withSchema(nestedSchema).addValue("same").build(), + ImmutableList.of("a", "b"), + ImmutableMap.of("x", 1), + new byte[] {1, 2, 3}, + null) + .build(); + Row insert = + Row.withSchema(beamSchema) + .addValues( + 1, + Row.withSchema(nestedSchema).addValue("same").build(), + ImmutableList.of("a", "b"), + ImmutableMap.of("x", 1), + new byte[] {1, 2, 3}, + null) + .build(); + + List> output = + process( + icebergSchema, + pkRow(1), + Collections.singletonList(delete), + Collections.singletonList(insert), + new Instant(0L)); + + assertThat(output, empty()); + } + + private List> process( + org.apache.iceberg.Schema icebergSchema, + Row pk, + List deletes, + List inserts, + Instant timestamp) + throws Exception { + CoGbkResult result = + CoGbkResult.of(ResolveChanges.DELETES, deletes).and(ResolveChanges.INSERTS, inserts); + try (DoFnTester, CoGbkResult>, Row> tester = + DoFnTester.of(new ResolveChanges(scanConfig(icebergSchema)))) { + tester.processTimestampedElement( + TimestampedValue.of(KV.of(KV.of(101L, pk), result), timestamp)); + return tester.getMutableOutput(tester.getMainOutputTag()); + } + } + + private IcebergScanConfig scanConfig(org.apache.iceberg.Schema icebergSchema) { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + IcebergCatalogConfig catalogConfig = + IcebergCatalogConfig.builder() + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build(); + catalogConfig.catalog().createTable(tableId, icebergSchema); + return IcebergScanConfig.builder() + .setCatalogConfig(catalogConfig) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(icebergSchema)) + .setUseCdc(true) + .build(); + } + + private static Row simpleRow(int id, String data) { + return Row.withSchema(SIMPLE_BEAM_SCHEMA).addValues(id, data).build(); + } + + private static Row pkRow(int id) { + return Row.withSchema(PK_SCHEMA).addValue(id).build(); + } + + private static String kindAndData(ValueInSingleWindow value) { + ValueKind kind = value.getValueKind(); + return kind.name() + ":" + value.getValue().getString("data"); + } +} 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; + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFnTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFnTest.java new file mode 100644 index 000000000000..c3c081c1eed7 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/SnapshotWindowFnTest.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 static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import java.util.Collection; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link SnapshotWindowFn}. */ +@RunWith(JUnit4.class) +public class SnapshotWindowFnTest { + @Test + public void identicalTimestampsShareWindowAndAdjacentTimestampsDoNot() throws Exception { + SnapshotWindowFn fn = new SnapshotWindowFn(); + Instant timestamp = new Instant(1_000L); + + IntervalWindow first = onlyWindow(fn, timestamp); + IntervalWindow second = onlyWindow(fn, timestamp); + IntervalWindow adjacent = onlyWindow(fn, timestamp.plus(Duration.millis(1))); + + assertEquals(new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1))), first); + assertEquals(first, second); + assertNotEquals(first, adjacent); + assertEquals( + new IntervalWindow(timestamp.plus(Duration.millis(1)), timestamp.plus(Duration.millis(2))), + adjacent); + } + + @Test + public void sideInputMappingStartsAtMainWindowMaxTimestamp() { + SnapshotWindowFn fn = new SnapshotWindowFn(); + IntervalWindow mainWindow = new IntervalWindow(new Instant(10L), new Instant(20L)); + + IntervalWindow sideInputWindow = fn.getDefaultWindowMappingFn().getSideInputWindow(mainWindow); + + assertEquals( + new IntervalWindow( + mainWindow.maxTimestamp(), mainWindow.maxTimestamp().plus(Duration.millis(1L))), + sideInputWindow); + } + + @SuppressWarnings("NonCanonicalType") + private static IntervalWindow onlyWindow(SnapshotWindowFn fn, Instant timestamp) + throws Exception { + Collection windows = + fn.assignWindows( + fn.new AssignContext() { + @Override + public Object element() { + return "element"; + } + + @Override + public Instant timestamp() { + return timestamp; + } + + @Override + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + }); + assertThat( + windows, contains(new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1L))))); + return windows.iterator().next(); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdfTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdfTest.java new file mode 100644 index 000000000000..c62a9d6fb4ed --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/cdc/WatchForSnapshotsSdfTest.java @@ -0,0 +1,265 @@ +/* + * 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.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.sdk.io.iceberg.IcebergScanConfig; +import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.TestDataWarehouse; +import org.apache.beam.sdk.io.iceberg.TestFixtures; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; +import org.apache.beam.sdk.values.OutputBuilder; +import org.apache.beam.sdk.values.TimestampedValue; +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.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; +import org.joda.time.Duration; +import org.joda.time.Instant; +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; + +/** Unit tests for {@link WatchForSnapshotsSdf}. */ +@RunWith(JUnit4.class) +public class WatchForSnapshotsSdfTest { + private static final Schema CDC_SCHEMA = + new Schema( + ImmutableList.of( + required(1, "id", Types.LongType.get()), 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 earliestStreamingRestrictionEmitsSnapshotsInSequenceOrder() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + commitAppend(table, "s1.parquet", records("one", 1L)); + commitAppend(table, "s2.parquet", records("two", 2L)); + commitAppend(table, "s3.parquet", records("three", 3L)); + List snapshots = Lists.newArrayList(table.snapshots()); + + WatchForSnapshotsSdf sdf = + new WatchForSnapshotsSdf( + scanConfigBuilder(table, tableId) + .setStreaming(true) + .setStartingStrategy(StartingStrategy.EARLIEST) + .setPollInterval(Duration.millis(1L)) + .build()); + + OffsetRange restriction = sdf.initialRestriction(); + assertEquals(1L, restriction.getFrom()); + assertEquals(Long.MAX_VALUE, restriction.getTo()); + + CapturingOutputReceiver out = new CapturingOutputReceiver(); + ManualWatermarkEstimator watermark = + sdf.newWatermarkEstimator(sdf.initialWatermarkState()); + DoFn.ProcessContinuation continuation = + sdf.process(sdf.newTracker(restriction), watermark, out); + + Long[] expectedSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).toArray(Long[]::new); + List actualSnapshotIds = + out.values.stream().map(TimestampedValue::getValue).collect(Collectors.toList()); + assertTrue(continuation.shouldResume()); + assertEquals(Duration.millis(1L), continuation.resumeDelay()); + assertThat(actualSnapshotIds, contains(expectedSnapshotIds)); + assertEquals( + Instant.ofEpochMilli(snapshots.get(snapshots.size() - 1).timestampMillis()), + watermark.currentWatermark()); + } + + @Test + public void boundedSnapshotRangeUsesInclusiveLowerAndExclusiveUpperSequence() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + commitAppend(table, "s1.parquet", records("one", 1L)); + commitAppend(table, "s2.parquet", records("two", 2L)); + commitAppend(table, "s3.parquet", records("three", 3L)); + commitAppend(table, "s4.parquet", records("four", 4L)); + List snapshots = Lists.newArrayList(table.snapshots()); + Snapshot second = snapshots.get(1); + Snapshot third = snapshots.get(2); + + WatchForSnapshotsSdf sdf = + new WatchForSnapshotsSdf( + scanConfigBuilder(table, tableId) + .setStreaming(true) + .setFromSnapshotInclusive(second.snapshotId()) + .setToSnapshot(third.snapshotId()) // this is inclusive + .setPollInterval(Duration.standardSeconds(30L)) + .build()); + + OffsetRange restriction = sdf.initialRestriction(); + assertEquals(second.sequenceNumber(), restriction.getFrom()); + assertEquals(third.sequenceNumber() + 1, restriction.getTo()); + + CapturingOutputReceiver out = new CapturingOutputReceiver(); + DoFn.ProcessContinuation continuation = + sdf.process( + sdf.newTracker(restriction), + sdf.newWatermarkEstimator(sdf.initialWatermarkState()), + out); + + List outputSnapshotIds = + out.values.stream().map(TimestampedValue::getValue).collect(Collectors.toList()); + + assertFalse(continuation.shouldResume()); + assertThat(outputSnapshotIds, contains(second.snapshotId(), third.snapshotId())); + assertEquals(2, out.values.size()); + assertEquals(Instant.ofEpochMilli(second.timestampMillis()), out.values.get(0).getTimestamp()); + assertEquals(Instant.ofEpochMilli(third.timestampMillis()), out.values.get(1).getTimestamp()); + } + + @Test + public void streamingDefaultStartsAtLatestSnapshotAndEarliestStartsAtFirst() throws Exception { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + commitAppend(table, "s1.parquet", records("one", 1L)); + commitAppend(table, "s2.parquet", records("two", 2L)); + Snapshot latest = table.currentSnapshot(); + + WatchForSnapshotsSdf defaultSdf = + new WatchForSnapshotsSdf( + scanConfigBuilder(table, tableId) + .setStreaming(true) + .setPollInterval(Duration.standardSeconds(1L)) + .build()); + WatchForSnapshotsSdf earliestSdf = + new WatchForSnapshotsSdf( + scanConfigBuilder(table, tableId) + .setStreaming(true) + .setStartingStrategy(StartingStrategy.EARLIEST) + .setPollInterval(Duration.standardSeconds(1L)) + .build()); + + assertEquals(latest.sequenceNumber(), defaultSdf.initialRestriction().getFrom()); + assertEquals(1L, earliestSdf.initialRestriction().getFrom()); + } + + @Test + public void emptyTableReturnsResumeAndAdvancesIdleWatermark() { + TableIdentifier tableId = tableId(); + Table table = warehouse.createTable(tableId, CDC_SCHEMA, null, tableProperties()); + WatchForSnapshotsSdf sdf = + new WatchForSnapshotsSdf( + scanConfigBuilder(table, tableId) + .setStreaming(true) + .setMaxSnapshotDiscoveryDelay(Duration.ZERO) + .setPollInterval(Duration.millis(25L)) + .build()); + ManualWatermarkEstimator watermark = + sdf.newWatermarkEstimator(sdf.initialWatermarkState()); + Instant beforeProcess = Instant.now(); + CapturingOutputReceiver out = new CapturingOutputReceiver(); + + DoFn.ProcessContinuation continuation = + sdf.process(sdf.newTracker(sdf.initialRestriction()), watermark, out); + + assertTrue(continuation.shouldResume()); + assertEquals(Duration.millis(25L), continuation.resumeDelay()); + assertThat(out.values, empty()); + assertThat(watermark.currentWatermark(), greaterThan(beforeProcess.minus(Duration.millis(1L)))); + assertThat(watermark.currentWatermark(), lessThanOrEqualTo(Instant.now())); + } + + private TableIdentifier tableId() { + return TableIdentifier.of("default", testName.getMethodName()); + } + + private IcebergScanConfig.Builder scanConfigBuilder(Table table, TableIdentifier tableId) { + return IcebergScanConfig.builder() + .setCatalogConfig( + IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties( + ImmutableMap.of("type", "hadoop", "warehouse", warehouse.location)) + .build()) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) + .setUseCdc(true); + } + + private static Map tableProperties() { + return ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + } + + private void commitAppend(Table table, String fileName, List records) throws IOException { + DataFile file = + warehouse.writeRecords(testName.getMethodName() + "-" + fileName, table.schema(), records); + table.newFastAppend().appendFile(file).commit(); + table.refresh(); + } + + private static List records(String data, long... ids) { + ImmutableList.Builder records = ImmutableList.builder(); + for (long id : ids) { + records.add(TestFixtures.createRecord(CDC_SCHEMA, ImmutableMap.of("id", id, "data", data))); + } + return records.build(); + } + + private static final class CapturingOutputReceiver implements DoFn.OutputReceiver { + private final ImmutableList.Builder> builder = ImmutableList.builder(); + private List> values = ImmutableList.of(); + + @Override + public OutputBuilder builder(Long value) { + throw new UnsupportedOperationException("Use outputWithTimestamp in this test receiver."); + } + + @Override + public void outputWithTimestamp(Long value, Instant timestamp) { + builder.add(TimestampedValue.of(value, timestamp)); + values = builder.build(); + } + } +} From 37ac60af4d6c368db14fa10ca691ed7d4e56602f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 1 Jun 2026 17:25:42 -0400 Subject: [PATCH 6/9] test name --- .../beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 b83d1b5eef07..f8f253868d2f 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 @@ -578,7 +578,7 @@ public void testStreamingReadWithColumnPruning_drop() throws Exception { } @Test - public void testBatchCdcReadMixedDeleteAndOverwriteSnapshots() throws Exception { + public void testStreamingCdcReadMixedDeleteAndOverwriteSnapshots() throws Exception { Table table = createCdcTable(); DataFile firstFile = commitCdcAppend( @@ -636,7 +636,7 @@ public void testBatchCdcReadMixedDeleteAndOverwriteSnapshots() throws Exception PCollection changes = rows.apply("Format CDC Changes", ParDo.of(new FormatCdcChange())); - assertThat(rows.isBounded(), equalTo(BOUNDED)); + assertThat(rows.isBounded(), equalTo(UNBOUNDED)); assertEquals(CDC_BEAM_SCHEMA, rows.getSchema()); PAssert.that(changes) .containsInAnyOrder( From 9ccc80491b61e43b5e46e46c69cf05c19c1b5b4c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 2 Jun 2026 10:09:40 -0400 Subject: [PATCH 7/9] use PK columns for CDC ITs --- .../iceberg/catalog/IcebergCatalogBaseIT.java | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) 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 f8f253868d2f..ccc211b0fab8 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 @@ -79,6 +79,7 @@ import org.apache.beam.sdk.values.ValueKind; 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.ImmutableSet; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; @@ -532,7 +533,7 @@ public void testReadWithNestedFieldFilter() throws Exception { @Test public void testStreamingReadWithFilter() throws Exception { - Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + Table table = catalog.createTable(TableIdentifier.parse(tableId()), CDC_ICEBERG_SCHEMA); List expectedRows = populateTable(table).stream() @@ -557,7 +558,7 @@ public void testStreamingReadWithFilter() throws Exception { @Test public void testStreamingReadWithColumnPruning_drop() throws Exception { - Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + Table table = catalog.createTable(TableIdentifier.parse(tableId()), CDC_ICEBERG_SCHEMA); List expectedRows = populateTable(table); @@ -713,7 +714,9 @@ public void testWriteReadWithFilter() throws IOException { @Test public void testReadWriteStreaming() throws IOException { - Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + org.apache.iceberg.Schema schemaWithPk = + new org.apache.iceberg.Schema(ICEBERG_SCHEMA.columns(), ImmutableSet.of(1)); + Table table = catalog.createTable(TableIdentifier.parse(tableId()), schemaWithPk); List expectedRows = populateTable(table); Map config = managedIcebergConfig(tableId()); @@ -1127,7 +1130,9 @@ && checkStateNotNull(rec.getBoolean("bool_field")) == bool) } public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throws Exception { - Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + org.apache.iceberg.Schema schemaWithPk = + new org.apache.iceberg.Schema(ICEBERG_SCHEMA.columns(), ImmutableSet.of(1)); + Table table = catalog.createTable(TableIdentifier.parse(tableId()), schemaWithPk); populateTable(table, "a"); // first snapshot Thread.sleep(AFTER_UPDATE_SLEEP_MS); @@ -1174,13 +1179,6 @@ private Table createCdcTable() { "full")); } - private Map cdcReadConfig(long fromSnapshotId, long toSnapshotId) { - Map config = new HashMap<>(managedIcebergConfig(tableId())); - config.put("from_snapshot", fromSnapshotId); - config.put("to_snapshot", toSnapshotId); - return config; - } - private static String cdcChange(ValueKind valueKind, Snapshot snapshot, long id, String data) { return String.format("%s:%d:%d:%s", valueKind, snapshot.timestampMillis(), id, data); } From 4b2440464221fcb076f274e8a2b15a5d1098c62d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 2 Jun 2026 11:42:02 -0400 Subject: [PATCH 8/9] import changes from iceberg#14264 --- .../sdk/io/iceberg/cdc/ChangelogScanner.java | 4 +- .../iceberg/BaseIncrementalChangelogScan.java | 993 ++++++++++++++++++ .../java/org/apache/iceberg/package-info.java | 20 + 3 files changed, 1015 insertions(+), 2 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/iceberg/package-info.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java index 9dba01fa7e6e..b84939dff3e8 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java @@ -53,6 +53,7 @@ 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.Iterables; import org.apache.iceberg.AddedRowsScanTask; +import org.apache.iceberg.BaseIncrementalChangelogScan; import org.apache.iceberg.ChangelogScanTask; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataOperations; @@ -233,8 +234,7 @@ public void process(@Element Long snapshotId, MultiOutputReceiver out) throws IO @Nullable Expression filter = scanConfig.getFilter(); IncrementalChangelogScan scan = - table - .newIncrementalChangelogScan() + new BaseIncrementalChangelogScan(table) .toSnapshot(snapshotId) .project(scanConfig.getProjectedSchema()); if (fromSnapshotId != null) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java b/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java new file mode 100644 index 000000000000..74a77cf47766 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java @@ -0,0 +1,993 @@ +/* + * 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.iceberg; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Comparator; +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.function.Supplier; +import java.util.stream.Collectors; +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.FluentIterable; +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.Sets; +import org.apache.iceberg.ManifestGroup.CreateTasksFunction; +import org.apache.iceberg.ManifestGroup.TaskContext; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ManifestEvaluator; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PartitionSet; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.SortedMerge; +import org.apache.iceberg.util.TableScanUtil; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("nullness") +public class BaseIncrementalChangelogScan + extends BaseIncrementalScan< + IncrementalChangelogScan, ChangelogScanTask, ScanTaskGroup> + implements IncrementalChangelogScan { + private static final DeleteFileIndex EMPTY = DeleteFileIndex.builderFor(null).build(); + private static final Logger LOG = LoggerFactory.getLogger(BaseIncrementalChangelogScan.class); + + public BaseIncrementalChangelogScan(Table table) { + this(table, table.schema(), TableScanContext.empty()); + } + + private BaseIncrementalChangelogScan(Table table, Schema schema, TableScanContext context) { + super(table, schema, context); + } + + @Override + protected IncrementalChangelogScan newRefinedScan( + Table newTable, Schema newSchema, TableScanContext newContext) { + return new BaseIncrementalChangelogScan(newTable, newSchema, newContext); + } + + // Private fields to track build call count and cache (accessed via package-private methods for + // testing) + private int existingDeleteIndexBuildCallCount = 0; + // Cache for the built index (null if not built yet) + private DeleteFileIndex cachedExistingDeleteIndex = null; + + @Override + protected CloseableIterable doPlanFiles( + Long fromSnapshotIdExclusive, long toSnapshotIdInclusive) { + + Deque changelogSnapshots = + orderedChangelogSnapshots(fromSnapshotIdExclusive, toSnapshotIdInclusive); + + if (changelogSnapshots.isEmpty()) { + return CloseableIterable.empty(); + } + + Set changelogSnapshotIds = toSnapshotIds(changelogSnapshots); + + Set newDataManifests = + FluentIterable.from(changelogSnapshots) + .transformAndConcat(snapshot -> snapshot.dataManifests(table().io())) + .filter(manifest -> changelogSnapshotIds.contains(manifest.snapshotId())) + .toSet(); + + // Build per-snapshot delete file indexes for added deletes + Map addedDeletesBySnapshot = buildAddedDeleteIndexes(changelogSnapshots); + + // Check if existing delete index is needed for equality deletes + boolean hasEqualityDeletes = + addedDeletesBySnapshot.values().stream() + .anyMatch(index -> !index.isEmpty() && index.hasEqualityDeletes()); + + // Build existing index early if needed for equality deletes, otherwise use lazy initialization + DeleteFileIndex existingDeleteIndex = + hasEqualityDeletes ? buildExistingDeleteIndexTracked(fromSnapshotIdExclusive) : EMPTY; + + ManifestGroup manifestGroup = + new ManifestGroup(table().io(), newDataManifests, ImmutableList.of()) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .select(scanColumns()) + .filterData(filter()) + .filterManifestEntries(entry -> changelogSnapshotIds.contains(entry.snapshotId())) + .ignoreExisting() + .columnsToKeepStats(columnsToKeepStats()); + + if (shouldIgnoreResiduals()) { + manifestGroup = manifestGroup.ignoreResiduals(); + } + + if (newDataManifests.size() > 1 && shouldPlanWithExecutor()) { + manifestGroup = manifestGroup.planWith(planExecutor()); + } + + // Create a supplier that reuses already-built index or builds lazily when first DELETED entry + // is encountered + Supplier existingDeleteIndexSupplier = + () -> { + if (cachedExistingDeleteIndex != null) { + return cachedExistingDeleteIndex; + } + return buildExistingDeleteIndexTracked(fromSnapshotIdExclusive); + }; + + // Plan data file tasks (ADDED and DELETED) + Map> cumulativeDeletesMap = + buildCumulativeDeletesBySnapshot(changelogSnapshots, addedDeletesBySnapshot); + + CloseableIterable dataFileTasks = + manifestGroup.plan( + new CreateDataFileChangeTasks( + changelogSnapshots, + existingDeleteIndexSupplier, + addedDeletesBySnapshot, + cumulativeDeletesMap, + table().specs(), + isCaseSensitive())); + + // Find EXISTING data files affected by newly added delete files and create tasks for them + CloseableIterable deletedRowsTasks = + planDeletedRowsTasks( + changelogSnapshots, existingDeleteIndex, addedDeletesBySnapshot, changelogSnapshotIds); + + // Merge tasks from both iterables in order by changeOrdinal + Comparator byOrdinal = + Comparator.comparing(ChangelogScanTask::changeOrdinal) + .thenComparing(ChangelogScanTask::commitSnapshotId); + + return new SortedMerge<>(byOrdinal, ImmutableList.of(dataFileTasks, deletedRowsTasks)); + } + + @Override + public CloseableIterable> planTasks() { + return TableScanUtil.planTaskGroups( + planFiles(), targetSplitSize(), splitLookback(), splitOpenFileCost()); + } + + // builds a collection of changelog snapshots (oldest to newest) + // the order of the snapshots is important as it is used to determine change ordinals + private Deque orderedChangelogSnapshots(Long fromIdExcl, long toIdIncl) { + Deque changelogSnapshots = new ArrayDeque<>(); + + for (Snapshot snapshot : SnapshotUtil.ancestorsBetween(table(), toIdIncl, fromIdExcl)) { + if (!snapshot.operation().equals(DataOperations.REPLACE)) { + changelogSnapshots.addFirst(snapshot); + } + } + + return changelogSnapshots; + } + + private Set toSnapshotIds(Collection snapshots) { + return snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + } + + private static Map computeSnapshotOrdinals(Deque snapshots) { + Map snapshotOrdinals = Maps.newHashMap(); + + int ordinal = 0; + + for (Snapshot snapshot : snapshots) { + snapshotOrdinals.put(snapshot.snapshotId(), ordinal++); + } + + return snapshotOrdinals; + } + + /** + * Builds a delete file index for existing deletes that were present before the start snapshot. + * These deletes should be applied to data files but should not generate DELETE changelog rows. + * Uses manifest pruning and caching to optimize performance. + */ + private DeleteFileIndex buildExistingDeleteIndex(Long fromSnapshotIdExclusive) { + if (fromSnapshotIdExclusive == null) { + return EMPTY; + } + Snapshot fromSnapshot = table().snapshot(fromSnapshotIdExclusive); + Preconditions.checkState( + fromSnapshot != null, "Cannot find starting snapshot: %s", fromSnapshotIdExclusive); + + List existingDeleteManifests = fromSnapshot.deleteManifests(table().io()); + if (existingDeleteManifests.isEmpty()) { + return EMPTY; + } + + // Prune manifests based on partition filter to avoid processing irrelevant manifests + List prunedManifests = pruneManifestsByPartition(existingDeleteManifests); + if (prunedManifests.isEmpty()) { + return EMPTY; + } + + // Load delete files from manifests + Iterable deleteFiles = loadDeleteFiles(prunedManifests, null); + + return DeleteFileIndex.builderFor(deleteFiles) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .build(); + } + + /** + * Wrapper method that tracks build calls and caches the result for reuse. This ensures we only + * build the index once even if called from multiple places. + */ + private DeleteFileIndex buildExistingDeleteIndexTracked(Long fromSnapshotIdExclusive) { + if (cachedExistingDeleteIndex != null) { + return cachedExistingDeleteIndex; + } + existingDeleteIndexBuildCallCount++; + cachedExistingDeleteIndex = buildExistingDeleteIndex(fromSnapshotIdExclusive); + return cachedExistingDeleteIndex; + } + + // Visible for testing + int getExistingDeleteIndexBuildCallCount() { + return existingDeleteIndexBuildCallCount; + } + + // Visible for testing + boolean wasExistingDeleteIndexBuilt() { + return existingDeleteIndexBuildCallCount > 0; + } + + /** + * Builds per-snapshot delete file indexes for newly added delete files in each changelog + * snapshot. These deletes should generate DELETE changelog rows. Uses caching to avoid re-parsing + * manifests. + */ + private Map buildAddedDeleteIndexes(Deque changelogSnapshots) { + Map addedDeletesBySnapshot = Maps.newConcurrentMap(); + Tasks.foreach(changelogSnapshots) + .retry(3) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(planExecutor()) + .onFailure( + (snapshot, exc) -> + LOG.warn( + "Failed to build delete index for snapshot {}", snapshot.snapshotId(), exc)) + .run( + snapshot -> { + List snapshotDeleteManifests = snapshot.deleteManifests(table().io()); + if (snapshotDeleteManifests.isEmpty()) { + addedDeletesBySnapshot.put(snapshot.snapshotId(), EMPTY); + return; + } + + // Filter to only include delete files added in this snapshot + List addedDeleteManifests = + snapshotDeleteManifests.stream() + .filter(manifest -> manifest.snapshotId().equals(snapshot.snapshotId())) + .collect(Collectors.toUnmodifiableList()); + + if (addedDeleteManifests.isEmpty()) { + addedDeletesBySnapshot.put(snapshot.snapshotId(), EMPTY); + } else { + // Load delete files from manifests + Iterable deleteFiles = + loadDeleteFiles(addedDeleteManifests, snapshot.snapshotId()); + + DeleteFileIndex index = + DeleteFileIndex.builderFor(deleteFiles) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .build(); + addedDeletesBySnapshot.put(snapshot.snapshotId(), index); + } + }); + return addedDeletesBySnapshot; + } + + /** + * Plans tasks for EXISTING data files that are affected by newly added delete files. These files + * were not added or deleted in the changelog snapshot range, but have new delete files applied to + * them. + */ + private CloseableIterable planDeletedRowsTasks( + Deque changelogSnapshots, + DeleteFileIndex existingDeleteIndex, + Map addedDeletesBySnapshot, + Set changelogSnapshotIds) { + + Map snapshotOrdinals = computeSnapshotOrdinals(changelogSnapshots); + List tasks = Lists.newArrayList(); + + // Build a map of file statuses and collect affected partitions for each snapshot + Pair>, PartitionSet> fileStatusAndPartitions = + buildFileStatusBySnapshot(changelogSnapshots, changelogSnapshotIds); + Map> fileStatusBySnapshot = + fileStatusAndPartitions.first(); + PartitionSet affectedPartitions = fileStatusAndPartitions.second(); + + // Accumulate actual DeleteFile entries chronologically + List accumulatedDeletes = Lists.newArrayList(); + + // Start with deletes from before the changelog range + if (!existingDeleteIndex.isEmpty()) { + for (DeleteFile df : existingDeleteIndex.referencedDeleteFiles()) { + accumulatedDeletes.add(df); + } + } + + for (Snapshot snapshot : changelogSnapshots) { + DeleteFileIndex addedDeleteIndex = addedDeletesBySnapshot.get(snapshot.snapshotId()); + if (addedDeleteIndex.isEmpty()) { + continue; + } + + // Collect partitions of newly added delete files for pruning (important for the current + // snapshot) + for (DeleteFile df : addedDeleteIndex.referencedDeleteFiles()) { + affectedPartitions.add(df.specId(), df.partition()); + } + + DeleteFileIndex cumulativeDeleteIndex = + buildDeleteIndex(accumulatedDeletes, affectedPartitions); + + // Process data files for this snapshot + // Use a local set per snapshot to track processed files + Set alreadyProcessedPaths = Sets.newHashSet(); + processSnapshotForDeletedRowsTasks( + snapshot, + addedDeleteIndex, + cumulativeDeleteIndex, + fileStatusBySnapshot.get(snapshot.snapshotId()), + alreadyProcessedPaths, + snapshotOrdinals, + affectedPartitions, + tasks); + + // Accumulate this snapshot's added deletes for subsequent snapshots + for (DeleteFile df : addedDeleteIndex.referencedDeleteFiles()) { + accumulatedDeletes.add(df); + } + } + + return CloseableIterable.withNoopClose(tasks); + } + + /** + * Builds a map of file statuses for each snapshot, tracking which files were added or deleted in + * each snapshot. + */ + private Pair>, PartitionSet> + buildFileStatusBySnapshot( + Deque changelogSnapshots, Set changelogSnapshotIds) { + + Map> fileStatusBySnapshot = Maps.newConcurrentMap(); + java.util.Queue localPartitionsQueue = + new java.util.concurrent.ConcurrentLinkedQueue<>(); + + Tasks.foreach(changelogSnapshots) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(planExecutor()) + .run( + snapshot -> { + Map fileStatuses = Maps.newHashMap(); + PartitionSet localAffected = PartitionSet.create(table().specs()); + + List changedDataManifests = + FluentIterable.from(snapshot.dataManifests(table().io())) + .filter(manifest -> manifest.snapshotId().equals(snapshot.snapshotId())) + .toList(); + + if (!changedDataManifests.isEmpty()) { + ManifestGroup changedGroup = + new ManifestGroup(table().io(), changedDataManifests, ImmutableList.of()) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .select(scanColumns()) + .filterData(filter()) + .ignoreExisting() + .columnsToKeepStats(columnsToKeepStats()); + + try (CloseableIterable> entries = changedGroup.entries()) { + for (ManifestEntry entry : entries) { + if (changelogSnapshotIds.contains(entry.snapshotId())) { + fileStatuses.put(entry.file().location(), entry.status()); + localAffected.add(entry.file().specId(), entry.file().partition()); + } + } + } catch (Exception e) { + throw new RuntimeException( + "Failed to collect file statuses for snapshot " + snapshot.snapshotId(), e); + } + } + + fileStatusBySnapshot.put(snapshot.snapshotId(), fileStatuses); + localPartitionsQueue.add(localAffected); + }); + + PartitionSet globalAffected = PartitionSet.create(table().specs()); + for (PartitionSet local : localPartitionsQueue) { + globalAffected.addAll(local); + } + + return Pair.of(fileStatusBySnapshot, globalAffected); + } + + private List pruneManifestsByAffectedPartitions( + List manifests, PartitionSet affectedPartitions) { + if (affectedPartitions.isEmpty()) { + return manifests; + } + + Expression affectedExpr = buildAffectedPartitionExpression(affectedPartitions); + if (affectedExpr == Expressions.alwaysFalse()) { + return manifests; + } + + List pruned = Lists.newArrayList(); + for (ManifestFile manifest : manifests) { + PartitionSpec spec = table().specs().get(manifest.partitionSpecId()); + if (spec == null || spec.isUnpartitioned()) { + pruned.add(manifest); + } else if (manifestOverlapsFilter(manifest, spec, affectedExpr)) { + pruned.add(manifest); + } + } + return pruned; + } + + private Expression buildAffectedPartitionExpression(PartitionSet affectedPartitions) { + Expression combined = null; + + for (Pair pair : affectedPartitions) { + int specId = pair.first(); + StructLike partition = pair.second(); + PartitionSpec spec = table().specs().get(specId); + if (spec == null) { + continue; + } else if (spec.isUnpartitioned()) { + return Expressions.alwaysTrue(); // FALLBACK: Global delete exists, include ALL manifests! + } + + Expression specExpr = null; + for (int i = 0; i < spec.fields().size(); i++) { + org.apache.iceberg.PartitionField field = spec.fields().get(i); + Object value = partition.get(i, Object.class); + if (value != null) { + String columnName = table().schema().findColumnName(field.sourceId()); + if (columnName != null) { + Expression equalExpr = Expressions.equal(columnName, value); + specExpr = (specExpr == null) ? equalExpr : Expressions.and(specExpr, equalExpr); + } + } + } + + if (specExpr != null) { + combined = (combined == null) ? specExpr : Expressions.or(combined, specExpr); + } + } + + return combined != null ? combined : Expressions.alwaysFalse(); + } + + /** + * Builds a map of snapshot ID -> all delete files that were added in the scan range up to that + * snapshot, PRUNING files that were removed in the middle. + */ + private Map> buildCumulativeDeletesBySnapshot( + Deque snapshots, Map addedDeletesBySnapshot) { + Map> result = Maps.newHashMap(); + List accumulatedDeletes = Lists.newArrayList(); + + for (Snapshot snapshot : snapshots) { + // Save state first, so that this snapshot's tasks can use any deletes active up to this point + result.put(snapshot.snapshotId(), Lists.newArrayList(accumulatedDeletes)); + + // Check for removed deletes and prune from accumulatedDeletes for FUTURE snapshots + List changedDeletes = + FluentIterable.from(snapshot.deleteManifests(table().io())) + .filter(manifest -> manifest.snapshotId().equals(snapshot.snapshotId())) + .toList(); + + if (!changedDeletes.isEmpty()) { + Iterable removedDeletes = + loadRemovedDeleteFiles(changedDeletes, snapshot.snapshotId()); + Set removedPaths = Sets.newHashSet(); + for (DeleteFile rdf : removedDeletes) { + removedPaths.add(rdf.location()); + } + accumulatedDeletes.removeIf(df -> removedPaths.contains(df.location())); + } + + // Add new deletes for FUTURE snapshots + DeleteFileIndex addedDeleteIndex = addedDeletesBySnapshot.get(snapshot.snapshotId()); + if (addedDeleteIndex != null && !addedDeleteIndex.isEmpty()) { + for (DeleteFile df : addedDeleteIndex.referencedDeleteFiles()) { + accumulatedDeletes.add(df); + } + } + } + + return result; + } + + /** + * Builds a delete index from the accumulated list of delete files, pruning by affected + * partitions. + */ + private DeleteFileIndex buildDeleteIndex( + List accumulatedDeletes, PartitionSet affectedPartitions) { + if (accumulatedDeletes.isEmpty()) { + return EMPTY; + } + + List filteredDeletes = accumulatedDeletes; + if (!affectedPartitions.isEmpty()) { + filteredDeletes = Lists.newArrayList(); + for (DeleteFile df : accumulatedDeletes) { + PartitionSpec spec = table().specs().get(df.specId()); + if (spec == null || spec.isUnpartitioned()) { + filteredDeletes.add(df); // Always include unpartitioned deletes + } else if (affectedPartitions.contains(df.specId(), df.partition())) { + filteredDeletes.add(df); + } + } + } + + return DeleteFileIndex.builderFor(filteredDeletes) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .build(); + } + + /** + * Processes data files for a snapshot to create DeletedRowsScanTask for existing files affected + * by new delete files. + */ + private void processSnapshotForDeletedRowsTasks( + Snapshot snapshot, + DeleteFileIndex addedDeleteIndex, + DeleteFileIndex cumulativeDeleteIndex, + Map currentSnapshotFiles, + Set alreadyProcessedPaths, + Map snapshotOrdinals, + PartitionSet affectedPartitions, + List tasks) { + + // Get all data files that exist in this snapshot, pruned by affected partitions + List allDataManifests = snapshot.dataManifests(table().io()); + List prunedManifests = + pruneManifestsByAffectedPartitions(allDataManifests, affectedPartitions); + + ManifestGroup allDataGroup = + new ManifestGroup(table().io(), prunedManifests, ImmutableList.of()) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .select(scanColumns()) + .filterData(filter()) + .ignoreDeleted() + .columnsToKeepStats(columnsToKeepStats()); + + if (shouldIgnoreResiduals()) { + allDataGroup = allDataGroup.ignoreResiduals(); + } + + String schemaString = SchemaParser.toJson(schema()); + + // Cache per specId - same for all files with same specId + Map specStringCache = Maps.newHashMap(); + Map residualCache = Maps.newHashMap(); + Expression residualFilter = shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter(); + + try (CloseableIterable> entries = allDataGroup.entries()) { + for (ManifestEntry entry : entries) { + DataFile dataFile = entry.file(); + String filePath = dataFile.location(); + + // Skip if this file was ADDED or DELETED in this snapshot + // (those are handled by CreateDataFileChangeTasks) + if (currentSnapshotFiles.containsKey(filePath)) { + continue; + } + + // Skip if we already created a task for this file in this snapshot + // Note: alreadyProcessedPaths is local to this snapshot's processing + if (alreadyProcessedPaths.contains(filePath)) { + continue; + } + + // Check if this data file is affected by newly added delete files + DeleteFile[] addedDeletes = addedDeleteIndex.forEntry(entry); + if (addedDeletes.length == 0) { + continue; + } + + // This data file was EXISTING but has new delete files applied + // Get existing deletes from before this snapshot (cumulative) + DeleteFile[] existingDeletes = + cumulativeDeleteIndex.isEmpty() + ? new DeleteFile[0] + : cumulativeDeleteIndex.forEntry(entry); + + // Create a DeletedRowsScanTask + int changeOrdinal = snapshotOrdinals.get(snapshot.snapshotId()); + + // Use cached values (calculate once per specId) + int specId = dataFile.specId(); + String specString = + specStringCache.computeIfAbsent( + specId, id -> PartitionSpecParser.toJson(table().specs().get(id))); + ResidualEvaluator residuals = + residualCache.computeIfAbsent( + specId, + id -> { + PartitionSpec spec = table().specs().get(id); + return ResidualEvaluator.of(spec, residualFilter, isCaseSensitive()); + }); + + tasks.add( + new BaseDeletedRowsScanTask( + changeOrdinal, + snapshot.snapshotId(), + dataFile.copy(shouldKeepStats()), + addedDeletes, + existingDeletes, + schemaString, + specString, + residuals)); + + // Mark this file as processed for this snapshot + alreadyProcessedPaths.add(filePath); + } + } catch (Exception e) { + throw new RuntimeException("Failed to plan deleted rows tasks", e); + } + } + + private boolean shouldKeepStats() { + Set columns = columnsToKeepStats(); + return columns != null && !columns.isEmpty(); + } + + /** + * Loads delete files from manifests by parsing each manifest. + * + * @param manifests the delete manifests to load + * @return list of delete files + */ + private Iterable loadDeleteFiles( + List manifests, Long targetSnapshotId) { + Queue allDeleteFiles = new ConcurrentLinkedQueue<>(); + + Tasks.foreach(manifests) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(planExecutor()) + .run( + manifest -> { + List deleteFiles = + loadDeleteFilesFromManifest(manifest, targetSnapshotId); + allDeleteFiles.addAll(deleteFiles); + }); + + return allDeleteFiles; + } + + private Iterable loadRemovedDeleteFiles( + List manifests, Long targetSnapshotId) { + Queue allDeleteFiles = new ConcurrentLinkedQueue<>(); + + Tasks.foreach(manifests) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(planExecutor()) + .run( + manifest -> { + List deleteFiles = + loadRemovedDeleteFilesFromManifest(manifest, targetSnapshotId); + allDeleteFiles.addAll(deleteFiles); + }); + + return allDeleteFiles; + } + + private List loadRemovedDeleteFilesFromManifest( + ManifestFile manifest, Long targetSnapshotId) { + List deleteFiles = Lists.newArrayList(); + + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(manifest, table().io(), table().specs())) { + for (ManifestEntry entry : reader.entries()) { + if (entry.status() == ManifestEntry.Status.DELETED + && entry.snapshotId().equals(targetSnapshotId)) { + DeleteFile file = entry.file(); + + if (!partitionMatchesFilter(file)) { + continue; + } + + Set columns = + file.content() == FileContent.POSITION_DELETES + ? Set.of(MetadataColumns.DELETE_FILE_PATH.fieldId()) + : Set.copyOf(file.equalityFieldIds()); + deleteFiles.add(ContentFileUtil.copy(file, true, columns)); + } + } + } catch (Exception e) { + throw new RuntimeException("Failed to read delete manifest: " + manifest.path(), e); + } + + return deleteFiles; + } + + /** + * Prunes delete manifests based on partition filter to avoid processing irrelevant manifests. + * This significantly improves performance when only a subset of partitions are relevant to the + * scan. + * + * @param manifests all delete manifests to consider + * @return list of manifests that might contain relevant delete files + */ + private List pruneManifestsByPartition(List manifests) { + Expression currentFilter = filter(); + + // If there's no filter, return all manifests + if (currentFilter == null || currentFilter.equals(Expressions.alwaysTrue())) { + return manifests; + } + + List prunedManifests = Lists.newArrayList(); + + for (ManifestFile manifest : manifests) { + PartitionSpec spec = table().specs().get(manifest.partitionSpecId()); + if (spec == null || spec.isUnpartitioned()) { + // Include unpartitioned manifests + prunedManifests.add(manifest); + } else if (manifestOverlapsFilter(manifest, spec, currentFilter)) { + // Check if manifest partition range overlaps with filter + prunedManifests.add(manifest); + } + } + + return prunedManifests; + } + + /** + * Checks if a manifest's partition range overlaps with the given filter. + * + * @param manifest the manifest to check + * @param spec the partition spec for the manifest + * @param filter the scan filter + * @return true if the manifest might contain matching partitions, false otherwise + */ + private boolean manifestOverlapsFilter( + ManifestFile manifest, PartitionSpec spec, Expression filter) { + try { + // Use inclusive projection to transform row filter to partition filter + Expression partitionFilter = Projections.inclusive(spec, isCaseSensitive()).project(filter); + + // Create evaluator for the partition filter + ManifestEvaluator evaluator = + ManifestEvaluator.forPartitionFilter(partitionFilter, spec, isCaseSensitive()); + + // Check if manifest could contain matching partitions + return evaluator.eval(manifest); + } catch (Exception e) { + // If evaluation fails, be conservative and include the manifest + return true; + } + } + + /** + * Checks if a delete file's partition overlaps with the current scan filter. This enables + * partition pruning to reduce memory footprint and planning overhead by skipping delete files + * that cannot possibly match any rows in the scan. + * + * @param file the delete file to check + * @return true if the delete file's partition might contain matching rows, false otherwise + */ + private boolean partitionMatchesFilter(DeleteFile file) { + // If there's no filter, all partitions match + Expression currentFilter = filter(); + if (currentFilter == null || currentFilter.equals(Expressions.alwaysTrue())) { + return true; + } + + // Get the partition spec for this delete file + PartitionSpec spec = table().specs().get(file.specId()); + if (spec == null || spec.isUnpartitioned()) { + // If spec not found or table is unpartitioned, be conservative and include the file + return true; + } + + try { + // Project the row filter to partition space using inclusive projection + // This transforms expressions on source columns to expressions on partition columns + Expression partitionFilter = + Projections.inclusive(spec, isCaseSensitive()).project(currentFilter); + + // Evaluate the projected filter against the delete file's partition + Evaluator evaluator = new Evaluator(spec.partitionType(), partitionFilter, isCaseSensitive()); + return evaluator.eval(file.partition()); + } catch (Exception e) { + // If evaluation fails, be conservative and include the file + return true; + } + } + + /** + * Loads delete files from a single manifest, parsing the manifest entries. + * + * @param manifest the delete manifest to load + * @return list of delete files from this manifest + */ + private List loadDeleteFilesFromManifest( + ManifestFile manifest, Long targetSnapshotId) { + List deleteFiles = Lists.newArrayList(); + + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(manifest, table().io(), table().specs())) { + for (ManifestEntry entry : reader.entries()) { + if (entry.status() != ManifestEntry.Status.DELETED + && (targetSnapshotId == null || entry.snapshotId().equals(targetSnapshotId))) { + // Only include live delete files, copy with minimal stats to save memory + DeleteFile file = entry.file(); + + // Apply partition pruning - skip delete files that cannot match the scan filter + if (!partitionMatchesFilter(file)) { + continue; + } + + Set columns = + file.content() == FileContent.POSITION_DELETES + ? Set.of(MetadataColumns.DELETE_FILE_PATH.fieldId()) + : Set.copyOf(file.equalityFieldIds()); + deleteFiles.add(ContentFileUtil.copy(file, true, columns)); + } + } + } catch (Exception e) { + throw new RuntimeException("Failed to read delete manifest: " + manifest.path(), e); + } + + return deleteFiles; + } + + private static class CreateDataFileChangeTasks implements CreateTasksFunction { + private static final DeleteFile[] NO_DELETES = new DeleteFile[0]; + + private final Map snapshotOrdinals; + private final Supplier existingDeleteIndexSupplier; + private final Map addedDeletesBySnapshot; + private final Map> cumulativeDeletesMap; + private final Map specsById; + private final boolean caseSensitive; + + CreateDataFileChangeTasks( + Deque snapshots, + Supplier existingDeleteIndexSupplier, + Map addedDeletesBySnapshot, + Map> cumulativeDeletesMap, + Map specsById, + boolean caseSensitive) { + this.snapshotOrdinals = computeSnapshotOrdinals(snapshots); + this.existingDeleteIndexSupplier = existingDeleteIndexSupplier; + this.addedDeletesBySnapshot = addedDeletesBySnapshot; + this.cumulativeDeletesMap = cumulativeDeletesMap; + this.specsById = specsById; + this.caseSensitive = caseSensitive; + } + + @Override + public CloseableIterable apply( + CloseableIterable> entries, TaskContext context) { + + return CloseableIterable.transform( + entries, + entry -> { + long commitSnapshotId = entry.snapshotId(); + int changeOrdinal = snapshotOrdinals.get(commitSnapshotId); + DataFile dataFile = entry.file().copy(context.shouldKeepStats()); + + switch (entry.status()) { + case ADDED: + // For ADDED data files, attach delete files added in this snapshot + DeleteFile[] addedFileDeletes = getDeletesForAddedFile(entry, commitSnapshotId); + return new BaseAddedRowsScanTask( + changeOrdinal, + commitSnapshotId, + dataFile, + addedFileDeletes, + context.schemaAsString(), + context.specAsString(), + context.residuals()); + + case DELETED: + // For DELETED data files, attach ALL deletes that were present up to deletion + // This includes existing deletes AND deletes added in the scan range + DeleteFile[] deletedFileDeletes = getDeletesForDeletedFile(entry, commitSnapshotId); + return new BaseDeletedDataFileScanTask( + changeOrdinal, + commitSnapshotId, + dataFile, + deletedFileDeletes, + context.schemaAsString(), + context.specAsString(), + context.residuals()); + + default: + throw new IllegalArgumentException("Unexpected entry status: " + entry.status()); + } + }); + } + + /** + * Gets delete files that apply to an ADDED data file. Only includes deletes added in the same + * snapshot as the file. + */ + private DeleteFile[] getDeletesForAddedFile( + ManifestEntry entry, long commitSnapshotId) { + DeleteFileIndex addedDeleteIndex = addedDeletesBySnapshot.get(commitSnapshotId); + return addedDeleteIndex == null || addedDeleteIndex.isEmpty() + ? NO_DELETES + : addedDeleteIndex.forEntry(entry); + } + + /** + * Gets all delete files that were applied to a DELETED data file up to the point it was + * deleted. This includes existing deletes and all deletes added in the scan range up to (but + * not including) the deletion snapshot. + */ + private DeleteFile[] getDeletesForDeletedFile( + ManifestEntry entry, long deletionSnapshotId) { + + List allDeletes = Lists.newArrayList(); + + // Build existing delete index lazily when first DELETED entry is encountered + DeleteFileIndex existingDeleteIndex = existingDeleteIndexSupplier.get(); + DeleteFile[] existingDeletes = + existingDeleteIndex.isEmpty() ? NO_DELETES : existingDeleteIndex.forEntry(entry); + for (DeleteFile df : existingDeletes) { + allDeletes.add(df); + } + + // Add all deletes from snapshots in the scan range BEFORE the deletion + List cumulativeDeletes = cumulativeDeletesMap.get(deletionSnapshotId); + if (cumulativeDeletes != null && !cumulativeDeletes.isEmpty()) { + DeleteFileIndex tempIndex = + DeleteFileIndex.builderFor(cumulativeDeletes) + .specsById(specsById) + .caseSensitive(caseSensitive) + .build(); + DeleteFile[] applicable = tempIndex.forEntry(entry); + for (DeleteFile deleteFile : applicable) { + allDeletes.add(deleteFile); + } + } + + return allDeletes.isEmpty() ? NO_DELETES : allDeletes.toArray(new DeleteFile[0]); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/package-info.java b/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/package-info.java new file mode 100644 index 000000000000..a32b307b45d3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/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. + */ + +/** Classes copied from unreleased Iceberg core. */ +package org.apache.iceberg; From a1789bc9c0083bcf447f70d62ecc5f124d993cd7 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 2 Jun 2026 11:57:17 -0400 Subject: [PATCH 9/9] apply fixes --- .../sdk/io/iceberg/cdc/ChangelogScanner.java | 2 ++ .../iceberg/BaseIncrementalChangelogScan.java | 23 ++++++++++++++++++- 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java index b84939dff3e8..a45da3c30364 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java @@ -233,6 +233,8 @@ public void process(@Element Long snapshotId, MultiOutputReceiver out) throws IO @Nullable Long fromSnapshotId = snapshot.parentId(); @Nullable Expression filter = scanConfig.getFilter(); + // TODO(ahmedabu98): replace this with table.newIncrementalChangelogScan() when + // https://github.com/apache/iceberg/pull/14264/ gets merged and released. IncrementalChangelogScan scan = new BaseIncrementalChangelogScan(table) .toSnapshot(snapshotId) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java b/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java index 74a77cf47766..6b12e16690ba 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java @@ -45,6 +45,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PartitionMap; import org.apache.iceberg.util.PartitionSet; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.SortedMerge; @@ -53,12 +54,32 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Copied over from Iceberg PR #14264. + */ @SuppressWarnings("nullness") public class BaseIncrementalChangelogScan extends BaseIncrementalScan< IncrementalChangelogScan, ChangelogScanTask, ScanTaskGroup> implements IncrementalChangelogScan { - private static final DeleteFileIndex EMPTY = DeleteFileIndex.builderFor(null).build(); + private static final DeleteFileIndex EMPTY = createEmptyInstance(); + + private static DeleteFileIndex createEmptyInstance() { + try { + var constructor = + DeleteFileIndex.class.getDeclaredConstructor( + DeleteFileIndex.EqualityDeletes.class, + PartitionMap.class, + PartitionMap.class, + Map.class, + Map.class); + constructor.setAccessible(true); + return constructor.newInstance(null, null, null, null, null); + } catch (Exception e) { + throw new RuntimeException("Failed to initialize EMPTY DeleteFileIndex", e); + } + } + private static final Logger LOG = LoggerFactory.getLogger(BaseIncrementalChangelogScan.class); public BaseIncrementalChangelogScan(Table table) {