-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathbigquery_test.py
More file actions
152 lines (134 loc) · 4.97 KB
/
bigquery_test.py
File metadata and controls
152 lines (134 loc) · 4.97 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
import unittest
from unittest import mock
from parameterized import parameterized
import apache_beam as beam
# pylint: disable=ungrouped-imports
try:
from apache_beam.transforms.enrichment_handlers.bigquery import BigQueryEnrichmentHandler
from apache_beam.transforms.enrichment_handlers.bigquery_it_test import condition_value_fn
from apache_beam.transforms.enrichment_handlers.bigquery_it_test import query_fn
except ImportError:
raise unittest.SkipTest(
'Google Cloud BigQuery dependencies are not installed.')
class TestBigQueryEnrichment(unittest.TestCase):
def setUp(self) -> None:
self.project = 'apache-beam-testing'
@parameterized.expand([
("", "", [], None, None, 1, 2),
("table", "", ["id"], condition_value_fn, None, 2, 10),
("table", "id='{}'", ["id"], condition_value_fn, None, 2, 10),
("table", "id='{}'", ["id"], None, query_fn, 2, 10),
])
def test_valid_params(
self,
table_name,
row_restriction_template,
fields,
condition_value_fn,
query_fn,
min_batch_size,
max_batch_size):
"""
TC 1: Only batch size are provided. It should raise an error.
TC 2: Either of `row_restriction template` or `query_fn` is not provided.
TC 3: Both `fields` and `condition_value_fn` are provided.
TC 4: Query construction details are provided along with `query_fn`.
"""
with self.assertRaises(ValueError):
_ = BigQueryEnrichmentHandler(
project=self.project,
table_name=table_name,
row_restriction_template=row_restriction_template,
fields=fields,
condition_value_fn=condition_value_fn,
query_fn=query_fn,
min_batch_size=min_batch_size,
max_batch_size=max_batch_size,
)
def test_batch_mode_fans_out_response_for_duplicate_keys(self):
handler = BigQueryEnrichmentHandler(
project=self.project,
table_name='project.dataset.table',
row_restriction_template="id='{}'",
fields=['id'],
min_batch_size=2,
max_batch_size=2,
)
requests = [beam.Row(id='1', name='first'), beam.Row(id='1', name='second')]
with mock.patch.object(handler,
'_execute_query',
return_value=[{'id': '1', 'value': 'enriched'}]):
responses = handler(requests)
self.assertEqual(
responses,
[
(requests[0], beam.Row(id='1', value='enriched')),
(requests[1], beam.Row(id='1', value='enriched')),
],
)
def test_batch_mode_emits_empty_rows_for_all_unmatched_duplicate_keys(self):
handler = BigQueryEnrichmentHandler(
project=self.project,
table_name='project.dataset.table',
row_restriction_template="id='{}'",
fields=['id'],
min_batch_size=2,
max_batch_size=2,
throw_exception_on_empty_results=False,
)
requests = [beam.Row(id='1', name='first'), beam.Row(id='1', name='second')]
with mock.patch.object(handler, '_execute_query', return_value=None):
responses = handler(requests)
self.assertEqual(
responses,
[(requests[0], beam.Row()), (requests[1], beam.Row())],
)
def test_batch_elements_kwargs_include_max_batch_duration_secs(self):
handler = BigQueryEnrichmentHandler(
project=self.project,
table_name='project.dataset.table',
row_restriction_template="id='{}'",
fields=['id'],
min_batch_size=2,
max_batch_size=10,
max_batch_duration_secs=0.75,
)
self.assertEqual(
handler.batch_elements_kwargs(),
{
'min_batch_size': 2,
'max_batch_size': 10,
'max_batch_duration_secs': 0.75,
})
def test_batch_elements_kwargs_omit_max_batch_duration_secs_by_default(self):
handler = BigQueryEnrichmentHandler(
project=self.project,
table_name='project.dataset.table',
row_restriction_template="id='{}'",
fields=['id'],
min_batch_size=2,
max_batch_size=10,
)
self.assertEqual(
handler.batch_elements_kwargs(), {
'min_batch_size': 2,
'max_batch_size': 10,
})
if __name__ == '__main__':
unittest.main()