-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Add streaming upsert write option. #2863
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
Changes from all commits
b047073
bab1b22
9d3d1ed
49835e1
34f02c7
dcdd403
28c18f3
8e397e3
4391ba7
4d33f18
56822c8
9ae91cc
8a8a812
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,6 +41,7 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter<RowData> { | |
| private final Schema schema; | ||
| private final Schema deleteSchema; | ||
| private final RowDataWrapper wrapper; | ||
| private final boolean upsert; | ||
|
|
||
| BaseDeltaTaskWriter(PartitionSpec spec, | ||
| FileFormat format, | ||
|
|
@@ -50,11 +51,13 @@ abstract class BaseDeltaTaskWriter extends BaseTaskWriter<RowData> { | |
| long targetFileSize, | ||
| Schema schema, | ||
| RowType flinkSchema, | ||
| List<Integer> equalityFieldIds) { | ||
| List<Integer> equalityFieldIds, | ||
| boolean upsert) { | ||
| super(spec, format, appenderFactory, fileFactory, io, targetFileSize); | ||
| this.schema = schema; | ||
| this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); | ||
| this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); | ||
| this.upsert = upsert; | ||
| } | ||
|
|
||
| abstract RowDataDeltaWriter route(RowData row); | ||
|
|
@@ -70,11 +73,19 @@ public void write(RowData row) throws IOException { | |
| switch (row.getRowKind()) { | ||
| case INSERT: | ||
| case UPDATE_AFTER: | ||
| if (upsert) { | ||
| writer.delete(row); | ||
| } | ||
|
Contributor
Author
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. I think we could only delete row on INSERT. I don't think there will be only have
Member
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. For an update operation in flink, the For the downstream iceberg sink, we need to handle all the
Contributor
Author
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. I think we can only delete row on
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. By name, I thought INSERT is "add a new row". Then we don't need to add a delete for it. But I guess it actually means "append a row (new or updated)".
Contributor
Author
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. As @openinx mentioned in #1996 (comment), we need to transform INSERT/UPDATE_AFTER to be UPSERT(delete + insert). If we don't add a delete on INSERT row when upsert mode is enable, we will get duplicate rows for same primary key. |
||
| writer.write(row); | ||
| break; | ||
|
|
||
| case DELETE: | ||
| case UPDATE_BEFORE: | ||
| if (upsert) { | ||
| break; // UPDATE_BEFORE is not necessary for UPDATE, we do nothing to prevent delete one row twice | ||
| } | ||
|
Comment on lines
+83
to
+85
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. Non-blocking question: Are there possible concerns with events coming out of order for some reason? I guess since the table commits are serializable, this isn't a concern as the same row for these equality fields shouldn't be updated twice in the same commit?
Member
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. It's a good question, @kbendick ! Let's describe the out-of-order in two dimension:
Hopefully, I've answered your question, @kbendick :-) |
||
| writer.delete(row); | ||
| break; | ||
| case DELETE: | ||
| writer.delete(row); | ||
| break; | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -41,6 +41,7 @@ | |||
| import org.apache.iceberg.DataFile; | ||||
| import org.apache.iceberg.DistributionMode; | ||||
| import org.apache.iceberg.FileFormat; | ||||
| import org.apache.iceberg.PartitionField; | ||||
Reo-LEI marked this conversation as resolved.
Show resolved
Hide resolved
|
||||
| import org.apache.iceberg.PartitionSpec; | ||||
| import org.apache.iceberg.Schema; | ||||
| import org.apache.iceberg.SerializableTable; | ||||
|
|
@@ -58,6 +59,8 @@ | |||
|
|
||||
| import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; | ||||
| import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; | ||||
| import static org.apache.iceberg.TableProperties.UPSERT_MODE_ENABLE; | ||||
| import static org.apache.iceberg.TableProperties.UPSERT_MODE_ENABLE_DEFAULT; | ||||
| import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; | ||||
| import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_DEFAULT; | ||||
| import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; | ||||
|
|
@@ -125,6 +128,7 @@ public static class Builder { | |||
| private boolean overwrite = false; | ||||
| private DistributionMode distributionMode = null; | ||||
| private Integer writeParallelism = null; | ||||
| private boolean upsert = false; | ||||
| private List<String> equalityFieldColumns = null; | ||||
| private String uidPrefix = null; | ||||
|
|
||||
|
|
@@ -212,6 +216,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. | ||||
| * | ||||
| * @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. | ||||
| * | ||||
|
|
@@ -321,7 +339,27 @@ private SingleOutputStreamOperator<WriteResult> appendWriter(DataStream<RowData> | |||
| equalityFieldIds.add(field.fieldId()); | ||||
| } | ||||
| } | ||||
| IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, flinkRowType, equalityFieldIds); | ||||
|
|
||||
| // Fallback to use upsert mode parsed from table properties if don't specify in job level. | ||||
| boolean upsertMode = upsert || PropertyUtil.propertyAsBoolean(table.properties(), | ||||
| UPSERT_MODE_ENABLE, UPSERT_MODE_ENABLE_DEFAULT); | ||||
|
|
||||
| // Validate the equality fields and partition fields if we enable the upsert mode. | ||||
| if (upsertMode) { | ||||
| Preconditions.checkState(!overwrite, | ||||
| "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); | ||||
|
Comment on lines
+348
to
+350
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. Can you add a test that verifies the builder doesn't allow Maybe I missed it, but seems like an important thing to have a test for in case of future code refractors.
Member
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. Agreed !
Contributor
Author
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. I have added a unittest to cover this, thanks for your reminder!:) @kbendick |
||||
| Preconditions.checkState(!equalityFieldIds.isEmpty(), | ||||
| "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); | ||||
| if (!table.spec().isUnpartitioned()) { | ||||
|
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. for my own learning, does partition field must be included in equality fields? e.g., we can have an equality field (like
Contributor
Author
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.
As @openinx comment as above, we shoule restrict the partition fields is a subset of equality fields to ensure we can delete the old data in same partition.
I think that is not a valid scenario, to keep
Member
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. If we have a table with If we just partition the table by |
||||
| for (PartitionField partitionField : table.spec().fields()) { | ||||
| Preconditions.checkState(equalityFieldIds.contains(partitionField.sourceId()), | ||||
| "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", | ||||
| partitionField, equalityFieldColumns); | ||||
| } | ||||
| } | ||||
| } | ||||
|
|
||||
| IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, flinkRowType, equalityFieldIds, upsertMode); | ||||
|
|
||||
| int parallelism = writeParallelism == null ? input.getParallelism() : writeParallelism; | ||||
| SingleOutputStreamOperator<WriteResult> writerStream = input | ||||
|
|
@@ -390,15 +428,17 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { | |||
|
|
||||
| static IcebergStreamWriter<RowData> createStreamWriter(Table table, | ||||
| RowType flinkRowType, | ||||
| List<Integer> equalityFieldIds) { | ||||
| 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); | ||||
|
|
||||
| Table serializableTable = SerializableTable.copyOf(table); | ||||
| TaskWriterFactory<RowData> taskWriterFactory = new RowDataTaskWriterFactory( | ||||
| serializableTable, flinkRowType, targetFileSize, | ||||
| fileFormat, equalityFieldIds); | ||||
| fileFormat, equalityFieldIds, upsert); | ||||
|
|
||||
| return new IcebergStreamWriter<>(table.name(), taskWriterFactory); | ||||
| } | ||||
|
|
||||
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.
Two questions, one that's somewhat unrelated:
write.upsert.enabledflag, could we possibly use it to add our own support for CDC on top of Spark Structured Streaming?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.
Yes, it's only used for streaming mode right now. The batch upsert semantic has been implemented correctly by the MERGE INTO clause.
In theory, it's possible to add the CDC support for spark sturctured streaming, though the spark structured streaming does not support CDC event natively (I mean flink support INSERT/DELETE/UPDATE_BEFORE/UPDATE_AFTER events natively while Spark streaming doesn't unless we add extra field to indicate what's the operation type it is). I think @XuQianJin-Stars @chenjunjiedada 's team are working on this issue in their own repo.