1717#
1818# pytype: skip-file
1919import glob
20+ import json
2021import logging
2122import os
2223import random
@@ -124,11 +125,35 @@ def test_kafka_read(
124125 | beam .Map (lambda element : beam .Row (payload = element .encode ('utf-8' ))))
125126
126127
128+ @beam .ptransform .ptransform_fn
129+ def test_pubsub_read (
130+ pbegin ,
131+ topic : Optional [str ] = None ,
132+ subscription : Optional [str ] = None ,
133+ format : Optional [str ] = None ,
134+ schema : Optional [Any ] = None ,
135+ attributes : Optional [List [str ]] = None ,
136+ attributes_map : Optional [str ] = None ,
137+ id_attribute : Optional [str ] = None ,
138+ timestamp_attribute : Optional [str ] = None ):
139+
140+ pubsub_messages = input_data .pubsub_messages_data ()
141+
142+ return (
143+ pbegin
144+ | beam .Create ([json .loads (msg .data ) for msg in pubsub_messages ])
145+ | beam .Map (lambda element : beam .Row (** element )))
146+
147+
127148TEST_PROVIDERS = {
128- 'TestEnrichment' : test_enrichment , 'TestReadFromKafka' : test_kafka_read
149+ 'TestEnrichment' : test_enrichment ,
150+ 'TestReadFromKafka' : test_kafka_read ,
151+ 'TestReadFromPubSub' : test_pubsub_read
129152}
130-
131- INPUT_TRANSFORM_TEST_PROVIDERS = ['TestReadFromKafka' ]
153+ """
154+ Transforms not requiring inputs.
155+ """
156+ INPUT_TRANSFORM_TEST_PROVIDERS = ['TestReadFromKafka' , 'TestReadFromPubSub' ]
132157
133158
134159def check_output (expected : List [str ]):
@@ -186,6 +211,7 @@ def test_yaml_example(self):
186211 f"Missing '# Expected:' tag in example file '{ pipeline_spec_file } '" )
187212 for i , line in enumerate (expected ):
188213 expected [i ] = line .replace ('# ' , '' ).replace ('\n ' , '' )
214+ expected = [line for line in expected if line ]
189215 pipeline_spec = yaml .load (
190216 '' .join (lines ), Loader = yaml_transform .SafeLineLoader )
191217
@@ -418,7 +444,11 @@ def _kafka_test_preprocessor(
418444 'test_kafka_yaml' ,
419445 'test_spanner_read_yaml' ,
420446 'test_spanner_write_yaml' ,
421- 'test_enrich_spanner_with_bigquery_yaml'
447+ 'test_enrich_spanner_with_bigquery_yaml' ,
448+ 'test_pubsub_topic_to_bigquery_yaml' ,
449+ 'test_pubsub_subscription_to_bigquery_yaml' ,
450+ 'test_jdbc_to_bigquery_yaml' ,
451+ 'test_spanner_to_avro_yaml'
422452])
423453def _io_write_test_preprocessor (
424454 test_spec : dict , expected : List [str ], env : TestEnvironment ):
@@ -527,8 +557,11 @@ def _iceberg_io_read_test_preprocessor(
527557 return test_spec
528558
529559
530- @YamlExamplesTestSuite .register_test_preprocessor (
531- ['test_spanner_read_yaml' , 'test_enrich_spanner_with_bigquery_yaml' ])
560+ @YamlExamplesTestSuite .register_test_preprocessor ([
561+ 'test_spanner_read_yaml' ,
562+ 'test_enrich_spanner_with_bigquery_yaml' ,
563+ "test_spanner_to_avro_yaml"
564+ ])
532565def _spanner_io_read_test_preprocessor (
533566 test_spec : dict , expected : List [str ], env : TestEnvironment ):
534567 """
@@ -607,6 +640,71 @@ def _enrichment_test_preprocessor(
607640 return test_spec
608641
609642
643+ @YamlExamplesTestSuite .register_test_preprocessor ([
644+ 'test_pubsub_topic_to_bigquery_yaml' ,
645+ 'test_pubsub_subscription_to_bigquery_yaml'
646+ ])
647+ def _pubsub_io_read_test_preprocessor (
648+ test_spec : dict , expected : List [str ], env : TestEnvironment ):
649+ """
650+ Preprocessor for tests that involve reading from Pub/Sub.
651+
652+ This preprocessor replaces any ReadFromPubSub transform with a Create
653+ transform that reads from a predefined in-memory list of messages.
654+ This allows the test to verify the pipeline's correctness without relying
655+ on an active Pub/Sub subscription or topic.
656+ """
657+ if pipeline := test_spec .get ('pipeline' , None ):
658+ for transform in pipeline .get ('transforms' , []):
659+ if transform .get ('type' , '' ) == 'ReadFromPubSub' :
660+ transform ['type' ] = 'TestReadFromPubSub'
661+
662+ return test_spec
663+
664+
665+ @YamlExamplesTestSuite .register_test_preprocessor ([
666+ 'test_jdbc_to_bigquery_yaml' ,
667+ ])
668+ def _jdbc_io_read_test_preprocessor (
669+ test_spec : dict , expected : List [str ], env : TestEnvironment ):
670+ """
671+ Preprocessor for tests that involve reading from JDBC.
672+
673+ This preprocessor replaces any ReadFromJdbc transform with a Create
674+ transform that reads from a predefined in-memory list of records.
675+ This allows the test to verify the pipeline's correctness without
676+ relying on an active JDBC connection.
677+ """
678+ if pipeline := test_spec .get ('pipeline' , None ):
679+ for transform in pipeline .get ('transforms' , []):
680+ if transform .get ('type' , '' ).startswith ('ReadFromJdbc' ):
681+ config = transform ['config' ]
682+ url = config ['url' ]
683+ database = url .split ('/' )[- 1 ]
684+ if (table := config .get ('table' , None )) is None :
685+ table = config .get ('query' , '' ).split ('FROM' )[- 1 ].strip ()
686+ transform ['type' ] = 'Create'
687+ transform ['config' ] = {
688+ k : v
689+ for k , v in config .items () if k .startswith ('__' )
690+ }
691+ elements = INPUT_TABLES [("Jdbc" , database , table )]
692+ if config .get ('query' , None ):
693+ config ['query' ].replace ('select ' ,
694+ 'SELECT ' ).replace (' from ' , ' FROM ' )
695+ columns = set (
696+ '' .join (config ['query' ].split ('SELECT ' )[1 :]).split (
697+ ' FROM' , maxsplit = 1 )[0 ].split (', ' ))
698+ if columns != {'*' }:
699+ elements = [{
700+ column : element [column ]
701+ for column in element if column in columns
702+ } for element in elements ]
703+ transform ['config' ]['elements' ] = elements
704+
705+ return test_spec
706+
707+
610708INPUT_FILES = {'products.csv' : input_data .products_csv ()}
611709INPUT_TABLES = {
612710 ('shipment-test' , 'shipment' , 'shipments' ): input_data .
@@ -616,16 +714,17 @@ def _enrichment_test_preprocessor(
616714 ('db' , 'users' , 'NY' ): input_data .iceberg_dynamic_destinations_users_data (),
617715 ('BigTable' , 'beam-test' , 'bigtable-enrichment-test' ): input_data .
618716 bigtable_data (),
619- ('BigQuery' , 'ALL_TEST' , 'customers' ): input_data .bigquery_data ()
717+ ('BigQuery' , 'ALL_TEST' , 'customers' ): input_data .bigquery_data (),
718+ ('Jdbc' , 'shipment' , 'shipments' ): input_data .jdbc_shipments_data ()
620719}
621720YAML_DOCS_DIR = os .path .join (os .path .dirname (__file__ ))
622721
623722AggregationTest = YamlExamplesTestSuite (
624723 'AggregationExamplesTest' ,
625724 os .path .join (YAML_DOCS_DIR , '../transforms/aggregation/*.yaml' )).run ()
626- BlueprintsTest = YamlExamplesTestSuite (
627- 'BlueprintsExamplesTest ' ,
628- os .path .join (YAML_DOCS_DIR , '../transforms/blueprints /*.yaml' )).run ()
725+ BlueprintTest = YamlExamplesTestSuite (
726+ 'BlueprintExamplesTest ' ,
727+ os .path .join (YAML_DOCS_DIR , '../transforms/blueprint /*.yaml' )).run ()
629728ElementWiseTest = YamlExamplesTestSuite (
630729 'ElementwiseExamplesTest' ,
631730 os .path .join (YAML_DOCS_DIR , '../transforms/elementwise/*.yaml' )).run ()
0 commit comments