-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Iceberg CDC streaming source #37191
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Draft
ahmedabu98
wants to merge
5
commits into
apache:master
Choose a base branch
from
ahmedabu98:cdc_read
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+1,963
−43
Draft
Iceberg CDC streaming source #37191
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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<Record> createReader(FileScanTask task, Table table, Schema schema) { | ||
| String filePath = task.file().path().toString(); | ||
| public static CloseableIterable<Record> 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<Record> createReader( | ||
| ContentScanTask<?> task, Table table, IcebergScanConfig scanConfig) { | ||
| return createReader( | ||
| table, scanConfig, task.spec(), task.file(), task.start(), task.length(), task.residual()); | ||
| } | ||
|
|
||
| public static CloseableIterable<Record> 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<Integer, ?> 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<Record> 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<Record> 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<Integer, ?> constantsMap( | ||
| FileScanTask task, | ||
| PartitionSpec spec, | ||
| ContentFile<?> file, | ||
| BiFunction<Type, Object, Object> converter, | ||
| org.apache.iceberg.Schema schema) { | ||
| PartitionSpec spec = task.spec(); | ||
| Set<Integer> 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<Record> maybeApplyFilter( | |
| } | ||
| return iterable; | ||
| } | ||
|
|
||
| public static DeleteFilter<Record> genericDeleteFilter( | ||
| Table table, | ||
| IcebergScanConfig scanConfig, | ||
| String dataFilePath, | ||
| List<SerializableDeleteFile> 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<Record> genericDeleteReader( | ||
| Table table, | ||
| IcebergScanConfig scanConfig, | ||
| String dataFilePath, | ||
| List<SerializableDeleteFile> 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<Record> { | ||
| private final FileIO io; | ||
| private final InternalRecordWrapper asStructLike; | ||
|
|
||
| @SuppressWarnings("method.invocation") | ||
| public BeamDeleteFilter( | ||
| FileIO io, | ||
| String dataFilePath, | ||
| Schema tableSchema, | ||
| Schema projectedSchema, | ||
| List<DeleteFile> 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<DeleteFile> 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<DeleteFile> deleteFiles) { | ||
| super( | ||
| scanTask.file().location(), | ||
| deleteFiles, | ||
| scanTask.spec().schema(), | ||
| scanTask.spec().schema()); | ||
| this.io = io; | ||
| this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); | ||
| } | ||
|
Comment on lines
+293
to
+316
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
|
|
||
| @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<Record> { | ||
| private final FileIO io; | ||
| private final InternalRecordWrapper asStructLike; | ||
|
|
||
| @SuppressWarnings("method.invocation") | ||
| public BeamDeleteReader( | ||
| FileIO io, | ||
| String dataFilePath, | ||
| Schema tableSchema, | ||
| Schema projectedSchema, | ||
| List<DeleteFile> 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<DeleteFile> 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<DeleteFile> deleteFiles) { | ||
| super( | ||
| scanTask.file().location(), | ||
| deleteFiles, | ||
| scanTask.spec().schema(), | ||
| scanTask.spec().schema()); | ||
| this.io = io; | ||
| this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); | ||
| } | ||
ahmedabu98 marked this conversation as resolved.
Show resolved
Hide resolved
Comment on lines
+345
to
+368
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
|
|
||
| @Override | ||
| protected StructLike asStructLike(Record record) { | ||
| return asStructLike.wrap(record); | ||
| } | ||
|
|
||
| @Override | ||
| protected InputFile getInputFile(String location) { | ||
| return io.newInputFile(location); | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.