forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 12
Expand file tree
/
Copy pathxlang_jdbcio_it_test.py
More file actions
462 lines (402 loc) · 16 KB
/
xlang_jdbcio_it_test.py
File metadata and controls
462 lines (402 loc) · 16 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
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
#
# 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.
#
# pytype: skip-file
import datetime
import logging
import os
import time
import typing
import unittest
from decimal import Decimal
import pytest
from parameterized import parameterized
import apache_beam as beam
from apache_beam import coders
from apache_beam.io.jdbc import ReadFromJdbc
from apache_beam.io.jdbc import WriteToJdbc
from apache_beam.options.pipeline_options import StandardOptions
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.typehints.schemas import LogicalType
from apache_beam.typehints.schemas import MillisInstant
from apache_beam.utils.timestamp import Timestamp
# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
try:
import sqlalchemy
except ImportError:
sqlalchemy = None
# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports
# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
try:
from testcontainers.postgres import PostgresContainer
from testcontainers.mysql import MySqlContainer
except ImportError:
PostgresContainer = None
# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports
ROW_COUNT = 10
POSTGRES_BINARY_TYPE = ('BYTEA', 'BYTEA')
MYSQL_BINARY_TYPE = ('BINARY(10)', 'VARBINARY(10)')
JdbcTestRow = typing.NamedTuple(
"JdbcTestRow",
[("f_id", int), ("f_id_long", int), ("f_float", float), ("f_char", str),
("f_varchar", str), ("f_bytes", bytes), ("f_varbytes", bytes),
("f_timestamp", Timestamp), ("f_decimal", Decimal),
("f_date", datetime.date), ("f_time", datetime.time)],
)
coders.registry.register_coder(JdbcTestRow, coders.RowCoder)
CustomSchemaRow = typing.NamedTuple(
"CustomSchemaRow",
[
("renamed_id", int),
("renamed_id_long", int),
("renamed_float", float),
("renamed_char", str),
("renamed_varchar", str),
("renamed_bytes", bytes),
("renamed_varbytes", bytes),
("renamed_timestamp", Timestamp),
("renamed_decimal", Decimal),
("renamed_date", datetime.date),
("renamed_time", datetime.time),
],
)
coders.registry.register_coder(CustomSchemaRow, coders.RowCoder)
SimpleRow = typing.NamedTuple(
"SimpleRow", [("id", int), ("name", str), ("value", float)])
coders.registry.register_coder(SimpleRow, coders.RowCoder)
@pytest.mark.uses_gcp_java_expansion_service
@unittest.skipUnless(
os.environ.get('EXPANSION_JARS'),
"EXPANSION_JARS environment var is not provided, "
"indicating that jars have not been built")
@unittest.skipIf(sqlalchemy is None, 'sql alchemy package is not installed.')
@unittest.skipIf(
PostgresContainer is None, 'testcontainers package is not installed')
@unittest.skipIf(
TestPipeline().get_pipeline_options().view_as(StandardOptions).runner
is None,
'Do not run this test on precommit suites.')
@unittest.skipIf(
TestPipeline().get_pipeline_options().view_as(StandardOptions).runner
is not None and
"dataflowrunner" in TestPipeline().get_pipeline_options().view_as(
StandardOptions).runner.lower(),
'Do not run this test on dataflow runner.')
class CrossLanguageJdbcIOTest(unittest.TestCase):
DbData = typing.NamedTuple(
'DbData',
[('container_fn', typing.Any), ('classpath', typing.List[str]),
('db_string', str), ('connector', str)])
DB_CONTAINER_CLASSPATH_STRING = {
'postgres': DbData(
lambda: PostgresContainer('postgres:12.3'),
None,
'postgresql',
'org.postgresql.Driver'),
'mysql': DbData(
lambda: MySqlContainer(), ['mysql:mysql-connector-java:8.0.28'],
'mysql',
'com.mysql.cj.jdbc.Driver')
}
@classmethod
def setUpClass(cls):
cls.containers = {}
cls.engines = {}
cls.jdbc_configs = {}
for db_type, db_data in cls.DB_CONTAINER_CLASSPATH_STRING.items():
container = cls.start_container(db_data.container_fn)
cls.containers[db_type] = container
cls.engines[db_type] = sqlalchemy.create_engine(
container.get_connection_url())
cls.jdbc_configs[db_type] = {
'username': 'test',
'password': 'test',
'host': container.get_container_host_ip(),
'port': container.get_exposed_port(container.port_to_expose),
'database_name': 'test',
'driver_class_name': db_data.connector,
'classpath': db_data.classpath,
'jdbc_url': (
f'jdbc:{db_data.db_string}://{container.get_container_host_ip()}:'
f'{container.get_exposed_port(container.port_to_expose)}/test'),
'binary_type': POSTGRES_BINARY_TYPE
if db_type == 'postgres' else MYSQL_BINARY_TYPE
}
@classmethod
def tearDownClass(cls):
for db_type, container in cls.containers.items():
if container:
# Sometimes stopping the container raises ReadTimeout. We can ignore it
# here to avoid the test failure.
try:
container.stop()
except Exception: # pylint: disable=broad-except
logging.warning("Could not stop %s container", db_type)
@classmethod
def start_container(cls, container_init, max_retries=3):
# Creating a container with testcontainers sometimes raises ReadTimeout
# error. In java there are 2 retries set by default.
for attempt in range(max_retries):
try:
container = container_init()
container.start()
return container
except Exception: # pylint: disable=broad-except
if attempt == max_retries - 1:
logging.error(
'Failed to initialize container after %s attempts', max_retries)
raise
def create_test_table(self, connection, table_name, database):
binary_type = self.jdbc_configs[database]['binary_type']
connection.execute(
sqlalchemy.text(
f"CREATE TABLE IF NOT EXISTS {table_name}" +
"(f_id INTEGER, f_id_long BIGINT, f_float DOUBLE PRECISION, " +
"f_char CHAR(10), f_varchar VARCHAR(10), " +
f"f_bytes {binary_type[0]}, f_varbytes {binary_type[1]}, " +
"f_timestamp TIMESTAMP(3), f_decimal DECIMAL(10, 2), " +
"f_date DATE, f_time TIME(3))"))
def generate_test_data(self, count):
return [
JdbcTestRow(
i - 3,
i - 3,
i + 0.1,
f'Test{i}',
f'Test{i}',
f'Test{i}'.encode(),
f'Test{i}'.encode(),
Timestamp.of(seconds=round(time.time(), 3)),
Decimal(f'{i-1}.23'),
datetime.date(1969 + i, i % 12 + 1, i % 31 + 1),
datetime.time(i % 24, i % 60, i % 60, (i * 1000) % 1_000_000))
for i in range(count)
]
@parameterized.expand(['postgres', 'mysql'])
def test_xlang_jdbc_write_read(self, database):
table_name = f"jdbc_write_read_test_{database}"
with self.engines[database].begin() as connection:
self.create_test_table(connection, table_name, database)
test_rows = self.generate_test_data(ROW_COUNT)
expected_rows = []
for row in test_rows:
f_char = row.f_char + ' ' * (10 - len(row.f_char))
f_bytes = row.f_bytes
if database != 'postgres':
f_bytes = row.f_bytes + b'\0' * (10 - len(row.f_bytes))
expected_rows.append(
JdbcTestRow(
row.f_id,
row.f_id,
row.f_float,
f_char,
row.f_varchar,
f_bytes,
row.f_bytes,
row.f_timestamp,
row.f_decimal,
row.f_date,
row.f_time))
config = self.jdbc_configs[database]
with TestPipeline() as p:
p.not_use_test_runner_api = True
_ = (
p
| beam.Create(test_rows).with_output_types(JdbcTestRow)
| 'Write to jdbc' >> WriteToJdbc(
table_name=table_name,
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath'],
))
# Register MillisInstant logical type to override the mapping from Timestamp
# originally handled by MicrosInstant.
LogicalType.register_logical_type(MillisInstant)
with TestPipeline() as p:
p.not_use_test_runner_api = True
result = (
p
| 'Read from jdbc' >> ReadFromJdbc(
table_name=table_name,
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath']))
assert_that(result, equal_to(expected_rows))
# Try the same read using the partitioned reader code path.
# Outputs should be the same.
with TestPipeline() as p:
p.not_use_test_runner_api = True
result = (
p
| 'Partitioned read from jdbc' >> ReadFromJdbc(
table_name=table_name,
partition_column='f_id',
partitions=3,
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath']))
assert_that(result, equal_to(expected_rows))
@parameterized.expand(['postgres', 'mysql'])
def test_xlang_jdbc_read_with_explicit_schema(self, database):
if self.containers[database] is None:
self.skipTest(f"{database} container could not be initialized")
table_name = f"jdbc_schema_test_{database}"
with self.engines[database].begin() as connection:
self.create_test_table(connection, table_name, database)
test_rows = self.generate_test_data(ROW_COUNT)
expected_rows = []
for row in test_rows:
f_char = row.f_char
f_bytes = row.f_bytes
if database != 'postgres':
f_bytes = row.f_bytes + b'\0' * (10 - len(row.f_bytes))
expected_rows.append(
CustomSchemaRow(
row.f_id,
row.f_id,
row.f_float,
f_char,
row.f_varchar,
f_bytes,
row.f_bytes,
row.f_timestamp,
row.f_decimal,
row.f_date,
row.f_time))
def custom_row_equals(expected, actual):
return (
expected.renamed_id == actual.renamed_id and
expected.renamed_id_long == actual.renamed_id_long and
expected.renamed_float == actual.renamed_float and
expected.renamed_char.rstrip() == actual.renamed_char.rstrip() and
expected.renamed_varchar == actual.renamed_varchar and
expected.renamed_bytes == actual.renamed_bytes and
expected.renamed_timestamp == actual.renamed_timestamp and
expected.renamed_decimal == actual.renamed_decimal and
expected.renamed_date == actual.renamed_date and
expected.renamed_time == actual.renamed_time)
config = self.jdbc_configs[database]
# Run write pipeline
with TestPipeline() as p:
p.not_use_test_runner_api = True
_ = (
p
| beam.Create(test_rows).with_output_types(JdbcTestRow)
| 'Write to jdbc' >> WriteToJdbc(
table_name=table_name,
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath'],
))
# Register MillisInstant logical type to override the mapping from Timestamp
# originally handled by MicrosInstant.
LogicalType.register_logical_type(MillisInstant)
# Run read pipeline with custom schema
with TestPipeline() as p:
p.not_use_test_runner_api = True
result = (
p
| 'Read from jdbc with schema' >> ReadFromJdbc(
table_name=table_name,
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath'],
schema=CustomSchemaRow))
assert_that(result, equal_to(expected_rows, equals_fn=custom_row_equals))
@parameterized.expand(['postgres', 'mysql'])
def test_xlang_jdbc_custom_statements(self, database):
# Skip if container wasn't initialized
if self.containers[database] is None:
self.skipTest(f"{database} container could not be initialized")
# Create a simple table for this test
table_name = f"jdbc_custom_statements_{database}"
with self.engines[database].begin() as connection:
connection.execute(
sqlalchemy.text(
f"CREATE TABLE IF NOT EXISTS {table_name}" +
"(id INTEGER, name VARCHAR(50), value DOUBLE PRECISION)"))
test_rows = [
SimpleRow(1, "Item1", 10.5),
SimpleRow(2, "Item2", 20.75),
SimpleRow(3, "Item3", 30.25),
SimpleRow(4, "Item4", 40.0),
SimpleRow(-5, "Item5", 50.5)
]
config = self.jdbc_configs[database]
write_statement = f"INSERT INTO {table_name} (id, name, value) VALUES \
(?, ?, ?)"
with TestPipeline() as p:
p.not_use_test_runner_api = True
_ = (
p
| beam.Create(test_rows).with_output_types(SimpleRow)
| 'Write with custom statement' >> WriteToJdbc(
table_name="",
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath'],
statement=write_statement))
# Schema inference fails when there is a WHERE clause, so we pass explicit
# schema.
read_query = f"SELECT id, name, value FROM {table_name} WHERE value > 25.0"
expected_filtered_rows = [row for row in test_rows if row.value > 25.0]
with TestPipeline() as p:
p.not_use_test_runner_api = True
result = (
p
| 'Read with custom query' >> ReadFromJdbc(
table_name="",
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath'],
query=read_query,
schema=SimpleRow))
assert_that(result, equal_to(expected_filtered_rows))
# JdbcIO#readWithPartitions requires custom queries to be passed as a
# wrapped subquery to table_name.
with TestPipeline() as p:
p.not_use_test_runner_api = True
result = (
p
| 'Read with custom query' >> ReadFromJdbc(
table_name=f"({read_query}) as subq",
driver_class_name=config['driver_class_name'],
jdbc_url=config['jdbc_url'],
username=config['username'],
password=config['password'],
classpath=config['classpath'],
partition_column="id",
schema=SimpleRow))
assert_that(result, equal_to(expected_filtered_rows))
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()