Skip to content
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
wants to merge 35 commits into
base: master
Choose a base branch
from

Conversation

prodriguezdefino
Copy link
Contributor

@prodriguezdefino prodriguezdefino commented Sep 21, 2024

Enables the configuration of CDC writes into a BigQuery table by setting the primary key columns to be used in the row mutations. This change adds the configuration of CDC usage for the BigQueryStorageWriteApiSchemaTransformProvider.

By setting configuration.setUseCDCWritesWithPrimaryKey(List.of("col1", "col2")) on the provider's config, the transform creation will create a BigQuery.Write<Row> transform that will configure the right row mutation information by checking for a Row schema like:

{
  cdc_info:{
    mutation_type: "..."              // can be UPSERT or DELETE
    change_sequence_number: "..."     // used for custom insert ordering
  }
  record:{...}                        // actual data to be inserted
}

The implementation also enables the possibility of setting a dynamic destination alongside with CDC usage by using this Row schema:

{
  cdc_info:{
    mutation_type: "..."              // can be UPSERT or DELETE
    change_sequence_number: "..."     // used for custom insert ordering
  }
  destination: "..."                  // destination table for the data
  record:{...}                        // actual data to be inserted
}

Note: In case of using dynamic destination and CDC is only supported when all the destination share the same primary key columns.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

@prodriguezdefino
Copy link
Contributor Author

fixes #32527

@prodriguezdefino
Copy link
Contributor Author

I have a question @ahmedabu98, how to make sure the xlang tests for BQ have run?

This branch test output for the test looks pretty similar to those that ran in master the last time, but in both cases it seems that the BQ StorageWrite transform is not being registered.

Am I looking at the right place or are the IT for xlang run in some other different task?

@ahmedabu98
Copy link
Contributor

The test you are referencing is to generate wrappers for external transforms (unrelated here since this wrapper is handwritten).

The workflows testing GCP xlang IOs are in beam_PostCommit_Python_Xlang_Gcp_Dataflow and beam_PostCommit_Python_Xlang_Gcp_Direct. These are post commits so to trigger them you just need to modify their respective target files and push it to this branch:

…plemented table constraint support on the bigquery fake dataset services
@prodriguezdefino prodriguezdefino marked this pull request as ready for review September 24, 2024 05:52
@github-actions github-actions bot added build and removed build labels Sep 26, 2024
@github-actions github-actions bot added build and removed build labels Sep 27, 2024
@github-actions github-actions bot added build and removed build labels Sep 27, 2024
Copy link
Contributor

@ahmedabu98 ahmedabu98 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Left some more comments

Comment on lines 273 to 276
"In the case of using CDC writes and setting CREATE_IF_NEEDED mode for the tables"
+ " a primary key is required.")
@Nullable
public abstract List<String> getCdcWritesPrimaryKey();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit:

Suggested change
"In the case of using CDC writes and setting CREATE_IF_NEEDED mode for the tables"
+ " a primary key is required.")
@Nullable
public abstract List<String> getCdcWritesPrimaryKey();
"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();

Comment on lines 553 to 569
if (!Strings.isNullOrEmpty(configuration.getCreateDisposition())) {
checkArgument(
BigQueryStorageWriteApiSchemaTransformConfiguration.CREATE_DISPOSITIONS
.get(configuration.getCreateDisposition().toUpperCase())
.equals(CreateDisposition.CREATE_IF_NEEDED)
&& !Optional.ofNullable(configuration.getCdcWritesPrimaryKey())
.orElse(ImmutableList.of())
.isEmpty(),
"When using CDC writes into BigQuery, alongside with CREATE_IF_NEEDED mode,"
+ " a primary key should be provided.");
}
if (configuration.getTable().equals(DYNAMIC_DESTINATIONS)) {
checkArgument(
schema.getFieldNames().contains("destination"),
"When writing to dynamic destinations, we expect Row Schema with a "
+ "\"destination\" string field.");
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's lean on the existing checks -- I don't think we need to create new ones. The first check is already covered in BigQueryIO, and the second check is covered above in this file

write =
write
.to(dynamicDestination)
.to(new RowDynamicDestinations(schema.getField("record").getType().getRowSchema()))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we just instantiate new RowDynamicDestinations(<row schema>, <primary key>) here? And avoid instantiating it again in validateAndIncludeCDCInformation?

Comment on lines 525 to 526
} else if (Optional.ofNullable(configuration.getUseCdcWrites()).orElse(false)) {
write = validateAndIncludeCDCInformation(write, schema);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we make this a separate if block, outside of this if/else chain? We should be able to apply this method to both dynamic destination and single table cases. The only factor should be whether or not useCdcWrites is true

Comment on lines 587 to 592
RowDynamicDestinations destinations =
new RowDynamicDestinations(schema.getField("record").getType().getRowSchema())
.withPrimaryKey(configuration.getCdcWritesPrimaryKey());
if (!configuration.getTable().equals(DYNAMIC_DESTINATIONS)) {
destinations = destinations.withFixedDestination(configuration.getTable());
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we remove this duplicated code? (see previous comment)

Comment on lines 301 to 305
BigQueryStorageWriteApiSchemaTransformConfiguration.builder()
.setTable(dynamic)
.setUseCdcWritesWithPrimaryKey(primaryKeyColumns)
.setUseCdcWrites(true)
.setCdcWritesPrimaryKey(primaryKeyColumns)
.build();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder why this is possible without explicitly setting "at-least-once" mode

Comment on lines 2092 to 2096
use_cdc_writes: Configure the usage of CDC writes on BigQuery.
The argument can be used by passing True and the Beam Rows will be
sent as they are to the BigQuery sink which expects a 'record'
and 'cdc_info' properties.
Used for STORAGE_WRITE_API, working on 'at least once' mode.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"If True, your input elements are expected to have a 'record' field representing the record to write, and a 'cdc_info: {mutation_type: , change_sequence_number: }' field representing the mutation information."

cdc_writes_primary_key: When using CDC write on BigQuery and
CREATE_IF_NEEDED mode for the underlying tables a list of column names
is required to be configured as the primary key. Used for
STORAGE_WRITE_API.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
STORAGE_WRITE_API.
STORAGE_WRITE_API and at_least_once mode.

Comment on lines 329 to 331
use_cdc_writes=lambda row: beam.Row(
mutation_type="UPSERT",
change_sequence_number="AAA/" + str(row.value)),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note that when the user is writing Python dicts, they should be able to supply a function that works on those dicts (ie. they shouldn't have to know what a Beam Row). Most Python users are not aware that Beam Rows and the Java IO are being used under the hood.

use_cdc_writes = False
# if CDC functionality is configured we need to check if a callable has
# been passed to extract MutationInfo from the rows to be written
if callable(self._use_cdc_writes):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm realizing that this callable CDC writes option is more complex than I thought. For it to be complete and not confusing for users, we will have to provide this logic for both Python dict inputs and Beam Row inputs

I suggest we keep the CDC option to a boolean and make this improvement in a future PR

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let me try to have it implemented for both in this change, if you still see it incomplete I will remove it after the next review.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removed Callable argument for now, will work on it on a separated PR.

@github-actions github-actions bot added build and removed build labels Oct 2, 2024
@github-actions github-actions bot added build and removed build labels Oct 2, 2024
@github-actions github-actions bot added build and removed build labels Oct 2, 2024
@github-actions github-actions bot added build and removed build labels Oct 2, 2024
@github-actions github-actions bot added build and removed build labels Oct 2, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants