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 configuration of a CDC mutation info Callable for CDC Writes into BigQuery #32878

Draft
wants to merge 13 commits into
base: master
Choose a base branch
from
Draft
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@

{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 1
"modification": 2
}
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": 2
"modification": 3
}
201 changes: 150 additions & 51 deletions sdks/python/apache_beam/io/external/xlang_bigqueryio_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import time
import unittest
from decimal import Decimal
from typing import Dict

import pytest
from hamcrest.core import assert_that as hamcrest_assert
Expand Down Expand Up @@ -245,74 +246,89 @@ def test_write_with_beam_rows(self):
| StorageWriteToBigQuery(table=table_id))
hamcrest_assert(p, bq_matcher)

def test_write_with_beam_rows_cdc(self):
table = 'write_with_beam_rows_cdc'
table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table)

expected_data_on_bq = [
# (name, value)
{
"name": "cdc_test",
"value": 5,
}
]
EXPECTED_CDC_DATA = [
# (name, value, route)
{
"name": "cdc_test", "value": 5, "route": 3
}
]

rows_with_cdc = [
beam.Row(
row_mutation_info=beam.Row(
mutation_type="UPSERT", change_sequence_number="AAA/2"),
record=beam.Row(name="cdc_test", value=5)),
beam.Row(
row_mutation_info=beam.Row(
mutation_type="UPSERT", change_sequence_number="AAA/1"),
record=beam.Row(name="cdc_test", value=3))
]
CDC_ROW_DATA = [
beam.Row(name="cdc_test", value=5, route=3),
beam.Row(name="cdc_test", value=3, route=3),
beam.Row(name="cdc_test", value=2, route=1)
]

def run_and_validate_cdc_writes(
self, table_name, table_id, data, schema, use_cdc_writes):
bq_matcher = BigqueryFullResultMatcher(
project=self.project,
query="SELECT * FROM {}.{}".format(self.dataset_id, table),
data=self.parse_expected_data(expected_data_on_bq))
query="SELECT * FROM {}.{}".format(self.dataset_id, table_name),
data=self.parse_expected_data(self.EXPECTED_CDC_DATA))

with beam.Pipeline(argv=self.args) as p:
_ = (
p
| beam.Create(rows_with_cdc)
| beam.Create(data)
| beam.io.WriteToBigQuery(
table=table_id,
method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API,
use_at_least_once=True,
use_cdc_writes=True,
use_cdc_writes=use_cdc_writes,
schema=schema,
primary_key=["name"]))
hamcrest_assert(p, bq_matcher)

def test_write_with_dicts_cdc(self):
table = 'write_with_dicts_cdc'
def test_write_with_beam_rows_cdc_info_fn(self):
table = 'write_with_beam_rows_cdc_info_fn'
table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table)

expected_data_on_bq = [
# (name, value)
{
"name": "cdc_test",
"value": 5,
}
rows = self.CDC_ROW_DATA

def cdc_info_rows(row: beam.Row) -> beam.Row:
return beam.Row(
mutation_type="UPSERT",
change_sequence_number="AAA/" + str(row.value + row.route))

self.run_and_validate_cdc_writes(table, table_id, rows, None, cdc_info_rows)

def test_write_with_beam_rows_cdc(self):
table = 'write_with_beam_rows_cdc'
table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table)

rows_with_cdc = [
beam.Row(
row_mutation_info=beam.Row(
mutation_type="UPSERT", change_sequence_number="AAA/2"),
record=beam.Row(name="cdc_test", value=5, route=3)),
beam.Row(
row_mutation_info=beam.Row(
mutation_type="UPSERT", change_sequence_number="AAA/1"),
record=beam.Row(name="cdc_test", value=3, route=1))
]

self.run_and_validate_cdc_writes(table, table_id, rows_with_cdc, None, True)

def test_write_with_dicts_cdc(self):
table = 'write_with_dicts_cdc'
table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table)

data_with_cdc = [
# record: (name, value)
{
'row_mutation_info': {
'mutation_type': 'UPSERT', 'change_sequence_number': 'AAA/2'
},
'record': {
'name': 'cdc_test', 'value': 5
'name': 'cdc_test', 'value': 5, 'route': 3
}
},
{
'row_mutation_info': {
'mutation_type': 'UPSERT', 'change_sequence_number': 'AAA/1'
},
'record': {
'name': 'cdc_test', 'value': 3
'name': 'cdc_test', 'value': 3, 'route': 1
}
}
]
Expand Down Expand Up @@ -344,28 +360,51 @@ def test_write_with_dicts_cdc(self):
"name": "name", "type": "STRING"
}, {
"name": "value", "type": "INTEGER"
}, {
"name": "route", "type": "INTEGER"
}]
}
]
}

bq_matcher = BigqueryFullResultMatcher(
project=self.project,
query="SELECT * FROM {}.{}".format(self.dataset_id, table),
data=self.parse_expected_data(expected_data_on_bq))
self.run_and_validate_cdc_writes(
table, table_id, data_with_cdc, schema, True)

with beam.Pipeline(argv=self.args) as p:
_ = (
p
| beam.Create(data_with_cdc)
| beam.io.WriteToBigQuery(
table=table_id,
method=beam.io.WriteToBigQuery.Method.STORAGE_WRITE_API,
use_at_least_once=True,
use_cdc_writes=True,
schema=schema,
primary_key=["name"]))
hamcrest_assert(p, bq_matcher)
def test_write_with_dicts_cdc_info_fn(self):
table = 'write_with_dicts_cdc_info_fn'
table_id = '{}:{}.{}'.format(self.project, self.dataset_id, table)

data = [
# record: (name, value)
{
'name': 'cdc_test', 'value': 5, 'route': 3
},
{
'name': 'cdc_test', 'value': 3, 'route': 1
}
]

schema = {
"fields": [{
"name": "name", "type": "STRING"
}, {
"name": "value", "type": "INTEGER"
}, {
"name": "route", "type": "INTEGER"
}]
}

def cdc_info_fn(data: Dict) -> Dict:
if data["value"] == 3:
csn = 1
else:
csn = 2
return {
'mutation_type': 'UPSERT',
'change_sequence_number': 'AAA/' + str(csn)
}

self.run_and_validate_cdc_writes(table, table_id, data, schema, cdc_info_fn)

def test_write_to_dynamic_destinations(self):
base_table_spec = '{}.dynamic_dest_'.format(self.dataset_id)
Expand All @@ -391,6 +430,66 @@ def test_write_to_dynamic_destinations(self):
use_at_least_once=False))
hamcrest_assert(p, all_of(*bq_matchers))

def test_write_to_dynamic_destinations_rows_cdc_fn(self):
table_name_prefix = 'dynamic_dest_cdc_row_'
base_table_spec = '{}.{}'.format(self.dataset_id, table_name_prefix)
spec_with_project = '{}:{}'.format(self.project, base_table_spec)
table = table_name_prefix + "3"

def cdc_info_rows(row: beam.Row) -> beam.Row:
return beam.Row(
mutation_type="UPSERT",
change_sequence_number="AAA/" + str(row.value + row.route))

self.run_and_validate_cdc_writes(
table,
lambda record: spec_with_project + str(record.route),
self.CDC_ROW_DATA,
None,
cdc_info_rows)

def test_write_to_dynamic_destinations_dicts_cdc_fn(self):
table_name_prefix = 'dynamic_dest_cdc_dict_'
base_table_spec = '{}.{}'.format(self.dataset_id, table_name_prefix)
spec_with_project = '{}:{}'.format(self.project, base_table_spec)
table = table_name_prefix + "3"

data = [
# record: (name, value, route)
{
'name': 'cdc_test', 'value': 5, 'route': 3
},
{
'name': 'cdc_test', 'value': 3, 'route': 3
},
{
'name': 'cdc_test', 'value': 2, 'route': 1
}
]

schema = {
"fields": [{
"name": "name", "type": "STRING"
}, {
"name": "value", "type": "INTEGER"
}, {
"name": "route", "type": "INTEGER"
}]
}

def cdc_info_fn(data: Dict) -> Dict:
return {
'mutation_type': 'UPSERT',
'change_sequence_number': 'AAA/' + str(data["value"])
}

self.run_and_validate_cdc_writes(
table,
lambda data: spec_with_project + str(data["route"]),
data,
schema,
cdc_info_fn)

def test_write_to_dynamic_destinations_with_beam_rows(self):
base_table_spec = '{}.dynamic_dest_'.format(self.dataset_id)
spec_with_project = '{}:{}'.format(self.project, base_table_spec)
Expand Down
Loading
Loading