-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Transform INSERT as one DELETE following one INSERT if configure to use UPSERT #1996
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
Closed
Closed
Changes from all commits
Commits
Show all changes
2 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
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -39,6 +39,7 @@ | |
| import org.apache.flink.types.Row; | ||
| import org.apache.iceberg.DataFile; | ||
| import org.apache.iceberg.FileFormat; | ||
| import org.apache.iceberg.PartitionField; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.flink.FlinkSchemaUtil; | ||
|
|
@@ -115,6 +116,7 @@ public static class Builder { | |
| private TableSchema tableSchema; | ||
| private boolean overwrite = false; | ||
| private Integer writeParallelism = null; | ||
| private boolean upsert = false; | ||
| private List<String> equalityFieldColumns = null; | ||
|
|
||
| private Builder() { | ||
|
|
@@ -172,6 +174,20 @@ public Builder writeParallelism(int newWriteParallelism) { | |
| return this; | ||
| } | ||
|
|
||
| /** | ||
| * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which means it will | ||
| * DELETE the old records and then INSERT the new records. In partitioned table, the partition fields should be | ||
| * a subset of equality fields, otherwise the old row that located in partition-A could not be deleted by the | ||
| * new row that located in partition-B. | ||
|
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. Does anything validate this constraint? |
||
| * | ||
| * @param enable indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. | ||
| * @return {@link Builder} to connect the iceberg table. | ||
| */ | ||
| public Builder upsert(boolean enable) { | ||
| this.upsert = enable; | ||
| return this; | ||
| } | ||
|
|
||
| /** | ||
| * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. | ||
| * | ||
|
|
@@ -209,7 +225,22 @@ public DataStreamSink<RowData> build() { | |
| } | ||
| } | ||
|
|
||
| IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema, equalityFieldIds); | ||
| // Convert the iceberg schema to flink's RowType. | ||
| RowType flinkSchema = convertToRowType(table, tableSchema); | ||
|
|
||
| // Convert the INSERT stream to be an UPSERT stream if needed. | ||
| if (upsert) { | ||
| Preconditions.checkState(!equalityFieldIds.isEmpty(), | ||
| "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); | ||
| if (!table.spec().isUnpartitioned()) { | ||
| for (PartitionField partitionField : table.spec().fields()) { | ||
| Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()), | ||
| "Partition field '%s' is not included in equality fields: '%s'", partitionField, equalityFieldColumns); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, flinkSchema, equalityFieldIds, upsert); | ||
| IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite); | ||
|
|
||
| this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism; | ||
|
|
@@ -227,8 +258,7 @@ public DataStreamSink<RowData> build() { | |
| } | ||
| } | ||
|
|
||
| static IcebergStreamWriter<RowData> createStreamWriter(Table table, TableSchema requestedSchema, | ||
| List<Integer> equalityFieldIds) { | ||
| private static RowType convertToRowType(Table table, TableSchema requestedSchema) { | ||
| Preconditions.checkArgument(table != null, "Iceberg table should't be null"); | ||
|
|
||
| RowType flinkSchema; | ||
|
|
@@ -246,13 +276,22 @@ static IcebergStreamWriter<RowData> createStreamWriter(Table table, TableSchema | |
| flinkSchema = FlinkSchemaUtil.convert(table.schema()); | ||
| } | ||
|
|
||
| return flinkSchema; | ||
| } | ||
|
|
||
| static IcebergStreamWriter<RowData> createStreamWriter(Table table, | ||
| RowType flinkSchema, | ||
| List<Integer> equalityFieldIds, | ||
| boolean upsert) { | ||
| Preconditions.checkArgument(table != null, "Iceberg table should't be null"); | ||
|
|
||
| Map<String, String> props = table.properties(); | ||
| long targetFileSize = getTargetFileSizeBytes(props); | ||
| FileFormat fileFormat = getFileFormat(props); | ||
|
|
||
| TaskWriterFactory<RowData> taskWriterFactory = new RowDataTaskWriterFactory(table.schema(), flinkSchema, | ||
| table.spec(), table.locationProvider(), table.io(), table.encryption(), targetFileSize, fileFormat, props, | ||
| equalityFieldIds); | ||
| equalityFieldIds, upsert); | ||
|
|
||
| return new IcebergStreamWriter<>(table.name(), taskWriterFactory); | ||
| } | ||
|
|
||
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
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
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems like this should only happen for the
INSERTcase becauseUPDATE_AFTERimplies that there was anUPDATE_BEFOREthat will perform the delete. This would delete the same row twice in that case, causing more equality deletes to be written for the row.