forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathexamples_test.py
More file actions
954 lines (810 loc) · 33.9 KB
/
examples_test.py
File metadata and controls
954 lines (810 loc) · 33.9 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
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
# coding=utf-8
#
# 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 glob
import json
import logging
import os
import random
import sys
import unittest
from typing import Any
from typing import Callable
from typing import Dict
from typing import List
from typing import Optional
from typing import Union
from unittest import mock
import pytest
import yaml
import apache_beam as beam
from apache_beam import PCollection
from apache_beam.examples.snippets.util import assert_matches_stdout
from apache_beam.ml.inference.base import PredictionResult
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.typehints.row_type import RowTypeConstraint
from apache_beam.utils import subprocess_server
from apache_beam.yaml import yaml_provider
from apache_beam.yaml import yaml_transform
from apache_beam.yaml.readme_test import TestEnvironment
from apache_beam.yaml.readme_test import replace_recursive
from . import input_data
@beam.ptransform.ptransform_fn
def test_enrichment(
pcoll,
enrichment_handler: str,
handler_config: Dict[str, Any],
timeout: Optional[float] = 30):
"""
Mocks the Enrichment transform for testing purposes.
This PTransform simulates the behavior of the Enrichment transform by
looking up data from predefined in-memory tables based on the provided
`enrichment_handler` and `handler_config`.
Note: The Github action that invokes these tests does not have gcp
dependencies installed which is a prerequisite to
apache_beam.transforms.enrichment.Enrichment as a top-level import.
Args:
pcoll: The input PCollection.
enrichment_handler: A string indicating the type of enrichment handler
to simulate (e.g., 'BigTable', 'BigQuery').
handler_config: A dictionary containing configuration details for the
simulated handler (e.g., table names, row keys, fields).
timeout: An optional timeout value (ignored in this mock).
Returns:
A PCollection containing the enriched data.
"""
if enrichment_handler == 'BigTable':
row_key = handler_config['row_key']
bt_data = INPUT_TABLES[(
'BigTable', handler_config['instance_id'], handler_config['table_id'])]
products = {str(data[row_key]): data for data in bt_data}
def _fn(row):
left = row._asdict()
right = products[str(left[row_key])]
left['product'] = left.get('product', None) or right
return beam.Row(**left)
elif enrichment_handler == 'BigQuery':
row_key = handler_config['fields']
dataset, table = handler_config['table_name'].split('.')[-2:]
bq_data = INPUT_TABLES[('BigQuery', str(dataset), str(table))]
bq_data = {
tuple(str(data[key]) for key in row_key): data
for data in bq_data
}
def _fn(row):
left = row._asdict()
right = bq_data[tuple(str(left[k]) for k in row_key)]
row = {
key: left.get(key, None) or right[key]
for key in {*left.keys(), *right.keys()}
}
return beam.Row(**row)
else:
raise ValueError(f'{enrichment_handler} is not a valid enrichment_handler.')
return pcoll | beam.Map(_fn)
@beam.ptransform.ptransform_fn
def test_kafka_read(
pcoll,
format,
topic,
bootstrap_servers,
auto_offset_reset_config,
consumer_config):
"""
This PTransform simulates the behavior of the ReadFromKafka transform
with the RAW format by simply using some fixed sample text data and
encode it to raw bytes.
Args:
pcoll: The input PCollection.
format: The format of the Kafka messages (e.g., 'RAW').
topic: The name of Kafka topic to read from.
bootstrap_servers: A list of Kafka bootstrap servers to connect to.
auto_offset_reset_config: A configuration for the auto offset reset
consumer_config: A dictionary containing additional consumer configurations
Returns:
A PCollection containing the sample text data in bytes.
"""
return (
pcoll | beam.Create(input_data.text_data().split('\n'))
| beam.Map(lambda element: beam.Row(payload=element.encode('utf-8'))))
@beam.ptransform.ptransform_fn
def test_pubsub_read(
pcoll,
topic: Optional[str] = None,
subscription: Optional[str] = None,
format: Optional[str] = None,
schema: Optional[Any] = None,
attributes: Optional[List[str]] = None,
attributes_map: Optional[str] = None,
id_attribute: Optional[str] = None,
timestamp_attribute: Optional[str] = None):
pubsub_messages = input_data.pubsub_messages_data()
return (
pcoll
| beam.Create([json.loads(msg.data) for msg in pubsub_messages])
| beam.Map(lambda element: beam.Row(**element)))
@beam.ptransform.ptransform_fn
def test_run_inference(pcoll, inference_tag, model_handler):
"""
This PTransform simulates the behavior of the RunInference transform.
Args:
pcoll: The input PCollection.
inference_tag: The tag to use for the returned inference.
model_handler: A configuration for the respective ML model handler
Returns:
A PCollection containing the enriched data.
"""
def _fn(row):
input = row._asdict()
row = {
inference_tag: PredictionResult(
input['comment_text'],
[{
'label': 'POSITIVE'
if 'happy' in input['comment_text'] else 'NEGATIVE',
'score': 0.95
}]),
**input
}
return beam.Row(**row)
user_type = RowTypeConstraint.from_user_type(pcoll.element_type.user_type)
user_schema_fields = [(name, type(typ) if not isinstance(typ, type) else typ)
for (name,
typ) in user_type._fields] if user_type else []
inference_output_type = RowTypeConstraint.from_fields([
('example', Any), ('inference', Any), ('model_id', Optional[str])
])
schema = RowTypeConstraint.from_fields(
user_schema_fields + [(str(inference_tag), inference_output_type)])
return pcoll | beam.Map(_fn).with_output_types(schema)
TEST_PROVIDERS = {
'TestEnrichment': test_enrichment,
'TestReadFromKafka': test_kafka_read,
'TestReadFromPubSub': test_pubsub_read,
'TestRunInference': test_run_inference
}
"""
Transforms not requiring inputs.
"""
INPUT_TRANSFORM_TEST_PROVIDERS = ['TestReadFromKafka', 'TestReadFromPubSub']
def check_output(expected: List[str]):
"""
Helper function to check the output of a pipeline against expected values.
This function takes a list of expected output strings and returns a
callable that can be used within a Beam pipeline to assert that the
actual output matches the expected output.
Args:
expected: A list of strings representing the expected output elements.
Returns:
A callable that takes a list of PCollections and asserts their combined
elements match the expected output.
"""
def _check_inner(actual: List[PCollection[str]]):
formatted_actual = actual | beam.Flatten() | beam.Map(
lambda row: str(beam.Row(**row._asdict())))
assert_matches_stdout(formatted_actual, expected)
return _check_inner
def create_test_method(
pipeline_spec_file: str,
custom_preprocessors: List[Callable[..., Union[Dict, List]]]):
"""
Generates a test method for a given YAML pipeline specification file.
This function reads the YAML file, extracts the expected output (if present),
and creates a test function that uses `TestPipeline` to run the pipeline
defined in the YAML file. It also applies any custom preprocessors registered
for this test.
Args:
pipeline_spec_file: The path to the YAML file containing the pipeline
specification.
custom_preprocessors: A list of preprocessor functions to apply before
running the test.
Returns:
A test method (Callable) that can be added to a unittest.TestCase class.
"""
@mock.patch('apache_beam.Pipeline', TestPipeline)
def test_yaml_example(self):
with open(pipeline_spec_file, encoding="utf-8") as f:
lines = f.readlines()
expected_key = '# Expected:\n'
if expected_key in lines:
expected = lines[lines.index('# Expected:\n') + 1:]
else:
raise ValueError(
f"Missing '# Expected:' tag in example file '{pipeline_spec_file}'")
for i, line in enumerate(expected):
expected[i] = line.replace('# ', '').replace('\n', '')
expected = [line for line in expected if line]
pipeline_spec = yaml.load(
''.join(lines), Loader=yaml_transform.SafeLineLoader)
with TestEnvironment() as env:
for fn in custom_preprocessors:
pipeline_spec = fn(pipeline_spec, expected, env)
with beam.Pipeline(options=PipelineOptions(
pickle_library='cloudpickle',
**yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get(
'options', {})))) as p:
actual = [
yaml_transform.expand_pipeline(
p,
pipeline_spec,
[
yaml_provider.InlineProvider(
TEST_PROVIDERS, INPUT_TRANSFORM_TEST_PROVIDERS)
])
]
if not actual[0]:
actual = list(p.transforms_stack[0].parts[-1].outputs.values())
for transform in p.transforms_stack[0].parts[:-1]:
if transform.transform.label == 'log_for_testing':
actual += list(transform.outputs.values())
check_output(expected)(actual)
def _python_deps_involved(spec_filename):
return any(
substr in spec_filename
for substr in ['deps', 'streaming_sentiment_analysis'])
if _python_deps_involved(pipeline_spec_file):
test_yaml_example = pytest.mark.no_xdist(test_yaml_example)
test_yaml_example = unittest.skipIf(
sys.platform == 'win32', "Github virtualenv permissions issues.")(
test_yaml_example)
# This test fails, with an import error, for some (but not all) cloud
# tox environments when run as a github action (not reproducible locally).
# Adding debugging makes the failure go away. All indications are that
# this is some testing environmental issue.
test_yaml_example = unittest.skipIf(
'-cloud' in os.environ.get('TOX_ENV_NAME', ''),
'Github actions environment issue.')(
test_yaml_example)
if 'java_deps' in pipeline_spec_file:
test_yaml_example = pytest.mark.xlang_sql_expansion_service(
test_yaml_example)
test_yaml_example = unittest.skipIf(
not os.path.exists(
subprocess_server.JavaJarServer.path_to_dev_beam_jar(
'sdks:java:extensions:sql:expansion-service:shadowJar')),
"Requires expansion service jars.")(
test_yaml_example)
return test_yaml_example
class YamlExamplesTestSuite:
"""
YamlExamplesTestSuites class is used to scan specified directories for .yaml
files and dynamically generate a Python test method. Additionally, it creates
a method to complete some preprocessing for mocking IO.
"""
_test_preprocessor: Dict[str, List[Callable[..., Union[Dict, List]]]] = {}
def __init__(self, name: str, path: str):
"""
Initializes the YamlExamplesTestSuite.
Args:
name: The name of the test suite. This will be used as the class name
for the dynamically generated test suite.
path: A string representing the path or glob pattern to search for
YAML example files.
"""
self._test_suite = self.create_test_suite(name, path)
def run(self):
"""
Runs the dynamically generated test suite.
This method simply returns the test suite class created during
initialization. The test runner (e.g., unittest.main()) can then be used
to discover and run the tests within this suite.
Returns:
The dynamically created unittest.TestCase subclass.
"""
return self._test_suite
@classmethod
def parse_test_methods(cls, path: str):
"""Scans a given path for YAML files and generates test methods.
This method uses glob to find files matching the provided path. For each
YAML file found, it constructs a unique test name and then calls
`create_test_method` to generate the actual test function.
It also retrieves any registered preprocessors for that specific test.
Args:
path: A string representing the path or glob pattern to search for
YAML example files.
Yields:
A tuple containing the generated test name (str) and the
corresponding test method (Callable).
"""
files = glob.glob(path)
if not files and os.path.exists(path) and os.path.isfile(path):
files = [path]
for file in files:
test_name = f'test_{file.split(os.sep)[-1].replace(".", "_")}'
custom_preprocessors = cls._test_preprocessor.get(test_name, [])
yield test_name, create_test_method(file, custom_preprocessors)
@classmethod
def create_test_suite(cls, name: str, path: str):
"""Dynamically creates a unittest.TestCase subclass with generated tests.
This method takes a suite name and a path (or glob pattern). It uses
`parse_test_methods` to find YAML files at the given path and generate
individual test methods for each. These generated test methods are then
added as attributes to a new class, which is a subclass of
`unittest.TestCase`.
Args:
name: The desired name for the dynamically created test suite class.
path: A string representing the path or glob pattern to search for
YAML example files, which will be used to generate test methods.
Returns:
A new class, subclass of `unittest.TestCase`, containing dynamically
generated test methods based on the YAML files found at the given path.
"""
return type(
name, (unittest.TestCase, ), dict(cls.parse_test_methods(path)))
@classmethod
def register_test_preprocessor(cls, test_names: Union[str, List]):
"""Decorator to register a preprocessor function for specific tests.
This decorator is used to associate a preprocessor function with one or
more test names. The preprocessor function will be called before the
corresponding test is executed, allowing for modification of the test
specification or environment setup.
Args:
test_names: A string or a list of strings representing the names of the
tests for which the preprocessor should be registered. The test names
should match the names generated by `parse_test_methods`.
Returns:
A decorator function that takes the preprocessor function as an argument
and registers it.
"""
if isinstance(test_names, str):
test_names = [test_names]
def apply(preprocessor):
for test_name in test_names:
if test_name not in cls._test_preprocessor:
cls._test_preprocessor[test_name] = []
cls._test_preprocessor[test_name].append(preprocessor)
return preprocessor
return apply
@YamlExamplesTestSuite.register_test_preprocessor('test_wordcount_minimal_yaml')
def _wordcount_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for the wordcount_minimal.yaml test.
This preprocessor generates a random input file based on the expected output
of the wordcount example. This allows the test to verify the pipeline's
correctness without relying on a fixed input file.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with the input file path replaced.
"""
all_words = []
for element in expected:
word = element.split('=')[1].split(',')[0].replace("'", '')
count = int(element.split('=')[2].replace(')', ''))
all_words += [word] * count
random.shuffle(all_words)
lines = []
while all_words:
line_length = random.randint(1, min(10, len(all_words)))
line = " ".join(
all_words.pop(random.randrange(len(all_words)))
for _ in range(line_length))
lines.append(line)
return replace_recursive(
test_spec,
'ReadFromText',
'path',
env.input_file('kinglear.txt', '\n'.join(lines)))
@YamlExamplesTestSuite.register_test_preprocessor(
['test_kafka_yaml', 'test_kafka_to_iceberg_yaml'])
def _kafka_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
test_spec = replace_recursive(
test_spec,
'ReadFromText',
'path',
env.input_file('kinglear.txt', input_data.text_data()))
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '') == 'ReadFromKafka':
transform['type'] = 'TestReadFromKafka'
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor([
'test_simple_filter_yaml',
'test_simple_filter_and_combine_yaml',
'test_iceberg_read_yaml',
'test_iceberg_write_yaml',
'test_kafka_yaml',
'test_spanner_read_yaml',
'test_spanner_write_yaml',
'test_enrich_spanner_with_bigquery_yaml',
'test_pubsub_topic_to_bigquery_yaml',
'test_pubsub_subscription_to_bigquery_yaml',
'test_jdbc_to_bigquery_yaml',
'test_spanner_to_avro_yaml',
'test_gcs_text_to_bigquery_yaml',
'test_sqlserver_to_bigquery_yaml',
'test_postgres_to_bigquery_yaml',
'test_kafka_to_iceberg_yaml',
'test_pubsub_to_iceberg_yaml',
'test_oracle_to_bigquery_yaml',
'test_mysql_to_bigquery_yaml',
'test_spanner_to_bigquery_yaml',
'test_streaming_sentiment_analysis_yaml',
'test_enrich_spanner_with_bigquery_yaml'
])
def _io_write_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve writing to IO.
This preprocessor replaces any WriteTo transform with a LogForTesting
transform. This allows the test to verify the data being written without
actually writing to an external system.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with WriteTo transforms replaced.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '').startswith('WriteTo'):
transform['type'] = 'LogForTesting'
transform['config'] = {
k: v
for (k, v) in transform.get('config', {}).items()
if (k.startswith('__') or k == 'error_handling')
}
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor([
'test_simple_filter_yaml',
'test_simple_filter_and_combine_yaml',
'test_gcs_text_to_bigquery_yaml'
])
def _file_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
This preprocessor replaces any file IO ReadFrom transform with a Create
transform that reads from a predefined in-memory dictionary. This allows
the test to verify the pipeline's correctness without relying on external
files.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with ReadFrom transforms replaced.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '').startswith('ReadFrom'):
file_name = transform['config']['path'].split('/')[-1]
return replace_recursive(
test_spec,
transform['type'],
'path',
env.input_file(file_name, INPUT_FILES[file_name]))
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor(['test_iceberg_read_yaml'])
def _iceberg_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from Iceberg.
This preprocessor replaces any ReadFromIceberg transform with a Create
transform that reads from a predefined in-memory dictionary. This allows
the test to verify the pipeline's correctness without relying on Iceberg
tables stored externally.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with ReadFromIceberg transforms replaced.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '') == 'ReadFromIceberg':
config = transform['config']
(db_name, table_name,
field_value_dynamic_destinations) = config['table'].split('.')
transform['type'] = 'Create'
transform['config'] = {
k: v
for k, v in config.items() if k.startswith('__')
}
transform['config']['elements'] = INPUT_TABLES[(
str(db_name),
str(table_name),
str(field_value_dynamic_destinations))]
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor([
'test_spanner_read_yaml',
'test_enrich_spanner_with_bigquery_yaml',
'test_spanner_to_avro_yaml',
'test_spanner_to_bigquery_yaml'
])
def _spanner_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from Spanner.
This preprocessor replaces any ReadFromSpanner transform with a Create
transform that reads from a predefined in-memory dictionary. This allows
the test to verify the pipeline's correctness without relying on external
Spanner instances.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with ReadFromSpanner transforms replaced.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '').startswith('ReadFromSpanner'):
config = transform['config']
instance, database = config['instance_id'], config['database_id']
if table := config.get('table', None) is None:
table = config.get('query', '').split('FROM')[-1].strip()
transform['type'] = 'Create'
transform['config'] = {
k: v
for k, v in config.items() if k.startswith('__')
}
elements = INPUT_TABLES[(str(instance), str(database), str(table))]
if config.get('query', None):
config['query'].replace('select ',
'SELECT ').replace(' from ', ' FROM ')
columns = set(
''.join(config['query'].split('SELECT ')[1:]).split(
' FROM', maxsplit=1)[0].split(', '))
if columns != {'*'}:
elements = [{
column: element[column]
for column in element if column in columns
} for element in elements]
transform['config']['elements'] = elements
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor(
['test_bigtable_enrichment_yaml', 'test_enrich_spanner_with_bigquery_yaml'])
def _enrichment_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve the Enrichment transform.
This preprocessor replaces the actual Enrichment transform with a mock
`TestEnrichment` transform. This allows the test to verify the pipeline's
correctness without requiring external services like BigTable or BigQuery.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with Enrichment transforms replaced.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '').startswith('Enrichment'):
transform['type'] = 'TestEnrichment'
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor([
'test_pubsub_topic_to_bigquery_yaml',
'test_pubsub_subscription_to_bigquery_yaml',
'test_pubsub_to_iceberg_yaml'
])
def _pubsub_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from Pub/Sub.
This preprocessor replaces any ReadFromPubSub transform with a Create
transform that reads from a predefined in-memory list of messages.
This allows the test to verify the pipeline's correctness without relying
on an active Pub/Sub subscription or topic.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '') == 'ReadFromPubSub':
transform['type'] = 'TestReadFromPubSub'
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor([
'test_jdbc_to_bigquery_yaml',
])
def _jdbc_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from generic Jdbc.
url syntax: 'jdbc:<database-type>://<host>:<port>/<database>'
"""
return _db_io_read_test_processor(
test_spec, lambda url: url.split('/')[-1], 'Jdbc')
@YamlExamplesTestSuite.register_test_preprocessor([
'test_sqlserver_to_bigquery_yaml',
])
def __sqlserver_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from SqlServer.
url syntax: 'jdbc:sqlserver://<host>:<port>;databaseName=<database>;
user=<user>;password=<password>;encrypt=false;trustServerCertificate=true'
"""
return _db_io_read_test_processor(
test_spec, lambda url: url.split(';')[1].split('=')[-1], 'SqlServer')
@YamlExamplesTestSuite.register_test_preprocessor([
'test_postgres_to_bigquery_yaml',
])
def __postgres_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from Postgres.
url syntax: 'jdbc:postgresql://<host>:<port>/shipment?user=<user>&
password=<password>'
"""
return _db_io_read_test_processor(
test_spec, lambda url: url.split('/')[3].split('?')[0], 'Postgres')
@YamlExamplesTestSuite.register_test_preprocessor([
'test_oracle_to_bigquery_yaml',
])
def __oracle_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from Oracle.
url syntax: 'jdbc:oracle:thin:system/oracle@<host>:{port}/<database>'
"""
return _db_io_read_test_processor(
test_spec, lambda url: url.split('/')[2], 'Oracle')
@YamlExamplesTestSuite.register_test_preprocessor([
'test_mysql_to_bigquery_yaml',
])
def __mysql_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve reading from MySql.
url syntax: 'jdbc:mysql://<host>:<port>/<database>?user=<user>&
password=<password>'
"""
return _db_io_read_test_processor(
test_spec, lambda url: url.split('/')[3].split('?')[0], 'MySql')
def _db_io_read_test_processor(
test_spec: dict, database_url_fn: Callable, database_type: str):
"""
This preprocessor replaces any ReadFrom<database> transform with a Create
transform that reads from a predefined in-memory list of records. This allows
the test to verify the pipeline's correctness without relying on an active
database.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
transform_name = f"ReadFrom{database_type}"
if transform.get('type', '').startswith(transform_name):
config = transform['config']
url = config['url']
database = database_url_fn(url)
if (table := config.get('table', None)) is None:
table = config.get('query', '').split('FROM')[-1].strip()
transform['type'] = 'Create'
transform['config'] = {
k: v
for k, v in config.items() if k.startswith('__')
}
elements = INPUT_TABLES[(database_type, database, table)]
if config.get('query', None):
config['query'].replace('select ',
'SELECT ').replace(' from ', ' FROM ')
columns = set(
''.join(config['query'].split('SELECT ')[1:]).split(
' FROM', maxsplit=1)[0].split(', '))
if columns != {'*'}:
elements = [{
column: element[column]
for column in element if column in columns
} for element in elements]
transform['config']['elements'] = elements
return test_spec
@YamlExamplesTestSuite.register_test_preprocessor(
'test_streaming_sentiment_analysis_yaml')
def _streaming_sentiment_analysis_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):
"""
Preprocessor for tests that involve the streaming sentiment analysis example.
This preprocessor replaces several IO transforms and the RunInference
transform.
This allows the test to verify the pipeline's correctness without relying on
external data sources and the model hosted on VertexAI.
Args:
test_spec: The dictionary representation of the YAML pipeline specification.
expected: A list of strings representing the expected output of the
pipeline.
env: The TestEnvironment object providing utilities for creating temporary
files.
Returns:
The modified test_spec dictionary with ... transforms replaced.
"""
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '') == 'PyTransform' and transform.get(
'name', '') == 'ReadFromGCS':
transform['windowing'] = {'type': 'fixed', 'size': '30s'}
file_name = 'youtube-comments.csv'
local_path = env.input_file(file_name, INPUT_FILES[file_name])
transform['config']['kwargs']['file_pattern'] = local_path
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '') == 'ReadFromKafka':
config = transform['config']
transform['type'] = 'ReadFromCsv'
transform['config'] = {
k: v
for k, v in config.items() if k.startswith('__')
}
transform['config']['path'] = ""
file_name = 'youtube-comments.csv'
test_spec = replace_recursive(
test_spec,
transform['type'],
'path',
env.input_file(file_name, INPUT_FILES[file_name]))
if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '') == 'RunInference':
transform['type'] = 'TestRunInference'
return test_spec
INPUT_FILES = {
'products.csv': input_data.products_csv(),
'kinglear.txt': input_data.text_data(),
'youtube-comments.csv': input_data.youtube_comments_csv()
}
INPUT_TABLES = {
('shipment-test', 'shipment', 'shipments'): input_data.shipments_data(),
('orders-test', 'order-database', 'orders'): input_data.
spanner_orders_data(),
('db', 'users', 'NY'): input_data.iceberg_dynamic_destinations_users_data(),
('BigTable', 'beam-test', 'bigtable-enrichment-test'): input_data.
bigtable_data(),
('BigQuery', 'ALL_TEST', 'customers'): input_data.bigquery_data(),
('Jdbc', 'shipment', 'shipments'): input_data.shipments_data(),
('SqlServer', 'shipment', 'shipments'): input_data.shipments_data(),
('Postgres', 'shipment', 'shipments'): input_data.shipments_data(),
('Oracle', 'shipment', 'shipments'): input_data.shipments_data(),
('MySql', 'shipment', 'shipments'): input_data.shipments_data()
}
YAML_DOCS_DIR = os.path.join(os.path.dirname(__file__))
AggregationTest = YamlExamplesTestSuite(
'AggregationExamplesTest',
os.path.join(YAML_DOCS_DIR, '../transforms/aggregation/*.yaml')).run()
BlueprintTest = YamlExamplesTestSuite(
'BlueprintExamplesTest',
os.path.join(YAML_DOCS_DIR, '../transforms/blueprint/*.yaml')).run()
ElementWiseTest = YamlExamplesTestSuite(
'ElementwiseExamplesTest',
os.path.join(YAML_DOCS_DIR, '../transforms/elementwise/*.yaml')).run()
ExamplesTest = YamlExamplesTestSuite(
'ExamplesTest', os.path.join(YAML_DOCS_DIR, '../*.yaml')).run()
IOTest = YamlExamplesTestSuite(
'IOExamplesTest', os.path.join(YAML_DOCS_DIR,
'../transforms/io/*.yaml')).run()
MLTest = YamlExamplesTestSuite(
'MLExamplesTest', os.path.join(YAML_DOCS_DIR,
'../transforms/ml/**/*.yaml')).run()
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()