-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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
Enable BigQuery CDC configuration for Python BigQuery sink #32529
Open
prodriguezdefino
wants to merge
36
commits into
apache:master
Choose a base branch
from
prodriguezdefino:python_xlang_cdc_bigquery
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.
+333
−18
Open
Changes from all commits
Commits
Show all changes
36 commits
Select commit
Hold shift + click to select a range
745cd58
include CDC configuration on the storage write transform provider
prodriguezdefino 504d2a4
adding the primary key configuration for CDC and tests
prodriguezdefino a2eabf9
fixing List.of references to use ImmutableList
prodriguezdefino 01703c1
fixing test, missing calling the cdc info row builder() method
prodriguezdefino 51360d5
fix test, add config validations
prodriguezdefino 542a49e
added the xlang params to storage write python wrapper
prodriguezdefino fe36fe8
adding missing comma
prodriguezdefino 1004f91
shortening property name
prodriguezdefino de9e948
changing xlang config property
prodriguezdefino ab40dd9
set use cdc schema property as nullable, added safe retrieval method
prodriguezdefino 27e5634
fixes property name reference and argument type definition
prodriguezdefino e804618
python format fix
prodriguezdefino 7b7255b
adding xlang IT with BQ
prodriguezdefino 2b901eb
adding missing primary key column to test
prodriguezdefino bb16979
python format fix
prodriguezdefino 12dce1b
format xlang test
prodriguezdefino 27a86db
more format xlang test fixes
prodriguezdefino e5766e7
and more format xlang test fixes
prodriguezdefino 09f298a
adding missing import
prodriguezdefino e972c89
missing self reference
prodriguezdefino ec3373b
enabled create if needed functionality for CDC python integration, im…
prodriguezdefino 23ea4c3
Update bigquery.py
prodriguezdefino 143566a
triggering the xlang tests
prodriguezdefino 4e390c7
fixing lint
prodriguezdefino 51bae40
addressing few comments
prodriguezdefino 6331691
cdc info is added after row transformation now
prodriguezdefino 56fe2b6
remove not used param
prodriguezdefino ce85c88
Merge branch 'master' into python_xlang_cdc_bigquery
prodriguezdefino bd324c5
removed typing information for callable
prodriguezdefino 3e5e31c
adding test for cdc using dicts as input and cdc write callable
prodriguezdefino 0073348
simplifying the xlang configuration from python perspective, will add…
prodriguezdefino 9dd4cfa
spotless apply
prodriguezdefino 4208517
wrong property passed to xlang builder
prodriguezdefino cd7b6cd
missing self
prodriguezdefino 72d1bf4
fixing xlang it
prodriguezdefino 1487acc
fixes wrong property reference
prodriguezdefino 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
3 changes: 2 additions & 1 deletion
3
.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json
This file contains 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 |
---|---|---|
@@ -1,4 +1,5 @@ | ||
|
||
{ | ||
"comment": "Modify this file in a trivial way to cause this test suite to run" | ||
"comment": "Modify this file in a trivial way to cause this test suite to run", | ||
"modification": 1 | ||
} |
2 changes: 1 addition & 1 deletion
2
.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json
This file contains 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 |
---|---|---|
@@ -1,4 +1,4 @@ | ||
{ | ||
"comment": "Modify this file in a trivial way to cause this test suite to run", | ||
"modification": 1 | ||
"modification": 2 | ||
} |
This file contains 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 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 |
---|---|---|
|
@@ -20,13 +20,15 @@ | |
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; | ||
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; | ||
|
||
import com.google.api.services.bigquery.model.TableConstraints; | ||
import com.google.api.services.bigquery.model.TableSchema; | ||
import com.google.auto.service.AutoService; | ||
import com.google.auto.value.AutoValue; | ||
import java.util.Arrays; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import javax.annotation.Nullable; | ||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; | ||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; | ||
|
@@ -37,6 +39,7 @@ | |
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageApiInsertError; | ||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; | ||
import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations; | ||
import org.apache.beam.sdk.io.gcp.bigquery.RowMutationInformation; | ||
import org.apache.beam.sdk.io.gcp.bigquery.TableDestination; | ||
import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; | ||
import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryStorageWriteApiSchemaTransformProvider.BigQueryStorageWriteApiSchemaTransformConfiguration; | ||
|
@@ -87,6 +90,14 @@ public class BigQueryStorageWriteApiSchemaTransformProvider | |
private static final String FAILED_ROWS_WITH_ERRORS_TAG = "FailedRowsWithErrors"; | ||
// magic string that tells us to write to dynamic destinations | ||
protected static final String DYNAMIC_DESTINATIONS = "DYNAMIC_DESTINATIONS"; | ||
protected static final String ROW_PROPERTY_MUTATION_INFO = "row_mutation_info"; | ||
protected static final String ROW_PROPERTY_MUTATION_TYPE = "mutation_type"; | ||
protected static final String ROW_PROPERTY_MUTATION_SQN = "change_sequence_number"; | ||
protected static final Schema ROW_SCHEMA_MUTATION_INFO = | ||
Schema.builder() | ||
.addStringField("mutation_type") | ||
.addStringField("change_sequence_number") | ||
.build(); | ||
|
||
@Override | ||
protected SchemaTransform from( | ||
|
@@ -257,6 +268,20 @@ public static Builder builder() { | |
@Nullable | ||
public abstract ErrorHandling getErrorHandling(); | ||
|
||
@SchemaFieldDescription( | ||
"This option enables the use of BigQuery CDC functionality. The expected PCollection" | ||
+ " should contain Beam Rows with a schema wrapping the record to be inserted and" | ||
+ " adding the CDC info similar to: {row_mutation_info: {mutation_type:\"...\", " | ||
+ "change_sequence_number:\"...\"}, record: {...}}") | ||
@Nullable | ||
public abstract Boolean getUseCdcWrites(); | ||
|
||
@SchemaFieldDescription( | ||
"If CREATE_IF_NEEDED disposition is set, BigQuery table(s) will be created with this" | ||
+ " primary key. Required when CDC writes are enabled with CREATE_IF_NEEDED.") | ||
@Nullable | ||
public abstract List<String> getPrimaryKey(); | ||
|
||
/** Builder for {@link BigQueryStorageWriteApiSchemaTransformConfiguration}. */ | ||
@AutoValue.Builder | ||
public abstract static class Builder { | ||
|
@@ -277,6 +302,10 @@ public abstract static class Builder { | |
|
||
public abstract Builder setErrorHandling(ErrorHandling errorHandling); | ||
|
||
public abstract Builder setUseCdcWrites(Boolean cdcWrites); | ||
|
||
public abstract Builder setPrimaryKey(List<String> pkColumns); | ||
|
||
/** Builds a {@link BigQueryStorageWriteApiSchemaTransformConfiguration} instance. */ | ||
public abstract BigQueryStorageWriteApiSchemaTransformProvider | ||
.BigQueryStorageWriteApiSchemaTransformConfiguration | ||
|
@@ -365,6 +394,34 @@ public TableSchema getSchema(String destination) { | |
} | ||
} | ||
|
||
private static class CdcWritesDynamicDestination extends RowDynamicDestinations { | ||
final String fixedDestination; | ||
final List<String> primaryKey; | ||
|
||
public CdcWritesDynamicDestination( | ||
Schema schema, String fixedDestination, List<String> primaryKey) { | ||
super(schema); | ||
this.fixedDestination = fixedDestination; | ||
this.primaryKey = primaryKey; | ||
} | ||
|
||
@Override | ||
public String getDestination(ValueInSingleWindow<Row> element) { | ||
return Optional.ofNullable(fixedDestination).orElseGet(() -> super.getDestination(element)); | ||
} | ||
|
||
@Override | ||
public TableConstraints getTableConstraints(String destination) { | ||
return Optional.ofNullable(this.primaryKey) | ||
.filter(pk -> !pk.isEmpty()) | ||
.map( | ||
pk -> | ||
new TableConstraints() | ||
.setPrimaryKey(new TableConstraints.PrimaryKey().setColumns(pk))) | ||
.orElse(null); | ||
} | ||
} | ||
|
||
@Override | ||
public PCollectionRowTuple expand(PCollectionRowTuple input) { | ||
// Check that the input exists | ||
|
@@ -453,6 +510,13 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { | |
} | ||
} | ||
|
||
void validateDynamicDestinationsExpectedSchema(Schema schema) { | ||
checkArgument( | ||
schema.getFieldNames().containsAll(Arrays.asList("destination", "record")), | ||
"When writing to dynamic destinations, we expect Row Schema with a " | ||
+ "\"destination\" string field and a \"record\" Row field."); | ||
} | ||
|
||
BigQueryIO.Write<Row> createStorageWriteApiTransform(Schema schema) { | ||
Method writeMethod = | ||
configuration.getUseAtLeastOnceSemantics() != null | ||
|
@@ -466,11 +530,11 @@ BigQueryIO.Write<Row> createStorageWriteApiTransform(Schema schema) { | |
.withFormatFunction(BigQueryUtils.toTableRow()) | ||
.withWriteDisposition(WriteDisposition.WRITE_APPEND); | ||
|
||
if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) { | ||
checkArgument( | ||
schema.getFieldNames().equals(Arrays.asList("destination", "record")), | ||
"When writing to dynamic destinations, we expect Row Schema with a " | ||
+ "\"destination\" string field and a \"record\" Row field."); | ||
// in case CDC writes are configured we validate and include them in the configuration | ||
if (Optional.ofNullable(configuration.getUseCdcWrites()).orElse(false)) { | ||
write = validateAndIncludeCDCInformation(write, schema); | ||
} else if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) { | ||
validateDynamicDestinationsExpectedSchema(schema); | ||
write = | ||
write | ||
.to(new RowDynamicDestinations(schema.getField("record").getType().getRowSchema())) | ||
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 we just instantiate |
||
|
@@ -485,6 +549,7 @@ BigQueryIO.Write<Row> createStorageWriteApiTransform(Schema schema) { | |
configuration.getCreateDisposition().toUpperCase()); | ||
write = write.withCreateDisposition(createDisposition); | ||
} | ||
|
||
if (!Strings.isNullOrEmpty(configuration.getWriteDisposition())) { | ||
WriteDisposition writeDisposition = | ||
BigQueryStorageWriteApiSchemaTransformConfiguration.WRITE_DISPOSITIONS.get( | ||
|
@@ -498,5 +563,52 @@ BigQueryIO.Write<Row> createStorageWriteApiTransform(Schema schema) { | |
|
||
return write; | ||
} | ||
|
||
BigQueryIO.Write<Row> validateAndIncludeCDCInformation( | ||
BigQueryIO.Write<Row> write, Schema schema) { | ||
checkArgument( | ||
schema.getFieldNames().containsAll(Arrays.asList(ROW_PROPERTY_MUTATION_INFO, "record")), | ||
"When writing using CDC functionality, we expect Row Schema with a " | ||
+ "\"" | ||
+ ROW_PROPERTY_MUTATION_INFO | ||
+ "\" Row field and a \"record\" Row field."); | ||
checkArgument( | ||
schema | ||
.getField(ROW_PROPERTY_MUTATION_INFO) | ||
.getType() | ||
.getRowSchema() | ||
.equals(ROW_SCHEMA_MUTATION_INFO), | ||
"When writing using CDC functionality, we expect a \"" | ||
+ ROW_PROPERTY_MUTATION_INFO | ||
+ "\" field of Row type with fields \"" | ||
+ ROW_PROPERTY_MUTATION_TYPE | ||
+ "\" and \"" | ||
+ ROW_PROPERTY_MUTATION_SQN | ||
+ "\" both of type string."); | ||
|
||
String tableDestination = null; | ||
|
||
if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) { | ||
validateDynamicDestinationsExpectedSchema(schema); | ||
} else { | ||
tableDestination = configuration.getTable(); | ||
} | ||
|
||
return write | ||
.to( | ||
new CdcWritesDynamicDestination( | ||
schema.getField("record").getType().getRowSchema(), | ||
tableDestination, | ||
configuration.getPrimaryKey())) | ||
.withFormatFunction(row -> BigQueryUtils.toTableRow(row.getRow("record"))) | ||
.withPrimaryKey(configuration.getPrimaryKey()) | ||
.withRowMutationInformationFn( | ||
row -> | ||
RowMutationInformation.of( | ||
RowMutationInformation.MutationType.valueOf( | ||
row.getRow(ROW_PROPERTY_MUTATION_INFO) | ||
.getString(ROW_PROPERTY_MUTATION_TYPE)), | ||
row.getRow(ROW_PROPERTY_MUTATION_INFO).getString(ROW_PROPERTY_MUTATION_SQN))); | ||
} | ||
} | ||
} |
This file contains 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
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.
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.
Note that BigQueryIO already attaches TableConstraints to the input destinations object:
beam/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
Lines 3537 to 3544 in d93f93a
To avoid duplicate logic, can we rely on the above and do away with the changes to
RowDynamicDestinations
?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.
the PTrasform instantiated by the provider does not execute this code. That’s the reason why I modified the row dynamic destination. I thought also on using a delegating dynamic destination but sadly that type has package limited access so I had only three options: introduce a larger change in the provider to directly use the BQIO apply, change visibility for the delegating dynamic destination, use inheritance for the row dyndest or modifying it. I thought the implemented option is the less disruptive from framework perspective.
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.
@ahmedabu98 I kept the modifications on the Row Dynamic Destinations given the access constraints on the delegating DynDest and that the shared check only applies to the case when a dynamic destination is not added into the BQIO.
Let me know if you want me to make more extensive changes to include the
expand
logic for BQIO.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.
Ahh the code block is executed only when dynamic destinations == null. Agreed a lil unfortunate but that's okay -- passing the primary key to RowDynamicDestinations and implementing
getTableConstraints()
should be enough