diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java index 4df3eecb18e5..d09e1a1cb3ae 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java @@ -42,11 +42,11 @@ * source creates a single range, while the unbounded implementation continuously polls for new * snapshots at the specified interval. */ -class IncrementalScanSource extends PTransform> { +public class IncrementalScanSource extends PTransform> { private static final Duration DEFAULT_POLL_INTERVAL = Duration.standardSeconds(60); - private final IcebergScanConfig scanConfig; + protected final IcebergScanConfig scanConfig; - IncrementalScanSource(IcebergScanConfig scanConfig) { + public IncrementalScanSource(IcebergScanConfig scanConfig) { this.scanConfig = scanConfig; } @@ -74,14 +74,15 @@ public PCollection expand(PBegin input) { } /** Continuously watches for new snapshots. */ - private PCollection>> unboundedSnapshots(PBegin input) { + protected PCollection>> unboundedSnapshots(PBegin input) { Duration pollInterval = MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); return input.apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, pollInterval)); } /** Creates a fixed snapshot range. */ - private PCollection>> boundedSnapshots(PBegin input, Table table) { + protected PCollection>> boundedSnapshots( + PBegin input, Table table) { checkStateNotNull( table.currentSnapshot().snapshotId(), "Table %s does not have any snapshots to read from.", 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 4b94663c64c5..8fbef14e3eb5 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 @@ -25,8 +25,15 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; 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.ContentFile; +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.types.Type; +import org.apache.iceberg.types.Types; import org.checkerframework.checker.nullness.qual.Nullable; class PartitionUtils { @@ -90,4 +97,51 @@ static PartitionSpec toPartitionSpec( return builder.build(); } + + /** + * Copied over from Apache Iceberg's PartitionUtil + */ + public static Map constantsMap( + PartitionSpec spec, ContentFile file, BiFunction convertConstant) { + 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.apply(Types.LongType.get(), file.firstRowId())); + } + + idToConstant.put( + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(), + convertConstant.apply(Types.LongType.get(), file.fileSequenceNumber())); + + // add _file + idToConstant.put( + MetadataColumns.FILE_PATH.fieldId(), + convertConstant.apply(Types.StringType.get(), file.location())); + + // add _spec_id + idToConstant.put( + MetadataColumns.SPEC_ID.fieldId(), + convertConstant.apply(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.apply( + 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 4b127fcdef22..e918902ea47d 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; import java.util.Collection; @@ -28,16 +29,22 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.sdk.io.iceberg.cdc.DeleteReader; +import org.apache.beam.sdk.io.iceberg.cdc.SerializableChangelogTask; 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.DeleteFile; import org.apache.iceberg.PartitionSpec; 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.DeleteFilter; import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; @@ -55,7 +62,6 @@ 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; @@ -72,16 +78,42 @@ 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( + SerializableChangelogTask task, Table table, IcebergScanConfig scanConfig) { + return createReader( + table, + scanConfig, + checkStateNotNull(table.specs().get(task.getSpecId())), + task.getDataFile().createDataFile(table.specs()), + task.getStart(), + task.getLength(), + task.getExpression(table.schema())); + } + + public static CloseableIterable createReader( + ContentScanTask task, Table table, IcebergScanConfig scanConfig) { + return createReader( + table, scanConfig, task.spec(), task.file(), task.start(), task.length(), task.residual()); + } + + public static CloseableIterable createReader( + Table table, + IcebergScanConfig scanConfig, + PartitionSpec spec, + ContentFile file, + long start, + long length, + Expression residual) { + Schema schema = scanConfig.getRequiredSchema(); InputFile inputFile; try (FileIO io = table.io()) { EncryptedInputFile encryptedInput = - EncryptedFiles.encryptedInput(io.newInputFile(filePath), task.file().keyMetadata()); + EncryptedFiles.encryptedInput(io.newInputFile(file.location()), file.keyMetadata()); inputFile = table.encryption().decrypt(encryptedInput); } Map idToConstants = - ReadUtils.constantsMap(task, IdentityPartitionConverters::convertConstant, table.schema()); + ReadUtils.constantsMap( + spec, file, IdentityPartitionConverters::convertConstant, table.schema()); ParquetReadOptions.Builder optionsBuilder; if (inputFile instanceof HadoopInputFile) { @@ -96,37 +128,40 @@ 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, + schema, + optionsBuilder.build(), + // TODO(ahmedabu98): Implement a Parquet-to-Beam Row reader, bypassing conversion to + // Iceberg + // Record + fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema, idToConstants), + mapping, + residual, + false, + true); + return maybeApplyFilter(records, scanConfig); } static Map constantsMap( - FileScanTask task, + PartitionSpec spec, + ContentFile file, 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); + return PartitionUtils.constantsMap(spec, file, converter); } else { return Collections.emptyMap(); } @@ -208,4 +243,138 @@ public static CloseableIterable maybeApplyFilter( } return iterable; } + + public static DeleteFilter genericDeleteFilter( + Table table, + IcebergScanConfig scanConfig, + String dataFilePath, + List deletes) { + return new BeamDeleteFilter( + table.io(), + dataFilePath, + scanConfig.getRequiredSchema(), + scanConfig.getProjectedSchema(), + deletes.stream() + .map(sdf -> sdf.createDeleteFile(table.specs(), table.sortOrders())) + .collect(Collectors.toList())); + } + + public static DeleteReader genericDeleteReader( + Table table, + IcebergScanConfig scanConfig, + String dataFilePath, + List deletes) { + return new BeamDeleteReader( + table.io(), + dataFilePath, + scanConfig.getRequiredSchema(), + scanConfig.getProjectedSchema(), + deletes.stream() + .map(sdf -> sdf.createDeleteFile(table.specs(), table.sortOrders())) + .collect(Collectors.toList())); + } + + public static class BeamDeleteFilter extends DeleteFilter { + private final FileIO io; + private final InternalRecordWrapper asStructLike; + + @SuppressWarnings("method.invocation") + public BeamDeleteFilter( + FileIO io, + String dataFilePath, + Schema tableSchema, + Schema projectedSchema, + List deleteFiles) { + super(dataFilePath, deleteFiles, tableSchema, projectedSchema); + this.io = io; + this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); + } + + // TODO: remove this (unused) + @SuppressWarnings("method.invocation") + public BeamDeleteFilter( + FileIO io, + SerializableChangelogTask scanTask, + Schema tableSchema, + Schema projectedSchema, + List deleteFiles) { + super(scanTask.getDataFile().getPath(), deleteFiles, tableSchema, projectedSchema); + this.io = io; + this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); + } + + // TODO: remove this (unused) + @SuppressWarnings("method.invocation") + public BeamDeleteFilter(FileIO io, ContentScanTask scanTask, List deleteFiles) { + super( + scanTask.file().location(), + deleteFiles, + scanTask.spec().schema(), + scanTask.spec().schema()); + 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 BeamDeleteReader extends DeleteReader { + private final FileIO io; + private final InternalRecordWrapper asStructLike; + + @SuppressWarnings("method.invocation") + public BeamDeleteReader( + FileIO io, + String dataFilePath, + Schema tableSchema, + Schema projectedSchema, + List deleteFiles) { + super(dataFilePath, deleteFiles, tableSchema, projectedSchema); + this.io = io; + this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); + } + + // TODO: remove this (unused) + @SuppressWarnings("method.invocation") + public BeamDeleteReader( + FileIO io, + SerializableChangelogTask scanTask, + Schema tableSchema, + Schema projectedSchema, + List deleteFiles) { + super(scanTask.getDataFile().getPath(), deleteFiles, tableSchema, projectedSchema); + this.io = io; + this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); + } + + // TODO: remove this (unused) + @SuppressWarnings("method.invocation") + public BeamDeleteReader(FileIO io, ContentScanTask scanTask, List deleteFiles) { + super( + scanTask.file().location(), + deleteFiles, + scanTask.spec().schema(), + scanTask.spec().schema()); + 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); + } + } } 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 81ec229df70f..452012766e3c 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 @@ -122,7 +122,10 @@ public boolean advance() throws IOException { InputFile input = decryptor.getInputFile(fileTask); Map idToConstants = ReadUtils.constantsMap( - fileTask, IdentityPartitionConverters::convertConstant, requiredSchema); + fileTask.spec(), + fileTask.file(), + IdentityPartitionConverters::convertConstant, + requiredSchema); 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 5c994c3e5651..ba3e4bfb59db 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(); @@ -101,6 +101,15 @@ public static Builder builder() { @SchemaFieldNumber("13") 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 { abstract Builder setPath(String path); @@ -131,6 +140,12 @@ 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(); } @@ -138,7 +153,7 @@ abstract static class Builder { * Create a {@link SerializableDataFile} from a {@link DataFile} and its associated {@link * PartitionKey}. */ - static SerializableDataFile from(DataFile f, String partitionPath) { + public static SerializableDataFile from(DataFile f, String partitionPath) { return SerializableDataFile.builder() .setPath(f.path().toString()) @@ -155,6 +170,9 @@ static SerializableDataFile from(DataFile f, String partitionPath) { .setNanValueCounts(f.nanValueCounts()) .setLowerBounds(toByteArrayMap(f.lowerBounds())) .setUpperBounds(toByteArrayMap(f.upperBounds())) + .setDataSequenceNumber(f.dataSequenceNumber()) + .setFileSequenceNumber(f.fileSequenceNumber()) + .setFirstRowId(f.firstRowId()) .build(); } @@ -192,14 +210,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; } @@ -210,8 +228,7 @@ DataFile createDataFile(Map partitionSpecs) { return output; } - private static @Nullable Map toByteBufferMap( - @Nullable Map input) { + static @Nullable Map toByteBufferMap(@Nullable Map input) { if (input == null) { return null; } @@ -244,10 +261,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; @@ -285,13 +305,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..d449403988a7 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SerializableDeleteFile.java @@ -0,0 +1,324 @@ +/* + * 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) { + return 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()) + .setLowerBounds(toByteArrayMap(deleteFile.lowerBounds())) + .setUpperBounds(toByteArrayMap(deleteFile.upperBounds())) + .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()) + .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 == null || getClass() != o.getClass()) { + return false; + } + SerializableDeleteFile that = (SerializableDeleteFile) o; + return getContentType().equals(that.getContentType()) + && getLocation().equals(that.getLocation()) + && getFileFormat().equals(that.getFileFormat()) + && getRecordCount() == that.getRecordCount() + && getFileSizeInBytes() == that.getFileSizeInBytes() + && getPartitionPath().equals(that.getPartitionPath()) + && getPartitionSpecId() == that.getPartitionSpecId() + && Objects.equals(getSortOrderId(), that.getSortOrderId()) + && Objects.equals(getEqualityFieldIds(), that.getEqualityFieldIds()) + && Objects.equals(getKeyMetadata(), that.getKeyMetadata()) + && Objects.equals(getSplitOffsets(), that.getSplitOffsets()) + && Objects.equals(getColumnSizes(), that.getColumnSizes()) + && Objects.equals(getValueCounts(), that.getValueCounts()) + && Objects.equals(getNullValueCounts(), that.getNullValueCounts()) + && Objects.equals(getNanValueCounts(), that.getNanValueCounts()) + && mapEquals(getLowerBounds(), that.getLowerBounds()) + && mapEquals(getUpperBounds(), that.getUpperBounds()) + && Objects.equals(getContentOffset(), that.getContentOffset()) + && Objects.equals(getContentSizeInBytes(), that.getContentSizeInBytes()) + && Objects.equals(getReferencedDataFile(), that.getReferencedDataFile()) + && Objects.equals(getDataSequenceNumber(), that.getDataSequenceNumber()) + && Objects.equals(getFileSequenceNumber(), that.getFileSequenceNumber()); + } + + @Override + public final int hashCode() { + int hashCode = + Objects.hash( + getContentType(), + getLocation(), + getFileFormat(), + getRecordCount(), + getFileSizeInBytes(), + getPartitionPath(), + getPartitionSpecId(), + getSortOrderId(), + getEqualityFieldIds(), + getKeyMetadata(), + getSplitOffsets(), + getColumnSizes(), + getValueCounts(), + getNullValueCounts(), + getNanValueCounts(), + getContentOffset(), + getContentSizeInBytes(), + getReferencedDataFile(), + getDataSequenceNumber(), + getFileSequenceNumber()); + hashCode = 31 * hashCode + computeMapByteHashCode(getLowerBounds()); + hashCode = 31 * hashCode + computeMapByteHashCode(getUpperBounds()); + return hashCode; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/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..2b35922a2270 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogDescriptor.java @@ -0,0 +1,67 @@ +/* + * 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; + +@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") + abstract long getStartSnapshotId(); + + @SchemaFieldNumber("2") + abstract long getEndSnapshotId(); + + @SchemaFieldNumber("3") + abstract int getChangeOrdinal(); + + @AutoValue.Builder + public abstract static class Builder { + abstract Builder setTableIdentifierString(String table); + + abstract Builder setStartSnapshotId(long snapshotId); + + abstract Builder setEndSnapshotId(long snapshotId); + + abstract Builder setChangeOrdinal(int ordinal); + + 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..4ae88e837c14 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ChangelogScanner.java @@ -0,0 +1,258 @@ +/* + * 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.SerializableChangelogTask.Type.ADDED_ROWS; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +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.SnapshotInfo; +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.TupleTag; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.IncrementalChangelogScan; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ChangelogScanner + extends DoFn< + KV>, KV>> { + 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 numBiDirectionalTasks = + Metrics.counter(ChangelogScanner.class, "numBiDirectionalTasks"); + public static final TupleTag>> + UNIDIRECTIONAL_CHANGES = new TupleTag<>(); + public static final TupleTag>> + BIDIRECTIONAL_CHANGES = new TupleTag<>(); + public static final KvCoder> OUTPUT_CODER = + KvCoder.of(ChangelogDescriptor.coder(), ListCoder.of(SerializableChangelogTask.coder())); + private final IcebergScanConfig scanConfig; + + ChangelogScanner(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @ProcessElement + public void process(@Element KV> element, MultiOutputReceiver out) + throws IOException { + // TODO: use TableCache here + Table table = scanConfig.getTable(); + table.refresh(); + + List snapshots = element.getValue(); + SnapshotInfo startSnapshot = snapshots.get(0); + SnapshotInfo endSnapshot = snapshots.get(snapshots.size() - 1); + @Nullable Long fromSnapshotId = startSnapshot.getParentId(); + long toSnapshot = endSnapshot.getSnapshotId(); + + IncrementalChangelogScan scan = + table + .newIncrementalChangelogScan() + .toSnapshot(toSnapshot) + .project(scanConfig.getProjectedSchema()); + if (fromSnapshotId != null) { + scan = scan.fromSnapshotExclusive(fromSnapshotId); + } + @Nullable Expression filter = scanConfig.getFilter(); + if (filter != null) { + scan = scan.filter(filter); + } + LOG.info("Planning to scan snapshot range [{}, {}]", fromSnapshotId, toSnapshot); + + createAndOutputReadTasks( + scan, startSnapshot, endSnapshot, SerializableTable.copyOf(table), out); + } + + private void createAndOutputReadTasks( + IncrementalChangelogScan scan, + SnapshotInfo startSnapshot, + SnapshotInfo endSnapshot, + Table table, + MultiOutputReceiver multiOutputReceiver) + throws IOException { + int numAddedRowsTasks = 0; + int numDeletedRowsTasks = 0; + int numDeletedFileTasks = 0; + + Map cachedSnapshotTimestamps = new HashMap<>(); + // Best effort maintain the same scan task groupings produced by Iceberg's binpacking, for + // better work load distribution among readers. + // This allows the user to control load per worker by tuning `read.split.target-size`: + // https://iceberg.apache.org/docs/latest/configuration/#read-properties + Map>> changelogScanTasks = new HashMap<>(); + + // keep track of the types of changes in each partition. do this for each ordinal + Map>> partitionChangeTypesPerOrdinal = + new HashMap<>(); + + try (CloseableIterable> scanTaskGroups = scan.planTasks()) { + for (ScanTaskGroup scanTaskGroup : scanTaskGroups) { + Map> ordinalTaskGroup = new HashMap<>(); + + for (ChangelogScanTask changelogScanTask : scanTaskGroup.tasks()) { + long snapshotId = changelogScanTask.commitSnapshotId(); + long timestampMillis = + cachedSnapshotTimestamps.computeIfAbsent( + snapshotId, (snapId) -> table.snapshot(snapId).timestampMillis()); + int ordinal = changelogScanTask.changeOrdinal(); + + SerializableChangelogTask task = + SerializableChangelogTask.from(changelogScanTask, timestampMillis); + String partition = task.getDataFile().getPartitionPath(); + + // gather metrics + switch (task.getType()) { + case ADDED_ROWS: + numAddedRowsTasks++; + break; + case DELETED_ROWS: + numDeletedRowsTasks++; + break; + case DELETED_FILE: + numDeletedFileTasks++; + break; + } + + partitionChangeTypesPerOrdinal + .computeIfAbsent(ordinal, (o) -> new HashMap<>()) + .computeIfAbsent(partition, (p) -> new HashSet<>()) + .add(task.getType()); + + ordinalTaskGroup.computeIfAbsent(ordinal, (o) -> new ArrayList<>()).add(task); + } + + for (Map.Entry> ordinalGroup : + ordinalTaskGroup.entrySet()) { + changelogScanTasks + .computeIfAbsent(ordinalGroup.getKey(), (unused) -> new ArrayList<>()) + .add(ordinalGroup.getValue()); + } + } + } + + int numUniDirTasks = 0; + int numBiDirTasks = 0; + + for (Map.Entry>> taskGroups : + changelogScanTasks.entrySet()) { + int ordinal = taskGroups.getKey(); + ChangelogDescriptor descriptor = + ChangelogDescriptor.builder() + .setTableIdentifierString(checkStateNotNull(startSnapshot.getTableIdentifierString())) + .setStartSnapshotId(startSnapshot.getSnapshotId()) + .setEndSnapshotId(endSnapshot.getSnapshotId()) + .setChangeOrdinal(ordinal) + .build(); + + for (List subgroup : taskGroups.getValue()) { + Instant timestamp = Instant.ofEpochMilli(subgroup.get(0).getTimestampMillis()); + + // Determine where each ordinal's tasks will go, based on the type of changes: + // 1. If an ordinal's changes are unidirectional (i.e. only inserts or only deletes), they + // should be processed directly in the fast path. + // 2. If an ordinal's changes are bidirectional (i.e. both inserts and deletes) within a + // partition, they will need more careful processing to determine if any updates have + // occurred. + Map> changeTypesPerPartition = + checkStateNotNull(partitionChangeTypesPerOrdinal.get(ordinal)); + List uniDirTasks = new ArrayList<>(); + List biDirTasks = new ArrayList<>(); + for (SerializableChangelogTask task : subgroup) { + Set partitionChangeTypes = + checkStateNotNull(changeTypesPerPartition.get(task.getDataFile().getPartitionPath())); + if (containsBiDirectionalChanges(partitionChangeTypes)) { + biDirTasks.add(task); + } else { + uniDirTasks.add(task); + } + } + + if (!uniDirTasks.isEmpty()) { + KV> uniDirOutput = + KV.of(descriptor, uniDirTasks); + multiOutputReceiver + .get(UNIDIRECTIONAL_CHANGES) + .outputWithTimestamp(uniDirOutput, timestamp); + numUniDirTasks += uniDirTasks.size(); + } + if (!biDirTasks.isEmpty()) { + KV> biDirOutput = + KV.of(descriptor, biDirTasks); + multiOutputReceiver + .get(BIDIRECTIONAL_CHANGES) + .outputWithTimestamp(biDirOutput, timestamp); + numBiDirTasks += biDirTasks.size(); + } + } + } + + int totalTasks = numAddedRowsTasks + numDeletedRowsTasks + numDeletedFileTasks; + totalChangelogScanTasks.inc(totalTasks); + numAddedRowsScanTasks.inc(numAddedRowsTasks); + numDeletedRowsScanTasks.inc(numDeletedRowsTasks); + numDeletedDataFileScanTasks.inc(numDeletedFileTasks); + numUniDirectionalTasks.inc(numUniDirTasks); + numBiDirectionalTasks.inc(numBiDirTasks); + + LOG.info( + "Snapshots [{}, {}] produced {} tasks:\n\t{} AddedRowsScanTasks\n\t{} DeletedRowsScanTasks\n\t{} DeletedDataFileScanTasks\n" + + "Observed {} uni-directional tasks and {} bi-directional tasks.", + startSnapshot.getSnapshotId(), + endSnapshot.getSnapshotId(), + totalTasks, + numAddedRowsTasks, + numDeletedRowsTasks, + numDeletedFileTasks, + numUniDirTasks, + numBiDirTasks); + } + + private static boolean containsBiDirectionalChanges( + Set changeTypes) { + return changeTypes.contains(ADDED_ROWS) && changeTypes.size() > 1; + } +} 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..e85bac6136a3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/DeleteReader.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.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.Collection; +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 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) { + this.filePath = filePath; + + 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()); + } + + protected DeleteReader( + String filePath, List deletes, Schema tableSchema, Schema requestedSchema) { + this(filePath, deletes, tableSchema, requestedSchema, true); + } + + 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; + } + + public CloseableIterable read(CloseableIterable records) { + return applyEqDeletes(applyPosDeletes(records)); + } + + 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 = deleteLoader().loadEqualityDeletes(deletes, deleteSchema); + Predicate isInDeleteSet = + record -> deleteSet.contains(projectRow.wrap(asStructLike(record))); + checkStateNotNull(isInDeleteSets).add(isInDeleteSet); + } + + return checkStateNotNull(isInDeleteSets); + } + + private CloseableIterable applyEqDeletes(CloseableIterable records) { + Predicate isEqDeleted = applyEqDeletes().stream().reduce(Predicate::or).orElse(t -> false); + + return CloseableIterable.filter(records, isEqDeleted); + } + + public PositionDeleteIndex deletedRowPositions() { + if (deleteRowPositions == null && !posDeletes.isEmpty()) { + deleteRowPositions = deleteLoader().loadPositionDeletes(posDeletes, filePath); + } + + return checkStateNotNull(deleteRowPositions); + } + + private CloseableIterable applyPosDeletes(CloseableIterable records) { + if (posDeletes.isEmpty()) { + return records; + } + + PositionDeleteIndex positionIndex = deletedRowPositions(); + Predicate isDeleted = record -> positionIndex.isDeleted(pos(record)); + return CloseableIterable.filter(records, isDeleted); + } + + 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..17bc97926a62 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/IncrementalChangelogSource.java @@ -0,0 +1,159 @@ +/* + * 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.BIDIRECTIONAL_CHANGES; +import static org.apache.beam.sdk.io.iceberg.cdc.ChangelogScanner.UNIDIRECTIONAL_CHANGES; +import static org.apache.beam.sdk.io.iceberg.cdc.ReadFromChangelogs.KEYED_DELETES; +import static org.apache.beam.sdk.io.iceberg.cdc.ReadFromChangelogs.KEYED_INSERTS; +import static org.apache.beam.sdk.io.iceberg.cdc.ReadFromChangelogs.UNIDIRECTIONAL_ROWS; +import static org.apache.beam.sdk.io.iceberg.cdc.ReconcileChanges.DELETES; +import static org.apache.beam.sdk.io.iceberg.cdc.ReconcileChanges.INSERTS; + +import java.util.List; +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.IncrementalScanSource; +import org.apache.beam.sdk.io.iceberg.SnapshotInfo; +import org.apache.beam.sdk.transforms.Flatten; +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.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +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; + +public class IncrementalChangelogSource extends IncrementalScanSource { + public IncrementalChangelogSource(IcebergScanConfig scanConfig) { + super(scanConfig); + } + + @Override + public PCollection expand(PBegin input) { + Table table = + scanConfig + .getCatalogConfig() + .catalog() + .loadTable(TableIdentifier.parse(scanConfig.getTableIdentifier())); + + PCollection>> snapshots = + MoreObjects.firstNonNull(scanConfig.getStreaming(), false) + ? unboundedSnapshots(input) + : boundedSnapshots(input, table); + + // scan each interval of snapshots and create groups of changelog tasks + PCollectionTuple changelogTasks = + snapshots + .apply(Redistribute.byKey()) + .apply( + "Create Changelog Tasks", + ParDo.of(new ChangelogScanner(scanConfig)) + .withOutputTags( + UNIDIRECTIONAL_CHANGES, TupleTagList.of(BIDIRECTIONAL_CHANGES))); + + // for changelog ordinal groups that have UNIDIRECTIONAL changes (i.e. all deletes, or all + // inserts), + // take the fast approach of just reading and emitting CDC records. + PCollection uniDirectionalCdcRows = + processUniDirectionalChanges( + changelogTasks.get(UNIDIRECTIONAL_CHANGES).setCoder(ChangelogScanner.OUTPUT_CODER)); + + // changelog ordinal groups that have BIDIRECTIONAL changes (i.e. both deletes and inserts) + // will need extra processing (including a shuffle) to identify any updates + PCollection biDirectionalCdcRows = + processBiDirectionalChanges( + changelogTasks.get(BIDIRECTIONAL_CHANGES).setCoder(ChangelogScanner.OUTPUT_CODER)); + + // Merge UNIDIRECTIONAL and BIDIRECTIONAL outputs + return PCollectionList.of(uniDirectionalCdcRows) + .and(biDirectionalCdcRows) + .apply(Flatten.pCollections()); + } + + private PCollection processUniDirectionalChanges( + PCollection>> + uniDirectionalChangelogs) { + return uniDirectionalChangelogs + .apply(Redistribute.arbitrarily()) + .apply( + "Read UniDirectional Changes", + ParDo.of(ReadFromChangelogs.of(scanConfig)) + .withOutputTags(UNIDIRECTIONAL_ROWS, TupleTagList.empty())) + .get(UNIDIRECTIONAL_ROWS) + .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema())); + } + + private PCollection processBiDirectionalChanges( + PCollection>> + biDirectionalChangelogs) { + PCollectionTuple biDirectionalKeyedRows = + biDirectionalChangelogs + .apply(Redistribute.arbitrarily()) + .apply( + "Read BiDirectional Changes", + ParDo.of(ReadFromChangelogs.withKeyedOutput(scanConfig)) + .withOutputTags(KEYED_INSERTS, TupleTagList.of(KEYED_DELETES))); + + // prior to CoGBK, set a windowing strategy to maintain the earliest timestamp in the window + // this allows us to emit records downstream that may have larger reified timestamps + Window>> windowingStrategy = + Window.>>into(new GlobalWindows()) + .withTimestampCombiner(TimestampCombiner.EARLIEST); + + // preserve the element's timestamp by moving it into the value + // in the normal case, this will be a no-op because all CDC rows in an ordinal have the same + // commit timestamp. + // but this will matter if we add custom watermarking, where record timestamps are + // derived from a specified column + KvCoder keyedOutputCoder = ReadFromChangelogs.keyedOutputCoder(scanConfig); + PCollection>> keyedInsertsWithTimestamps = + biDirectionalKeyedRows + .get(KEYED_INSERTS) + .setCoder(keyedOutputCoder) + .apply("Reify INSERT Timestamps", Reify.timestampsInValue()) + .apply(windowingStrategy); + PCollection>> keyedDeletesWithTimestamps = + biDirectionalKeyedRows + .get(KEYED_DELETES) + .setCoder(keyedOutputCoder) + .apply("Reify DELETE Timestamps", Reify.timestampsInValue()) + .apply(windowingStrategy); + + // CoGroup by record ID and emit any (DELETE + INSERT) pairs as updates: (UPDATE_BEFORE, + // UPDATE_AFTER) + return KeyedPCollectionTuple.of(INSERTS, keyedInsertsWithTimestamps) + .and(DELETES, keyedDeletesWithTimestamps) + .apply("CoGroupBy Row ID", CoGroupByKey.create()) + .apply("Reconcile Inserts and Deletes", ParDo.of(new ReconcileChanges())) + .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema())); + } +} 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..ad7c3c4288b5 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReadFromChangelogs.java @@ -0,0 +1,299 @@ +/* + * 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 java.io.IOException; +import java.util.List; +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; +import org.apache.beam.sdk.io.iceberg.SerializableDeleteFile; +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.splittabledofn.RestrictionTracker; +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.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.Table; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructProjection; +import org.joda.time.Instant; + +@DoFn.BoundedPerElement +public class ReadFromChangelogs + extends DoFn>, OutT> { + public static final TupleTag UNIDIRECTIONAL_ROWS = new TupleTag<>(); + public static final TupleTag> KEYED_INSERTS = new TupleTag<>(); + public static final TupleTag> KEYED_DELETES = new TupleTag<>(); + + private final Counter numAddedRowsScanTasksCompleted = + Metrics.counter(ReadFromChangelogs.class, "numAddedRowsScanTasksCompleted"); + private final Counter numDeletedRowsScanTasksCompleted = + Metrics.counter(ReadFromChangelogs.class, "numDeletedRowsScanTasksCompleted"); + private final Counter numDeletedDataFileScanTasksCompleted = + Metrics.counter(ReadFromChangelogs.class, "numDeletedDataFileScanTasksCompleted"); + + private final IcebergScanConfig scanConfig; + private final boolean keyedOutput; + private transient StructProjection recordIdProjection; + private transient org.apache.iceberg.Schema recordIdSchema; + private final Schema beamRowSchema; + private final Schema rowAndSnapshotIDBeamSchema; + private static final String SNAPSHOT_FIELD = "__beam__changelog__snapshot__id__"; + + private ReadFromChangelogs(IcebergScanConfig scanConfig, boolean keyedOutput) { + this.scanConfig = scanConfig; + this.keyedOutput = keyedOutput; + + this.beamRowSchema = icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); + org.apache.iceberg.Schema recordSchema = scanConfig.getProjectedSchema(); + this.recordIdSchema = recordSchema.select(recordSchema.identifierFieldNames()); + this.recordIdProjection = StructProjection.create(recordSchema, recordIdSchema); + + this.rowAndSnapshotIDBeamSchema = rowAndSnapshotIDBeamSchema(scanConfig); + } + + static ReadFromChangelogs of(IcebergScanConfig scanConfig) { + return new ReadFromChangelogs<>(scanConfig, false); + } + + static ReadFromChangelogs> withKeyedOutput(IcebergScanConfig scanConfig) { + return new ReadFromChangelogs<>(scanConfig, true); + } + + /** + * Determines the keyed output coder, which depends on the requested projected schema and the + * schema's identifier fields. + */ + static KvCoder keyedOutputCoder(IcebergScanConfig scanConfig) { + org.apache.iceberg.Schema recordSchema = scanConfig.getProjectedSchema(); + Schema rowAndSnapshotIDBeamSchema = rowAndSnapshotIDBeamSchema(scanConfig); + return KvCoder.of( + SchemaCoder.of(rowAndSnapshotIDBeamSchema), + SchemaCoder.of(icebergSchemaToBeamSchema(recordSchema))); + } + + private static Schema rowAndSnapshotIDBeamSchema(IcebergScanConfig scanConfig) { + org.apache.iceberg.Schema recordSchema = scanConfig.getProjectedSchema(); + org.apache.iceberg.Schema recordIdSchema = + recordSchema.select(recordSchema.identifierFieldNames()); + Schema rowIdBeamSchema = icebergSchemaToBeamSchema(recordIdSchema); + List fields = + ImmutableList.builder() + .add(Schema.Field.of(SNAPSHOT_FIELD, Schema.FieldType.INT64)) + .addAll(rowIdBeamSchema.getFields()) + .build(); + return new Schema(fields); + } + + @Setup + public void setup() { + // StructProjection is not serializable, so we need to recompute it when the DoFn gets + // deserialized + org.apache.iceberg.Schema recordSchema = scanConfig.getProjectedSchema(); + this.recordIdSchema = recordSchema.select(recordSchema.identifierFieldNames()); + this.recordIdProjection = StructProjection.create(recordSchema, recordIdSchema); + } + + @ProcessElement + public void process( + @Element KV> element, + RestrictionTracker tracker, + MultiOutputReceiver out) + throws IOException { + // TODO: use TableCache + Table table = scanConfig.getTable(); + table.refresh(); + + List tasks = element.getValue(); + + for (long l = tracker.currentRestriction().getFrom(); + l < tracker.currentRestriction().getTo(); + l++) { + if (!tracker.tryClaim(l)) { + return; + } + + SerializableChangelogTask task = tasks.get((int) l); + switch (task.getType()) { + case ADDED_ROWS: + processAddedRowsTask(task, table, out); + break; + case DELETED_ROWS: + processDeletedRowsTask(task, table, out); + break; + case DELETED_FILE: + processDeletedFileTask(task, table, out); + break; + } + } + } + + private void processAddedRowsTask( + SerializableChangelogTask task, Table table, MultiOutputReceiver outputReceiver) + throws IOException { + try (CloseableIterable fullIterable = ReadUtils.createReader(task, table, scanConfig)) { + DeleteFilter deleteFilter = + ReadUtils.genericDeleteFilter( + table, scanConfig, task.getDataFile().getPath(), task.getAddedDeletes()); + CloseableIterable filtered = deleteFilter.filter(fullIterable); + + for (Record rec : filtered) { + outputRecord( + "INSERT", + rec, + outputReceiver, + task.getCommitSnapshotId(), + task.getTimestampMillis(), + KEYED_INSERTS); + } + } + numAddedRowsScanTasksCompleted.inc(); + } + + private void processDeletedRowsTask( + SerializableChangelogTask task, Table table, MultiOutputReceiver outputReceiver) + throws IOException { + DeleteFilter existingDeletesFilter = + ReadUtils.genericDeleteFilter( + table, scanConfig, task.getDataFile().getPath(), task.getExistingDeletes()); + DeleteReader newDeletesReader = + ReadUtils.genericDeleteReader( + table, scanConfig, task.getDataFile().getPath(), task.getAddedDeletes()); + + try (CloseableIterable allRecords = ReadUtils.createReader(task, table, scanConfig)) { + CloseableIterable liveRecords = existingDeletesFilter.filter(allRecords); + CloseableIterable newlyDeletedRecords = newDeletesReader.read(liveRecords); + + for (Record rec : newlyDeletedRecords) { + // TODO: output with DELETE kind + outputRecord( + "DELETE", + rec, + outputReceiver, + task.getCommitSnapshotId(), + task.getTimestampMillis(), + KEYED_DELETES); + } + } + numDeletedRowsScanTasksCompleted.inc(); + } + + private void processDeletedFileTask( + SerializableChangelogTask task, Table table, MultiOutputReceiver outputReceiver) + throws IOException { + try (CloseableIterable fullIterable = ReadUtils.createReader(task, table, scanConfig)) { + DeleteFilter deleteFilter = + ReadUtils.genericDeleteFilter( + table, scanConfig, task.getDataFile().getPath(), task.getExistingDeletes()); + CloseableIterable filtered = deleteFilter.filter(fullIterable); + for (Record rec : filtered) { + // TODO: output with DELETE kind + outputRecord( + "DELETE-DF", + rec, + outputReceiver, + task.getCommitSnapshotId(), + task.getTimestampMillis(), + KEYED_DELETES); + } + } + numDeletedDataFileScanTasksCompleted.inc(); + } + + private void outputRecord( + String type, + Record rec, + MultiOutputReceiver outputReceiver, + long snapshotId, + long timestampMillis, + TupleTag> keyedTag) { + Row row = IcebergUtils.icebergRecordToBeamRow(beamRowSchema, rec); + Instant timestamp = Instant.ofEpochMilli(timestampMillis); + if (keyedOutput) { // slow path + StructProjection recId = recordIdProjection.wrap(rec); + // Create a Row ID consisting of: + // 1. the task's commit snapshot ID + // 2. the record ID column values + // This is needed to sufficiently distinguish a record change + Row id = structToBeamRow(snapshotId, recId, recordIdSchema, rowAndSnapshotIDBeamSchema); + outputReceiver.get(keyedTag).outputWithTimestamp(KV.of(id, row), timestamp); + } else { // fast path + System.out.printf("[UNIDIRECTIONAL] -- %s(%s, %s)\n%s%n", type, snapshotId, timestamp, row); + outputReceiver.get(UNIDIRECTIONAL_ROWS).outputWithTimestamp(row, timestamp); + } + } + + public static Row structToBeamRow( + long snapshotId, StructLike struct, org.apache.iceberg.Schema schema, Schema beamSchema) { + ImmutableMap.Builder values = ImmutableMap.builder(); + List columns = schema.columns(); + for (Types.NestedField column : columns) { + String name = column.name(); + Object value = schema.accessorForField(column.fieldId()).get(struct); + values.put(name, value); + } + // Include snapshot ID as part of the row ID. + // This is essential to ensure that the downstream ReconcileChanges compares rows + // within the same operation. + values.put(SNAPSHOT_FIELD, snapshotId); + return Row.withSchema(beamSchema).withFieldValues(values.build()).build(); + } + + @GetSize + public double getSize( + @Element KV> element, + @Restriction OffsetRange restriction) { + // TODO(ahmedabu98): this is just the compressed byte size. find a way to make a better estimate + long size = 0; + + for (long l = restriction.getFrom(); l < restriction.getTo(); l++) { + SerializableChangelogTask task = element.getValue().get((int) l); + size += task.getDataFile().getFileSizeInBytes(); + size += + task.getAddedDeletes().stream() + .mapToLong(SerializableDeleteFile::getFileSizeInBytes) + .sum(); + size += + task.getExistingDeletes().stream() + .mapToLong(SerializableDeleteFile::getFileSizeInBytes) + .sum(); + } + + return size; + } + + @GetInitialRestriction + public OffsetRange getInitialRange( + @Element KV> element) { + return new OffsetRange(0, element.getValue().size()); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReconcileChanges.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReconcileChanges.java new file mode 100644 index 000000000000..0647e546d64a --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/ReconcileChanges.java @@ -0,0 +1,86 @@ +/* + * 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.Iterator; +import org.apache.beam.sdk.transforms.DoFn; +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.TupleTag; +import org.joda.time.Instant; + +public class ReconcileChanges extends DoFn, Row> { + public static final TupleTag> DELETES = new TupleTag<>() {}; + public static final TupleTag> INSERTS = new TupleTag<>() {}; + + @DoFn.ProcessElement + public void processElement( + @Element KV element, + @Timestamp Instant timestamp, + OutputReceiver out) { + CoGbkResult result = element.getValue(); + + // iterables are lazy-loaded from the shuffle service + Iterable> deletes = result.getAll(DELETES); + Iterable> inserts = result.getAll(INSERTS); + + boolean hasDeletes = deletes.iterator().hasNext(); + boolean hasInserts = inserts.iterator().hasNext(); + + if (hasInserts && hasDeletes) { + // UPDATE: row ID exists in both streams + // - emit all deletes as 'UPDATE_BEFORE', and all inserts as 'UPDATE_AFTER' + // - emit extra inserts as 'UPDATE_AFTER' + // - ignore extra deletes (TODO: double check if this is a good decision) + Iterator> deletesIterator = deletes.iterator(); + Iterator> insertsIterator = inserts.iterator(); + while (deletesIterator.hasNext() && insertsIterator.hasNext()) { + // TODO: output as UPDATE_BEFORE kind + TimestampedValue updateBefore = deletesIterator.next(); + out.outputWithTimestamp(updateBefore.getValue(), updateBefore.getTimestamp()); + System.out.printf("[BIDIRECTIONAL] -- UpdateBefore\n%s\n", updateBefore); + + // TODO: output as UPDATE_AFTER kind + TimestampedValue updateAfter = insertsIterator.next(); + out.outputWithTimestamp(updateAfter.getValue(), updateAfter.getTimestamp()); + System.out.printf("[BIDIRECTIONAL] -- UpdateAfter\n%s\n", updateAfter); + } + while (insertsIterator.hasNext()) { + // TODO: output as UPDATE_AFTER kind + TimestampedValue insert = insertsIterator.next(); + out.outputWithTimestamp(insert.getValue(), insert.getTimestamp()); + System.out.printf("[BIDIRECTIONAL] -- Added(extra)\n%s\n", insert); + } + } else if (hasInserts) { + // INSERT only + for (TimestampedValue rec : inserts) { + System.out.printf("[BIDIRECTIONAL] -- Added\n%s\n", rec); + out.outputWithTimestamp(rec.getValue(), rec.getTimestamp()); + } + } else if (hasDeletes) { + // DELETE only + for (TimestampedValue rec : deletes) { + // TODO: output as DELETE kind + System.out.printf("[BIDIRECTIONAL] -- Deleted\n%s\n", rec); + out.outputWithTimestamp(rec.getValue(), rec.getTimestamp()); + } + } + } +} 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..6d5920ae5a7e --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/cdc/SerializableChangelogTask.java @@ -0,0 +1,223 @@ +/* + * 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 com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +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.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(); + + @SchemaFieldNumber("11") + public abstract long getTimestampMillis(); + + @SchemaIgnore + public Expression getExpression(Schema schema) { + return ExpressionParser.fromJson(getJsonExpression(), schema); + } + + @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) { + return setDataFile(SerializableDataFile.from(df, partitionPath)); + } + + 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 Builder setTimestampMillis(long timestampMillis); + + abstract SerializableChangelogTask build(); + } + + @SuppressWarnings("nullness") + public static SerializableChangelogTask from(ChangelogScanTask task, long timestampMillis) { + 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())) + .setSpecId(spec.specId()) + .setStart(contentScanTask.start()) + .setLength(contentScanTask.length()) + .setJsonExpression(ExpressionParser.toJson(contentScanTask.residual())) + .setTimestampMillis(timestampMillis); + + if (task instanceof AddedRowsScanTask) { + AddedRowsScanTask addedRowsTask = (AddedRowsScanTask) task; + builder = + builder + .setType(Type.ADDED_ROWS) + .setAddedDeletes(toSerializableDeletes(addedRowsTask.deletes(), spec)); + } else if (task instanceof DeletedRowsScanTask) { + DeletedRowsScanTask deletedRowsTask = (DeletedRowsScanTask) task; + builder = + builder + .setType(Type.DELETED_ROWS) + .setAddedDeletes(toSerializableDeletes(deletedRowsTask.addedDeletes(), spec)) + .setExistingDeletes(toSerializableDeletes(deletedRowsTask.existingDeletes(), spec)); + } else if (task instanceof DeletedDataFileScanTask) { + DeletedDataFileScanTask deletedFileTask = (DeletedDataFileScanTask) task; + builder = + builder + .setType(Type.DELETED_FILE) + .setExistingDeletes(toSerializableDeletes(deletedFileTask.existingDeletes(), spec)); + } else { + throw new IllegalStateException("Unknown ChangelogScanTask type: " + task.getClass()); + } + return builder.build(); + } + + private static List toSerializableDeletes( + List dfs, PartitionSpec spec) { + return dfs.stream() + .map(df -> SerializableDeleteFile.from(df, spec.partitionToPath(df.partition()))) + .collect(Collectors.toList()); + } + + public static Comparator comparator() { + return (task1, task2) -> { + int ordinalCompare = Integer.compare(task1.getOrdinal(), task2.getOrdinal()); + if (ordinalCompare != 0) { + return ordinalCompare; + } + + int op1Weight = getOperationWeight(task1.getOperation()); + int op2Weight = getOperationWeight(task2.getOperation()); + + return Integer.compare(op1Weight, op2Weight); + }; + } + + private static int getOperationWeight(ChangelogOperation op) { + switch (op) { + case DELETE: + case UPDATE_BEFORE: + return 0; + case INSERT: + case UPDATE_AFTER: + return 1; + default: + throw new UnsupportedOperationException("Unknown ChangelogOperation: " + op); + } + } +}